From ef72eeeecf4088409202462e1f1a015e2653592e Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 4 Apr 2025 17:26:56 +0800 Subject: [PATCH 01/74] fix the topic level segment.bytes error --- .../storage/internals/log/LogConfig.java | 2 +- .../tools/ConfigCommandIntegrationTest.java | 32 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 2e89cdee21d40..8ec2a9e32dd23 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -194,7 +194,7 @@ public Optional serverConfigName(String configName) { private static final LogConfigDef CONFIG = new LogConfigDef(); static { CONFIG. - define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), MEDIUM, + define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(1024 * 1024), MEDIUM, TopicConfig.SEGMENT_BYTES_DOC) .define(TopicConfig.SEGMENT_MS_CONFIG, LONG, DEFAULT_SEGMENT_MS, atLeast(1), MEDIUM, TopicConfig.SEGMENT_MS_DOC) .define(TopicConfig.SEGMENT_JITTER_MS_CONFIG, LONG, DEFAULT_SEGMENT_JITTER_MS, atLeast(0), MEDIUM, diff --git a/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java index 2587a6c150f9e..f64aa7d91a409 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java @@ -431,6 +431,38 @@ public void testUnsupportedVersionException() { } } + @ClusterTest( + serverProperties = {@ClusterConfigProperty(key = "log.segment.bytes", value = "1048577")} + ) + public void testAlterInvalidLogSegmentBytes() throws Exception { + try (Admin client = cluster.admin()) { + cluster.createTopic("test", 1, (short) 1); + TestUtils.waitForCondition( + () -> cluster.brokerSocketServers() + .stream() + .allMatch(broker -> broker.config().getInt("log.segment.bytes") == 1048577), + "Timeout waiting for topic config propagating to broker" + ); + + ConfigCommand.ConfigCommandOptions command = new ConfigCommand.ConfigCommandOptions( + toArray(asList("--bootstrap-server", cluster.bootstrapServers(), + "--alter", + "--topic", "test", + "--add-config", "segment.bytes=1000" + )) + ); + + Throwable exception = + assertThrows(ExecutionException.class, () -> ConfigCommand.alterConfig(client, command)).getCause(); + + assertInstanceOf(InvalidConfigurationException.class, exception); + assertEquals( + "Invalid value 1000 for configuration segment.bytes: Value must be at least 1048576", + exception.getMessage() + ); + } + } + private void assertNonZeroStatusExit(Stream args, Consumer checkErrOut) { AtomicReference exitStatus = new AtomicReference<>(); Exit.setExitProcedure((status, __) -> { From 36e59c8af8ed8b1a010d3b92d7b52b67e0313606 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 4 Apr 2025 17:29:31 +0800 Subject: [PATCH 02/74] use spotlessApply --- .../java/org/apache/kafka/storage/internals/log/LogConfig.java | 1 - 1 file changed, 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 8ec2a9e32dd23..1287c1a071fd0 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.record.LegacyRecord; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ConfigUtils; From 7a29f51304d07d4dc753a45816114526ecc47fbf Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 4 Apr 2025 19:50:08 +0800 Subject: [PATCH 03/74] update test default value --- .../unit/kafka/log/AbstractLogCleanerIntegrationTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index 98b8dafbc5b91..6e5454ad976f6 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -47,7 +47,7 @@ abstract class AbstractLogCleanerIntegrationTest { private val defaultMinCleanableDirtyRatio = 0.0F private val defaultMinCompactionLagMS = 0L private val defaultDeleteDelay = 1000 - private val defaultSegmentSize = 2048 + private val defaultSegmentSize = 1024 * 1024 private val defaultMaxCompactionLagMs = Long.MaxValue def time: MockTime From 1d66fb05a06f8c0dea4d23c53752805a97ad0d5c Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 4 Apr 2025 20:28:15 +0800 Subject: [PATCH 04/74] wip --- core/src/test/scala/unit/kafka/log/LogManagerTest.scala | 9 ++++----- core/src/test/scala/unit/kafka/log/LogTestUtils.scala | 3 --- core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala | 4 ++-- 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 5e721596ce057..4ef6013ef7f50 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -62,7 +62,7 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 60 * 1000 val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024 * 1024: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) val logConfig = new LogConfig(logProps) @@ -147,7 +147,7 @@ class LogManagerTest { assertEquals(1, logManager.liveLogDirs.size) val logFile = new File(logDir, name + "-0") assertTrue(logFile.exists) - assertFalse(logManager.directoryId(logFile.getParent).equals(DirectoryId.random())) + assertFalse(logManager.directoryId(logFile.getParent).get.equals(DirectoryId.random())) log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), 0) } @@ -868,8 +868,7 @@ class LogManagerTest { logMetrics.foreach { gauge => assertEquals(0, gauge.value()) } } - private def verifyRemainingSegmentsToRecoverMetric(spyLogManager: LogManager, - logDirs: Seq[File], + private def verifyRemainingSegmentsToRecoverMetric(logDirs: Seq[File], recoveryThreadsPerDataDir: Int, mockMap: ConcurrentHashMap[String, Integer], expectedParams: Map[String, Int]): Unit = { @@ -994,7 +993,7 @@ class LogManagerTest { val expectedRemainingSegmentsParams = Map[String, Int]( logDir1.getAbsolutePath -> expectedSegmentsPerLog, logDir2.getAbsolutePath -> expectedSegmentsPerLog) - verifyRemainingSegmentsToRecoverMetric(spyLogManager, logDirs, recoveryThreadsPerDataDir, mockMap, expectedRemainingSegmentsParams) + verifyRemainingSegmentsToRecoverMetric(logDirs, recoveryThreadsPerDataDir, mockMap, expectedRemainingSegmentsParams) } @Test diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index b6909848253d8..0341ab98e0abc 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -17,8 +17,6 @@ package kafka.log -import kafka.log.remote.RemoteLogManager - import java.io.File import java.util.Properties import kafka.utils.TestUtils @@ -107,7 +105,6 @@ object LogTestUtils { topicId: Option[Uuid] = None, numRemainingSegments: ConcurrentMap[String, Integer] = new ConcurrentHashMap[String, Integer], remoteStorageSystemEnable: Boolean = false, - remoteLogManager: Option[RemoteLogManager] = None, logOffsetsListener: LogOffsetsListener = LogOffsetsListener.NO_OP_OFFSETS_LISTENER): UnifiedLog = { UnifiedLog.create( dir, diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index a94de9a14d8cd..98d74898b3276 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -3668,7 +3668,7 @@ class UnifiedLogTest { new SimpleRecord("b".getBytes), new SimpleRecord("c".getBytes)) - val logConfig = LogTestUtils.createLogConfig(segmentBytes = records.sizeInBytes) + val logConfig = LogTestUtils.createLogConfig() val log = createLog(logDir, logConfig) val firstAppendInfo = log.appendAsLeader(records, 0) @@ -4681,7 +4681,7 @@ class UnifiedLogTest { val log = LogTestUtils.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint, maxTransactionTimeoutMs, producerStateManagerConfig, producerIdExpirationCheckIntervalMs, lastShutdownClean, topicId, new ConcurrentHashMap[String, Integer], - remoteStorageSystemEnable, remoteLogManager, logOffsetsListener) + remoteStorageSystemEnable, logOffsetsListener) logsToClose = logsToClose :+ log log } From 9398baec63b96ae58f3b2eda6ab04bdb0c7a3ffc Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 9 Apr 2025 20:23:40 +0800 Subject: [PATCH 05/74] revert test change --- .../kafka/log/AbstractLogCleanerIntegrationTest.scala | 2 +- core/src/test/scala/unit/kafka/log/LogManagerTest.scala | 9 +++++---- core/src/test/scala/unit/kafka/log/LogTestUtils.scala | 3 +++ core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala | 4 ++-- 4 files changed, 11 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index 6e5454ad976f6..98b8dafbc5b91 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -47,7 +47,7 @@ abstract class AbstractLogCleanerIntegrationTest { private val defaultMinCleanableDirtyRatio = 0.0F private val defaultMinCompactionLagMS = 0L private val defaultDeleteDelay = 1000 - private val defaultSegmentSize = 1024 * 1024 + private val defaultSegmentSize = 2048 private val defaultMaxCompactionLagMs = Long.MaxValue def time: MockTime diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 4ef6013ef7f50..5e721596ce057 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -62,7 +62,7 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 60 * 1000 val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024 * 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) val logConfig = new LogConfig(logProps) @@ -147,7 +147,7 @@ class LogManagerTest { assertEquals(1, logManager.liveLogDirs.size) val logFile = new File(logDir, name + "-0") assertTrue(logFile.exists) - assertFalse(logManager.directoryId(logFile.getParent).get.equals(DirectoryId.random())) + assertFalse(logManager.directoryId(logFile.getParent).equals(DirectoryId.random())) log.appendAsLeader(TestUtils.singletonRecords("test".getBytes()), 0) } @@ -868,7 +868,8 @@ class LogManagerTest { logMetrics.foreach { gauge => assertEquals(0, gauge.value()) } } - private def verifyRemainingSegmentsToRecoverMetric(logDirs: Seq[File], + private def verifyRemainingSegmentsToRecoverMetric(spyLogManager: LogManager, + logDirs: Seq[File], recoveryThreadsPerDataDir: Int, mockMap: ConcurrentHashMap[String, Integer], expectedParams: Map[String, Int]): Unit = { @@ -993,7 +994,7 @@ class LogManagerTest { val expectedRemainingSegmentsParams = Map[String, Int]( logDir1.getAbsolutePath -> expectedSegmentsPerLog, logDir2.getAbsolutePath -> expectedSegmentsPerLog) - verifyRemainingSegmentsToRecoverMetric(logDirs, recoveryThreadsPerDataDir, mockMap, expectedRemainingSegmentsParams) + verifyRemainingSegmentsToRecoverMetric(spyLogManager, logDirs, recoveryThreadsPerDataDir, mockMap, expectedRemainingSegmentsParams) } @Test diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index 0341ab98e0abc..b6909848253d8 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -17,6 +17,8 @@ package kafka.log +import kafka.log.remote.RemoteLogManager + import java.io.File import java.util.Properties import kafka.utils.TestUtils @@ -105,6 +107,7 @@ object LogTestUtils { topicId: Option[Uuid] = None, numRemainingSegments: ConcurrentMap[String, Integer] = new ConcurrentHashMap[String, Integer], remoteStorageSystemEnable: Boolean = false, + remoteLogManager: Option[RemoteLogManager] = None, logOffsetsListener: LogOffsetsListener = LogOffsetsListener.NO_OP_OFFSETS_LISTENER): UnifiedLog = { UnifiedLog.create( dir, diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 95015838f0af9..257f777885b20 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -3668,7 +3668,7 @@ class UnifiedLogTest { new SimpleRecord("b".getBytes), new SimpleRecord("c".getBytes)) - val logConfig = LogTestUtils.createLogConfig() + val logConfig = LogTestUtils.createLogConfig(segmentBytes = records.sizeInBytes) val log = createLog(logDir, logConfig) val firstAppendInfo = log.appendAsLeader(records, 0) @@ -4681,7 +4681,7 @@ class UnifiedLogTest { val log = LogTestUtils.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint, maxTransactionTimeoutMs, producerStateManagerConfig, producerIdExpirationCheckIntervalMs, lastShutdownClean, topicId, new ConcurrentHashMap[String, Integer], - remoteStorageSystemEnable, logOffsetsListener) + remoteStorageSystemEnable, remoteLogManager, logOffsetsListener) logsToClose = logsToClose :+ log log } From 0f766c5d467fc0d35ca4d8949fe478440c75adb4 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 9 Apr 2025 20:36:30 +0800 Subject: [PATCH 06/74] add new constructor for test --- .../scala/unit/kafka/log/LogTestUtils.scala | 2 +- .../storage/internals/log/LogConfig.java | 47 +++++++++++++++++++ 2 files changed, 48 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index b6909848253d8..4b6fa65c4434f 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -76,7 +76,6 @@ object LogTestUtils { remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs: java.lang.Long) logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) @@ -91,6 +90,7 @@ object LogTestUtils { logProps.put(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, remoteLogCopyDisable: java.lang.Boolean) logProps.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, remoteLogDeleteOnDisable: java.lang.Boolean) new LogConfig(logProps) + new LogConfig(logProps, segmentBytes) } def createLog(dir: File, diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 1287c1a071fd0..9b59d7bd35a9a 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -301,6 +301,53 @@ public LogConfig(Map props) { this(props, Set.of()); } + // visible for testing + public LogConfig(Map props, Integer segmentBytes) { + super(CONFIG, props, false); + if (props.containsKey(TopicConfig.SEGMENT_BYTES_CONFIG)) { + throw new IllegalArgumentException("This constructor should not be used when " + + TopicConfig.SEGMENT_BYTES_CONFIG + " is present in the properties"); + } + this.props = Collections.unmodifiableMap(props); + this.overriddenConfigs = Set.of(); + + this.segmentSize = segmentBytes; + this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); + this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); + this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); + this.flushInterval = getLong(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG); + this.flushMs = getLong(TopicConfig.FLUSH_MS_CONFIG); + this.retentionSize = getLong(TopicConfig.RETENTION_BYTES_CONFIG); + this.retentionMs = getLong(TopicConfig.RETENTION_MS_CONFIG); + this.maxMessageSize = getInt(TopicConfig.MAX_MESSAGE_BYTES_CONFIG); + this.indexInterval = getInt(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG); + this.fileDeleteDelayMs = getLong(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG); + this.deleteRetentionMs = getLong(TopicConfig.DELETE_RETENTION_MS_CONFIG); + this.compactionLagMs = getLong(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG); + this.maxCompactionLagMs = getLong(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG); + this.minCleanableRatio = getDouble(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG); + this.compact = getList(TopicConfig.CLEANUP_POLICY_CONFIG).stream() + .map(c -> c.toLowerCase(Locale.ROOT)) + .toList() + .contains(TopicConfig.CLEANUP_POLICY_COMPACT); + this.delete = getList(TopicConfig.CLEANUP_POLICY_CONFIG).stream() + .map(c -> c.toLowerCase(Locale.ROOT)) + .toList() + .contains(TopicConfig.CLEANUP_POLICY_DELETE); + this.uncleanLeaderElectionEnable = getBoolean(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG); + this.minInSyncReplicas = getInt(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG); + this.compressionType = BrokerCompressionType.forName(getString(TopicConfig.COMPRESSION_TYPE_CONFIG)); + this.compression = getCompression(); + this.preallocate = getBoolean(TopicConfig.PREALLOCATE_CONFIG); + this.messageTimestampType = TimestampType.forName(getString(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG)); + this.messageTimestampBeforeMaxMs = getLong(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG); + this.messageTimestampAfterMaxMs = getLong(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG); + this.leaderReplicationThrottledReplicas = Collections.unmodifiableList(getList(QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG)); + this.followerReplicationThrottledReplicas = Collections.unmodifiableList(getList(QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG)); + + remoteLogConfig = new RemoteLogConfig(this); + } + @SuppressWarnings({"this-escape"}) public LogConfig(Map props, Set overriddenConfigs) { super(CONFIG, props, false); From a0342edfe9994e7c4a6cd89334242c834b07d5bc Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 9 Apr 2025 20:41:22 +0800 Subject: [PATCH 07/74] add escape annotation --- .../java/org/apache/kafka/storage/internals/log/LogConfig.java | 1 + 1 file changed, 1 insertion(+) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 9b59d7bd35a9a..eae29501e95af 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -302,6 +302,7 @@ public LogConfig(Map props) { } // visible for testing + @SuppressWarnings({"this-escape"}) public LogConfig(Map props, Integer segmentBytes) { super(CONFIG, props, false); if (props.containsKey(TopicConfig.SEGMENT_BYTES_CONFIG)) { From f70746576ee4062b9e73e5915060901be045d118 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 9 Apr 2025 22:36:09 +0800 Subject: [PATCH 08/74] fix some test --- .../scala/kafka/raft/KafkaMetadataLog.scala | 67 +++++++++++++++++++ .../kafka/raft/KafkaMetadataLogTest.scala | 3 +- .../kafka/cluster/AbstractPartitionTest.scala | 2 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 2 +- .../scala/unit/kafka/log/LogManagerTest.scala | 2 +- .../kafka/tools/DumpLogSegmentsTest.scala | 25 ++++--- .../storage/internals/log/LogConfig.java | 6 +- 7 files changed, 90 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index ebc2f3c81cf5d..41d24b8c87c72 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -647,6 +647,73 @@ object KafkaMetadataLog extends Logging { metadataLog } + // only for testing + def apply( + topicPartition: TopicPartition, + topicId: Uuid, + dataDir: File, + time: Time, + scheduler: Scheduler, + config: MetadataLogConfig, + logSegmentBytes: Int, + ): KafkaMetadataLog = { + val props = new Properties() + props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.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") + LogConfig.validate(props) + val defaultLogConfig = new LogConfig(props, logSegmentBytes) + + 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}." + ) + } + + val log = UnifiedLog.create( + dataDir, + defaultLogConfig, + 0L, + 0L, + scheduler, + new BrokerTopicStats, + time, + Integer.MAX_VALUE, + new ProducerStateManagerConfig(Integer.MAX_VALUE, false), + Integer.MAX_VALUE, + new LogDirFailureChannel(5), + false, + Optional.of(topicId) + ) + + val metadataLog = new KafkaMetadataLog( + log, + time, + scheduler, + recoverSnapshots(log), + topicPartition, + config + ) + + // 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 recoverSnapshots( log: UnifiedLog ): mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]] = { diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 466fb954d133f..589d74fe1ab46 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -1133,7 +1133,8 @@ object KafkaMetadataLogTest { logDir, time, time.scheduler, - metadataLogConfig + metadataLogConfig, + metadataLogConfig.maxBatchSizeInBytes ) (logDir.toPath, metadataLog, metadataLogConfig) diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index d9eaa7b2aacc0..bd760d48c4c6a 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -69,7 +69,7 @@ class AbstractPartitionTest { TestUtils.clearYammerMetrics() val logProps = createLogProperties(Map.empty) - logConfig = new LogConfig(logProps) + logConfig = new LogConfig(logProps, logProps.get(TopicConfig.SEGMENT_BYTES_CONFIG).asInstanceOf[Int]) configRepository = MockConfigRepository.forTopic(topicPartition.topic, logProps) tmpDir = TestUtils.tempDir() diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 049e1793a139e..14ea70e8fdd39 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -59,7 +59,7 @@ class LogCleanerTest extends Logging { logProps.put(TopicConfig.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 = new LogConfig(logProps) + val logConfig = new LogConfig(logProps, 1024) val time = new MockTime() val throttler = new Throttler(Double.MaxValue, Long.MaxValue, "throttler", "entries", time) val tombstoneRetentionMs = 86400000 diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 5e721596ce057..e62490bb6d222 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -65,7 +65,7 @@ class LogManagerTest { logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) - val logConfig = new LogConfig(logProps) + val logConfig = new LogConfig(logProps, 1024) var logDir: File = _ var logManager: LogManager = _ val name = "kafka" diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 11d33787dd6e4..c16e38dc099d0 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -538,23 +538,26 @@ class DumpLogSegmentsTest { setPartitionId(0).setIsr(util.Arrays.asList(0, 1, 2)), 0.toShort) ) + val metadataLogConfig = new MetadataLogConfig( + 100 * 1024, + 100 * 1024, + 10 * 1000, + 100 * 1024, + 60 * 1000, + KafkaRaftClient.MAX_BATCH_SIZE_BYTES, + KafkaRaftClient.MAX_FETCH_SIZE_BYTES, + ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, + 1 + ) + val metadataLog = KafkaMetadataLog( KafkaRaftServer.MetadataPartition, KafkaRaftServer.MetadataTopicId, logDir, time, time.scheduler, - new MetadataLogConfig( - 100 * 1024, - 100 * 1024, - 10 * 1000, - 100 * 1024, - 60 * 1000, - KafkaRaftClient.MAX_BATCH_SIZE_BYTES, - KafkaRaftClient.MAX_FETCH_SIZE_BYTES, - ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, - 1 - ) + metadataLogConfig, + metadataLogConfig.logSegmentBytes() ) val lastContainedLogTimestamp = 10000 diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index eae29501e95af..54d8a9ae0e0fe 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -302,14 +302,16 @@ public LogConfig(Map props) { } // visible for testing - @SuppressWarnings({"this-escape"}) + @SuppressWarnings({"this-escape", "unchecked"}) public LogConfig(Map props, Integer segmentBytes) { super(CONFIG, props, false); if (props.containsKey(TopicConfig.SEGMENT_BYTES_CONFIG)) { throw new IllegalArgumentException("This constructor should not be used when " + TopicConfig.SEGMENT_BYTES_CONFIG + " is present in the properties"); } - this.props = Collections.unmodifiableMap(props); + Map typedProps = (Map) props; + typedProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes); + this.props = Map.copyOf(typedProps); this.overriddenConfigs = Set.of(); this.segmentSize = segmentBytes; From 768c8ed7e021917454d227b5af4eb598c45a4f80 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 10 Apr 2025 19:02:44 +0800 Subject: [PATCH 09/74] Revert "fix some test" This reverts commit f70746576ee4062b9e73e5915060901be045d118. --- .../scala/kafka/raft/KafkaMetadataLog.scala | 67 ------------------- .../kafka/raft/KafkaMetadataLogTest.scala | 3 +- .../kafka/cluster/AbstractPartitionTest.scala | 2 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 2 +- .../scala/unit/kafka/log/LogManagerTest.scala | 2 +- .../kafka/tools/DumpLogSegmentsTest.scala | 25 +++---- .../storage/internals/log/LogConfig.java | 6 +- 7 files changed, 17 insertions(+), 90 deletions(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 41d24b8c87c72..ebc2f3c81cf5d 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -647,73 +647,6 @@ object KafkaMetadataLog extends Logging { metadataLog } - // only for testing - def apply( - topicPartition: TopicPartition, - topicId: Uuid, - dataDir: File, - time: Time, - scheduler: Scheduler, - config: MetadataLogConfig, - logSegmentBytes: Int, - ): KafkaMetadataLog = { - val props = new Properties() - props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.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") - LogConfig.validate(props) - val defaultLogConfig = new LogConfig(props, logSegmentBytes) - - 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}." - ) - } - - val log = UnifiedLog.create( - dataDir, - defaultLogConfig, - 0L, - 0L, - scheduler, - new BrokerTopicStats, - time, - Integer.MAX_VALUE, - new ProducerStateManagerConfig(Integer.MAX_VALUE, false), - Integer.MAX_VALUE, - new LogDirFailureChannel(5), - false, - Optional.of(topicId) - ) - - val metadataLog = new KafkaMetadataLog( - log, - time, - scheduler, - recoverSnapshots(log), - topicPartition, - config - ) - - // 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 recoverSnapshots( log: UnifiedLog ): mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]] = { diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 589d74fe1ab46..466fb954d133f 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -1133,8 +1133,7 @@ object KafkaMetadataLogTest { logDir, time, time.scheduler, - metadataLogConfig, - metadataLogConfig.maxBatchSizeInBytes + metadataLogConfig ) (logDir.toPath, metadataLog, metadataLogConfig) diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index bd760d48c4c6a..d9eaa7b2aacc0 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -69,7 +69,7 @@ class AbstractPartitionTest { TestUtils.clearYammerMetrics() val logProps = createLogProperties(Map.empty) - logConfig = new LogConfig(logProps, logProps.get(TopicConfig.SEGMENT_BYTES_CONFIG).asInstanceOf[Int]) + logConfig = new LogConfig(logProps) configRepository = MockConfigRepository.forTopic(topicPartition.topic, logProps) tmpDir = TestUtils.tempDir() diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 14ea70e8fdd39..049e1793a139e 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -59,7 +59,7 @@ class LogCleanerTest extends Logging { logProps.put(TopicConfig.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 = new LogConfig(logProps, 1024) + val logConfig = new LogConfig(logProps) val time = new MockTime() val throttler = new Throttler(Double.MaxValue, Long.MaxValue, "throttler", "entries", time) val tombstoneRetentionMs = 86400000 diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index e62490bb6d222..5e721596ce057 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -65,7 +65,7 @@ class LogManagerTest { logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) - val logConfig = new LogConfig(logProps, 1024) + val logConfig = new LogConfig(logProps) var logDir: File = _ var logManager: LogManager = _ val name = "kafka" diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index c16e38dc099d0..11d33787dd6e4 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -538,26 +538,23 @@ class DumpLogSegmentsTest { setPartitionId(0).setIsr(util.Arrays.asList(0, 1, 2)), 0.toShort) ) - val metadataLogConfig = new MetadataLogConfig( - 100 * 1024, - 100 * 1024, - 10 * 1000, - 100 * 1024, - 60 * 1000, - KafkaRaftClient.MAX_BATCH_SIZE_BYTES, - KafkaRaftClient.MAX_FETCH_SIZE_BYTES, - ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, - 1 - ) - val metadataLog = KafkaMetadataLog( KafkaRaftServer.MetadataPartition, KafkaRaftServer.MetadataTopicId, logDir, time, time.scheduler, - metadataLogConfig, - metadataLogConfig.logSegmentBytes() + new MetadataLogConfig( + 100 * 1024, + 100 * 1024, + 10 * 1000, + 100 * 1024, + 60 * 1000, + KafkaRaftClient.MAX_BATCH_SIZE_BYTES, + KafkaRaftClient.MAX_FETCH_SIZE_BYTES, + ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, + 1 + ) ) val lastContainedLogTimestamp = 10000 diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 54d8a9ae0e0fe..eae29501e95af 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -302,16 +302,14 @@ public LogConfig(Map props) { } // visible for testing - @SuppressWarnings({"this-escape", "unchecked"}) + @SuppressWarnings({"this-escape"}) public LogConfig(Map props, Integer segmentBytes) { super(CONFIG, props, false); if (props.containsKey(TopicConfig.SEGMENT_BYTES_CONFIG)) { throw new IllegalArgumentException("This constructor should not be used when " + TopicConfig.SEGMENT_BYTES_CONFIG + " is present in the properties"); } - Map typedProps = (Map) props; - typedProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes); - this.props = Map.copyOf(typedProps); + this.props = Collections.unmodifiableMap(props); this.overriddenConfigs = Set.of(); this.segmentSize = segmentBytes; From 640ca6b63eeb2610615b79a07dd4a41eb8001b94 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 10 Apr 2025 19:02:44 +0800 Subject: [PATCH 10/74] Revert "add escape annotation" This reverts commit a0342edfe9994e7c4a6cd89334242c834b07d5bc. --- .../java/org/apache/kafka/storage/internals/log/LogConfig.java | 1 - 1 file changed, 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index eae29501e95af..9b59d7bd35a9a 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -302,7 +302,6 @@ public LogConfig(Map props) { } // visible for testing - @SuppressWarnings({"this-escape"}) public LogConfig(Map props, Integer segmentBytes) { super(CONFIG, props, false); if (props.containsKey(TopicConfig.SEGMENT_BYTES_CONFIG)) { From 2bfd478c1d783dfb119bd3d067e843ba9b57342f Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 10 Apr 2025 19:02:44 +0800 Subject: [PATCH 11/74] Revert "add new constructor for test" This reverts commit 0f766c5d467fc0d35ca4d8949fe478440c75adb4. --- .../scala/unit/kafka/log/LogTestUtils.scala | 2 +- .../storage/internals/log/LogConfig.java | 47 ------------------- 2 files changed, 1 insertion(+), 48 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index 4b6fa65c4434f..b6909848253d8 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -76,6 +76,7 @@ object LogTestUtils { remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs: java.lang.Long) logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) @@ -90,7 +91,6 @@ object LogTestUtils { logProps.put(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, remoteLogCopyDisable: java.lang.Boolean) logProps.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, remoteLogDeleteOnDisable: java.lang.Boolean) new LogConfig(logProps) - new LogConfig(logProps, segmentBytes) } def createLog(dir: File, diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 9b59d7bd35a9a..1287c1a071fd0 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -301,53 +301,6 @@ public LogConfig(Map props) { this(props, Set.of()); } - // visible for testing - public LogConfig(Map props, Integer segmentBytes) { - super(CONFIG, props, false); - if (props.containsKey(TopicConfig.SEGMENT_BYTES_CONFIG)) { - throw new IllegalArgumentException("This constructor should not be used when " + - TopicConfig.SEGMENT_BYTES_CONFIG + " is present in the properties"); - } - this.props = Collections.unmodifiableMap(props); - this.overriddenConfigs = Set.of(); - - this.segmentSize = segmentBytes; - this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); - this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); - this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); - this.flushInterval = getLong(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG); - this.flushMs = getLong(TopicConfig.FLUSH_MS_CONFIG); - this.retentionSize = getLong(TopicConfig.RETENTION_BYTES_CONFIG); - this.retentionMs = getLong(TopicConfig.RETENTION_MS_CONFIG); - this.maxMessageSize = getInt(TopicConfig.MAX_MESSAGE_BYTES_CONFIG); - this.indexInterval = getInt(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG); - this.fileDeleteDelayMs = getLong(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG); - this.deleteRetentionMs = getLong(TopicConfig.DELETE_RETENTION_MS_CONFIG); - this.compactionLagMs = getLong(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG); - this.maxCompactionLagMs = getLong(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG); - this.minCleanableRatio = getDouble(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG); - this.compact = getList(TopicConfig.CLEANUP_POLICY_CONFIG).stream() - .map(c -> c.toLowerCase(Locale.ROOT)) - .toList() - .contains(TopicConfig.CLEANUP_POLICY_COMPACT); - this.delete = getList(TopicConfig.CLEANUP_POLICY_CONFIG).stream() - .map(c -> c.toLowerCase(Locale.ROOT)) - .toList() - .contains(TopicConfig.CLEANUP_POLICY_DELETE); - this.uncleanLeaderElectionEnable = getBoolean(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG); - this.minInSyncReplicas = getInt(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG); - this.compressionType = BrokerCompressionType.forName(getString(TopicConfig.COMPRESSION_TYPE_CONFIG)); - this.compression = getCompression(); - this.preallocate = getBoolean(TopicConfig.PREALLOCATE_CONFIG); - this.messageTimestampType = TimestampType.forName(getString(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG)); - this.messageTimestampBeforeMaxMs = getLong(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG); - this.messageTimestampAfterMaxMs = getLong(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG); - this.leaderReplicationThrottledReplicas = Collections.unmodifiableList(getList(QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG)); - this.followerReplicationThrottledReplicas = Collections.unmodifiableList(getList(QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG)); - - remoteLogConfig = new RemoteLogConfig(this); - } - @SuppressWarnings({"this-escape"}) public LogConfig(Map props, Set overriddenConfigs) { super(CONFIG, props, false); From 51da598f200839dbafd3205750e15118d0f4a3a1 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 10 Apr 2025 23:04:13 +0800 Subject: [PATCH 12/74] fix all test --- .../clients/admin/StaticBrokerConfigTest.java | 4 +- .../kafka/common/config/TopicConfig.java | 4 ++ .../src/main/scala/kafka/log/LogCleaner.scala | 2 +- .../api/PlaintextAdminIntegrationTest.scala | 2 +- .../api/SaslSslAdminIntegrationTest.scala | 4 +- .../DynamicBrokerReconfigurationTest.scala | 2 +- .../kafka/cluster/AbstractPartitionTest.scala | 2 +- .../kafka/cluster/PartitionLockTest.scala | 2 +- .../AbstractLogCleanerIntegrationTest.scala | 2 +- .../kafka/log/LogCleanerManagerTest.scala | 18 ++--- .../scala/unit/kafka/log/LogCleanerTest.scala | 72 +++++++++---------- .../unit/kafka/log/LogConcurrencyTest.scala | 2 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 2 +- .../scala/unit/kafka/log/LogManagerTest.scala | 4 +- .../scala/unit/kafka/log/LogTestUtils.scala | 2 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 2 +- .../AlterReplicaLogDirsRequestTest.scala | 2 +- ...ControllerConfigurationValidatorTest.scala | 8 +-- .../server/DynamicConfigChangeTest.scala | 8 +-- .../unit/kafka/server/KafkaConfigTest.scala | 2 +- .../group/GroupCoordinatorServiceTest.java | 2 +- .../ReplicationControlManagerTest.java | 6 +- .../share/ShareCoordinatorServiceTest.java | 2 +- .../storage/internals/log/LogConfig.java | 15 +++- .../storage/internals/log/UnifiedLog.java | 8 +-- .../storage/internals/log/LocalLogTest.java | 2 +- .../PurgeRepartitionTopicIntegrationTest.java | 4 +- .../kafka/streams/StreamsConfigTest.java | 4 +- .../internals/InternalTopicManagerTest.java | 2 +- .../kafka/tools/GetOffsetShellTest.java | 2 +- 30 files changed, 103 insertions(+), 90 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java index 58240c0711e62..8d981d1fb7a01 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java @@ -53,7 +53,7 @@ public void testTopicConfigsGetImpactedIfStaticConfigsAddToController(ClusterIns Admin adminUsingBootstrapController = cluster.admin(Map.of(), true) ) { ConfigEntry configEntry = admin.createTopics(List.of(new NewTopic(TOPIC, 1, (short) 1))) - .config(TOPIC).get().get(TopicConfig.SEGMENT_BYTES_CONFIG); + .config(TOPIC).get().get(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG); assertEquals(ConfigEntry.ConfigSource.STATIC_BROKER_CONFIG, configEntry.source()); assertEquals(CUSTOM_VALUE, configEntry.value(), "Config value should be custom value since controller has related static config"); @@ -86,7 +86,7 @@ public void testTopicConfigsGetImpactedIfStaticConfigsAddToBroker(ClusterInstanc Admin adminUsingBootstrapController = cluster.admin(Map.of(), true) ) { ConfigEntry configEntry = admin.createTopics(List.of(new NewTopic(TOPIC, 1, (short) 1))) - .config(TOPIC).get().get(TopicConfig.SEGMENT_BYTES_CONFIG); + .config(TOPIC).get().get(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG); assertEquals(ConfigEntry.ConfigSource.DEFAULT_CONFIG, configEntry.source()); assertNotEquals(CUSTOM_VALUE, configEntry.value(), diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index d8043e9be9df7..f756d2746da07 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -214,6 +214,9 @@ public class TopicConfig { "configuration. If message.timestamp.type=CreateTime, the message will be rejected if the difference in " + "timestamps exceeds this specified threshold. This configuration is ignored if message.timestamp.type=LogAppendTime."; + // visible for testing + public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; + /** * @deprecated down-conversion is not possible in Apache Kafka 4.0 and newer, hence this configuration is a no-op, * and it is deprecated for removal in Apache Kafka 5.0. @@ -227,4 +230,5 @@ public class TopicConfig { @Deprecated public static final String MESSAGE_DOWNCONVERSION_ENABLE_DOC = "Down-conversion is not possible in Apache Kafka 4.0 and newer, " + "hence this configuration is no-op and it is deprecated for removal in Apache Kafka 5.0."; + } diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 50b160056b5fd..cda596e50696e 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -623,7 +623,7 @@ private[log] class Cleaner(val id: Int, info("Cleaning log %s (cleaning prior to %s, discarding tombstones prior to upper bound deletion horizon %s)...".format(log.name, new Date(cleanableHorizonMs), new Date(legacyDeleteHorizonMs))) val transactionMetadata = new CleanedTransactionMetadata - val groupedSegments = groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize, + val groupedSegments = groupSegmentsBySize(log.logSegments(0, endOffset), log.config.segmentSize(), log.config.maxIndexSize, cleanable.firstUncleanableOffset) for (group <- groupedSegments) cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, legacyDeleteHorizonMs, upperBoundOffset) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index d1e04f8197a5f..1f6035eef7d40 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -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(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "200") createTopic(topic, numPartitions = 1, replicationFactor = 1, config) client = createAdminClient diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 70b65368111ba..3eab51dfa31d9 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -567,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(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG -> "100000").asJava 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)) @@ -580,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 == TopicConfig.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 diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index d5fbdf78de1ae..8708c5bcce636 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -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) => diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index d9eaa7b2aacc0..da4d042874920 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -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(TopicConfig.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) } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 4a3051ddc9567..da899a36809c3 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -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(TopicConfig.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) } diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index 98b8dafbc5b91..8ef39e049798a 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -70,7 +70,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(TopicConfig.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) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 1befc2caa8767..90984cd8e9fd7 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -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(TopicConfig.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) @@ -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(TopicConfig.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) @@ -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(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -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(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -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(TopicConfig.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)) @@ -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(TopicConfig.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)) @@ -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(TopicConfig.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)) @@ -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(TopicConfig.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) @@ -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(TopicConfig.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 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 049e1793a139e..e561ad6881ff9 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -56,7 +56,7 @@ class LogCleanerTest extends Logging { val tmpdir = TestUtils.tempDir() val dir = TestUtils.randomPartitionLogDir(tmpdir) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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 = new LogConfig(logProps) @@ -148,7 +148,7 @@ class LogCleanerTest extends Logging { def testCleanSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -181,7 +181,7 @@ class LogCleanerTest extends Logging { // Construct a log instance. The replaceSegments() method of the log instance is overridden so that // it waits for another thread to execute deleteOldSegments() val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) @@ -271,7 +271,7 @@ class LogCleanerTest extends Logging { val originalMaxFileSize = 1024 val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact": java.lang.String) logProps.put(TopicConfig.PREALLOCATE_CONFIG, "true": java.lang.String) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -295,7 +295,7 @@ class LogCleanerTest extends Logging { def testDuplicateCheckAfterCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) var log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -464,7 +464,7 @@ class LogCleanerTest extends Logging { def testBasicTransactionAwareCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -497,7 +497,7 @@ class LogCleanerTest extends Logging { def testCleanWithTransactionsSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -543,7 +543,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -591,7 +591,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(capacity = Int.MaxValue, maxMessageSize = 100) val logProps = new Properties() logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, 100: java.lang.Integer) - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -619,7 +619,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -682,7 +682,7 @@ class LogCleanerTest extends Logging { def testCleanEmptyControlBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -714,7 +714,7 @@ class LogCleanerTest extends Logging { def testCommittedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -736,7 +736,7 @@ class LogCleanerTest extends Logging { def testAbortedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -765,7 +765,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -799,7 +799,7 @@ class LogCleanerTest extends Logging { val producerId = 1L val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, 0, AppendOrigin.REPLICATION) @@ -832,7 +832,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -896,7 +896,7 @@ class LogCleanerTest extends Logging { // Create cleaner with very small default max message size val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -968,7 +968,7 @@ class LogCleanerTest extends Logging { def createLogWithMessagesLargerThanMaxSize(largeMessageSize: Int): (UnifiedLog, FakeOffsetMap) = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -994,7 +994,7 @@ class LogCleanerTest extends Logging { def testCleaningWithDeletes(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1021,7 +1021,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 3 messages in the map val cleaner = makeCleaner(4) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1049,7 +1049,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsProducerLastSequence(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) log.appendAsLeader(record(0, 0), 0) // offset 0 @@ -1072,7 +1072,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsLastSequenceEvenIfTransactionAborted(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -1106,7 +1106,7 @@ class LogCleanerTest extends Logging { def testCleaningWithKeysConflictingWithTxnMarkerKeys(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val leaderEpoch = 5 val producerEpoch = 0.toShort @@ -1150,7 +1150,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 1 message in the map val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1181,7 +1181,7 @@ class LogCleanerTest extends Logging { def testCleaningWithUncleanableSection(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1224,7 +1224,7 @@ class LogCleanerTest extends Logging { def testLogToClean(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1242,7 +1242,7 @@ class LogCleanerTest extends Logging { def testLogToCleanWithUncleanableSection(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1275,7 +1275,7 @@ class LogCleanerTest extends Logging { // create a log with compaction turned off so we can append unkeyed messages val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1333,7 +1333,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsWithAbort(): Unit = { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1355,7 +1355,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsRetainingLastEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1387,7 +1387,7 @@ class LogCleanerTest extends Logging { def testSegmentGrouping(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1488,7 +1488,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1540,7 +1540,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) //mimic the effect of loading an empty index file logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 400: java.lang.Integer) @@ -1665,7 +1665,7 @@ class LogCleanerTest extends Logging { def testRecoveryAfterCrash(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 10: java.lang.Integer) @@ -1796,7 +1796,7 @@ class LogCleanerTest extends Logging { def testBuildOffsetMapFakeLarge(): Unit = { val map = new FakeOffsetMap(1000) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) @@ -1944,7 +1944,7 @@ class LogCleanerTest extends Logging { @Test def testCleaningBeyondMissingOffsets(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) val cleaner = makeCleaner(Int.MaxValue) diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 0da8366f443ee..0661070f819c7 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -60,7 +60,7 @@ class LogConcurrencyTest { @Test def testUncommittedDataNotConsumedFrequentSegmentRolls(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 237: Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 237: Integer) val logConfig = new LogConfig(logProps) testUncommittedDataNotConsumed(createLog(logConfig)) } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 8e417a695ee36..d942cc47ff945 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -245,7 +245,7 @@ class LogLoaderTest { @Test def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "640") + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "640") val logConfig = new LogConfig(logProps) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 5e721596ce057..7ff8fe6cf71fb 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -62,7 +62,7 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 60 * 1000 val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) val logConfig = new LogConfig(logProps) @@ -393,7 +393,7 @@ class LogManagerTest { logManager.shutdown() val segmentBytes = 10 * setSize val properties = new Properties() - properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes.toString) + properties.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes.toString) properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString) val configRepository = MockConfigRepository.forTopic(name, properties) diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index b6909848253d8..98c40cc610d12 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -76,7 +76,7 @@ object LogTestUtils { remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes: Integer) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs: java.lang.Long) logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 257f777885b20..845ef589cb337 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -2733,7 +2733,7 @@ class UnifiedLogTest { @Test def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000") + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000") logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") val logConfig = new LogConfig(logProps) diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index 8e2698b0842cf..81887d492614d 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -148,7 +148,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { // We don't want files with `.deleted` suffix are removed too fast, // so we can validate there will be orphan files and orphan files will be removed eventually. topicProperties.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, "10000") - topicProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1024") + topicProperties.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1024") createTopic(topic, partitionNum, 1, topicProperties) assertEquals(logDir1, brokers.head.logManager.getLog(tp).get.dir.getParent) diff --git a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala index 3056753f53bf4..671f8facd6ef1 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala @@ -20,7 +20,7 @@ package kafka.server import kafka.utils.TestUtils import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, BROKER_LOGGER, CLIENT_METRICS, GROUP, TOPIC} -import org.apache.kafka.common.config.TopicConfig.{REMOTE_LOG_STORAGE_ENABLE_CONFIG, SEGMENT_BYTES_CONFIG, SEGMENT_JITTER_MS_CONFIG, SEGMENT_MS_CONFIG} +import org.apache.kafka.common.config.TopicConfig.{REMOTE_LOG_STORAGE_ENABLE_CONFIG, INTERNAL_SEGMENT_BYTES_CONFIG, SEGMENT_JITTER_MS_CONFIG, SEGMENT_MS_CONFIG} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, InvalidTopicException} import org.apache.kafka.coordinator.group.GroupConfig import org.apache.kafka.server.metrics.ClientMetricsConfigs @@ -62,7 +62,7 @@ class ControllerConfigurationValidatorTest { def testNullTopicConfigValue(): Unit = { val config = new util.TreeMap[String, String]() config.put(SEGMENT_JITTER_MS_CONFIG, "10") - config.put(SEGMENT_BYTES_CONFIG, null) + config.put(INTERNAL_SEGMENT_BYTES_CONFIG, null) config.put(SEGMENT_MS_CONFIG, null) assertEquals("Null value not supported for topic configs: segment.bytes,segment.ms", assertThrows(classOf[InvalidConfigurationException], () => validator.validate( @@ -73,7 +73,7 @@ class ControllerConfigurationValidatorTest { def testValidTopicConfig(): Unit = { val config = new util.TreeMap[String, String]() config.put(SEGMENT_JITTER_MS_CONFIG, "1000") - config.put(SEGMENT_BYTES_CONFIG, "67108864") + config.put(INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") validator.validate(new ConfigResource(TOPIC, "foo"), config, emptyMap()) } @@ -81,7 +81,7 @@ class ControllerConfigurationValidatorTest { def testInvalidTopicConfig(): Unit = { val config = new util.TreeMap[String, String]() config.put(SEGMENT_JITTER_MS_CONFIG, "1000") - config.put(SEGMENT_BYTES_CONFIG, "67108864") + config.put(INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") config.put("foobar", "abc") assertEquals("Unknown topic config name: foobar", assertThrows(classOf[InvalidConfigurationException], () => validator.validate( diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 421def63c2ea6..fcc04dd8deb34 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -100,19 +100,19 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val tp = new TopicPartition("test", 0) val oldSegmentSize = 1000 val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) + logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) createTopic(tp.topic, 1, 1, logProps) TestUtils.retry(10000) { val logOpt = this.brokers.head.logManager.getLog(tp) assertTrue(logOpt.isDefined) - assertEquals(oldSegmentSize, logOpt.get.config.segmentSize) + assertEquals(oldSegmentSize, logOpt.get.config.segmentSize()) } val newSegmentSize = 2000 val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(TopicConfig.SEGMENT_BYTES_CONFIG, newSegmentSize.toString), + val op = new AlterConfigOp(new ConfigEntry(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), OpType.SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get } finally { @@ -120,7 +120,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } val log = brokers.head.logManager.getLog(tp).get TestUtils.retry(10000) { - assertEquals(newSegmentSize, log.config.segmentSize) + assertEquals(newSegmentSize, log.config.segmentSize()) } (1 to 50).foreach(i => TestUtils.produceMessage(brokers, tp.topic, i.toString)) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 1f6c3755131cc..85ad67aaa8ed4 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1086,7 +1086,7 @@ class KafkaConfigTest { assertDynamic(kafkaConfigProp, "5", () => config.lz4CompressionLevel) case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertDynamic(kafkaConfigProp, "5", () => config.zstdCompressionLevel) - case TopicConfig.SEGMENT_BYTES_CONFIG => + case TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 1048576, () => config.logSegmentBytes) case TopicConfig.SEGMENT_MS_CONFIG => assertDynamic(kafkaConfigProp, 10001L, () => config.logRollTimeMillis) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 6b117a0ec45a2..d6ca10973d6f1 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -418,7 +418,7 @@ public void testGroupMetadataTopicConfigs() { Properties expectedProperties = new Properties(); expectedProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); expectedProperties.put(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.PRODUCER.name); - expectedProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000"); + expectedProperties.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000"); assertEquals(expectedProperties, service.groupMetadataTopicConfigs()); } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index 1b5bf4fb29d24..f9c76d257c0e6 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -126,7 +126,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import static org.apache.kafka.common.config.TopicConfig.SEGMENT_BYTES_CONFIG; +import static org.apache.kafka.common.config.TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG; import static org.apache.kafka.common.metadata.MetadataRecordType.CLEAR_ELR_RECORD; import static org.apache.kafka.common.protocol.Errors.ELECTION_NOT_NEEDED; import static org.apache.kafka.common.protocol.Errors.ELIGIBLE_LEADERS_NOT_AVAILABLE; @@ -905,7 +905,7 @@ public void testCreateTopicsWithPolicy() { null, Map.of()), new CreateTopicPolicy.RequestMetadata("baz", null, null, Map.of(0, List.of(2, 1, 0)), - Map.of(SEGMENT_BYTES_CONFIG, "12300000")), + Map.of(INTERNAL_SEGMENT_BYTES_CONFIG, "12300000")), new CreateTopicPolicy.RequestMetadata("quux", null, null, Map.of(0, List.of(2, 1, 0)), Map.of()))); ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder(). @@ -916,7 +916,7 @@ public void testCreateTopicsWithPolicy() { ctx.createTestTopic("foo", 2, (short) 2, NONE.code()); ctx.createTestTopic("bar", 3, (short) 3, POLICY_VIOLATION.code()); ctx.createTestTopic("baz", new int[][] {new int[] {2, 1, 0}}, - Map.of(SEGMENT_BYTES_CONFIG, "12300000"), NONE.code()); + Map.of(INTERNAL_SEGMENT_BYTES_CONFIG, "12300000"), NONE.code()); ctx.createTestTopic("quux", new int[][] {new int[] {1, 2, 0}}, POLICY_VIOLATION.code()); } diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java index 399643e32a9ad..178a8176f36db 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java @@ -1892,7 +1892,7 @@ public void testShareStateTopicConfigs() { List propNames = List.of( TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.COMPRESSION_TYPE_CONFIG, - TopicConfig.SEGMENT_BYTES_CONFIG, + TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG ); Properties actual = service.shareGroupStateTopicConfigs(); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 1287c1a071fd0..dfacc91948614 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.LegacyRecord; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ConfigUtils; @@ -188,7 +189,8 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) - .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC); + .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) + .defineInternal(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { @@ -296,6 +298,7 @@ public Optional serverConfigName(String configName) { private final RemoteLogConfig remoteLogConfig; private final int maxMessageSize; private final Map props; + private final Integer internalSegmentSize; public LogConfig(Map props) { this(props, Set.of()); @@ -308,6 +311,7 @@ public LogConfig(Map props, Set overriddenConfigs) { this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); + this.internalSegmentSize = getInt(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG); this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); @@ -390,7 +394,7 @@ public long maxSegmentMs() { public int initFileSize() { if (preallocate) - return segmentSize; + return segmentSize(); else return 0; } @@ -456,6 +460,11 @@ public static Map configKeys() { return Collections.unmodifiableMap(CONFIG.configKeys()); } + public int segmentSize() { + if (internalSegmentSize != null) return internalSegmentSize; + return segmentSize; + } + /** * Check that property names are valid */ @@ -630,7 +639,7 @@ public static void validate(Map existingConfigs, @Override public String toString() { return "LogConfig{" + - "segmentSize=" + segmentSize + + "segmentSize=" + segmentSize() + ", segmentMs=" + segmentMs + ", segmentJitterMs=" + segmentJitterMs + ", maxIndexSize=" + maxIndexSize + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index dcef6929d1902..ff689898ecd62 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1177,9 +1177,9 @@ private LogAppendInfo append(MemoryRecords records, }); // check messages size does not exceed config.segmentSize - if (validRecords.sizeInBytes() > config().segmentSize) { + if (validRecords.sizeInBytes() > config().segmentSize()) { throw new RecordBatchTooLargeException("Message batch size is " + validRecords.sizeInBytes() + " bytes in append " + - "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize + "."); + "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize() + "."); } // maybe roll the log if this segment is full @@ -2034,12 +2034,12 @@ private LogSegment maybeRoll(int messagesSize, LogAppendInfo appendInfo) throws long maxTimestampInMessages = appendInfo.maxTimestamp(); long maxOffsetInMessages = appendInfo.lastOffset(); - if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize, appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { + if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize(), appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { logger.debug("Rolling new log segment (log_size = {}/{}}, " + "offset_index_size = {}/{}, " + "time_index_size = {}/{}, " + "inactive_time_ms = {}/{}).", - segment.size(), config().segmentSize, + segment.size(), config().segmentSize(), segment.offsetIndex().entries(), segment.offsetIndex().maxEntries(), segment.timeIndex().entries(), segment.timeIndex().maxEntries(), segment.timeWaitedForRoll(now, maxTimestampInMessages), config().segmentMs - segment.rollJitterMs()); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index 7dfba8f7a5954..baa450fa011b0 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -198,7 +198,7 @@ public void testUpdateConfig() { assertEquals(oldConfig, log.config()); Properties props = new Properties(); - props.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldConfig.segmentSize + 1); + props.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); LogConfig newConfig = new LogConfig(props); log.updateConfig(newConfig); assertEquals(newConfig, log.config()); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index 6023706847f15..4c6f42544c104 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -114,7 +114,7 @@ public final boolean conditionMet() { .get(); return config.get(TopicConfig.CLEANUP_POLICY_CONFIG).value().equals(TopicConfig.CLEANUP_POLICY_DELETE) && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(PURGE_INTERVAL_MS.toString()) - && config.get(TopicConfig.SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); + && config.get(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); } catch (final Exception e) { return false; } @@ -171,7 +171,7 @@ public void setup() { streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); - streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); + streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size final StreamsBuilder builder = new StreamsBuilder(); diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index a7f657ec54b70..e19da26f67a01 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -248,7 +248,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { props.put(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, 99_999L); props.put(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, 7L); props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "dummy:host"); - props.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), 100); + props.put(StreamsConfig.topicPrefix(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG), 100); final StreamsConfig streamsConfig = new StreamsConfig(props); final Map returnedProps = streamsConfig.getMainConsumerConfigs(groupId, clientId, threadIdx); @@ -263,7 +263,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { ); assertEquals(7L, returnedProps.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG)); assertEquals("dummy:host", returnedProps.get(StreamsConfig.APPLICATION_SERVER_CONFIG)); - assertEquals(100, returnedProps.get(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG))); + assertEquals(100, returnedProps.get(StreamsConfig.topicPrefix(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG))); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java index 14470db2efa8d..646e65aec50c0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java @@ -923,7 +923,7 @@ public void shouldCreateTopicWhenTopicLeaderNotAvailableAndThenTopicNotFound() { new NewTopic(topic1, Optional.of(1), Optional.of((short) 1)) .configs(mkMap(mkEntry(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE), mkEntry(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "CreateTime"), - mkEntry(TopicConfig.SEGMENT_BYTES_CONFIG, "52428800"), + mkEntry(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "52428800"), mkEntry(TopicConfig.RETENTION_MS_CONFIG, "-1")))))) .thenAnswer(answer -> new MockCreateTopicsResult(Collections.singletonMap(topic1, topicCreationFuture))); diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 5bb23cabdd98a..b85f25f341b9e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -97,7 +97,7 @@ private void setUpRemoteLogTopics() { Map rlsConfigs = new HashMap<>(); rlsConfigs.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"); rlsConfigs.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "1"); - rlsConfigs.put(TopicConfig.SEGMENT_BYTES_CONFIG, "100"); + rlsConfigs.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "100"); setupTopics(this::getRemoteLogStorageEnabledTopicName, rlsConfigs); sendProducerRecords(this::getRemoteLogStorageEnabledTopicName); } From 8c39276ff74485cd431a34fb8b0424466ee211ee Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 10 Apr 2025 23:30:59 +0800 Subject: [PATCH 13/74] fix some test --- .../kafka/coordinator/group/GroupCoordinatorServiceTest.java | 2 +- .../java/org/apache/kafka/storage/internals/log/LogConfig.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index d6ca10973d6f1..6b117a0ec45a2 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -418,7 +418,7 @@ public void testGroupMetadataTopicConfigs() { Properties expectedProperties = new Properties(); expectedProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); expectedProperties.put(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.PRODUCER.name); - expectedProperties.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000"); + expectedProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000"); assertEquals(expectedProperties, service.groupMetadataTopicConfigs()); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index dfacc91948614..a60a79e9c59de 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -257,7 +257,8 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) - .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC); + .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) + .defineInternal(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); } public final Set overriddenConfigs; From eefa646a522946500692b047bfb4a4ca3916c4fe Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 11 Apr 2025 00:13:59 +0800 Subject: [PATCH 14/74] fix MetadataLog Test --- .../scala/kafka/raft/KafkaMetadataLog.scala | 117 +++++++++++++----- .../kafka/raft/KafkaMetadataLogTest.scala | 2 +- .../kafka/tools/DumpLogSegmentsTest.scala | 2 +- 3 files changed, 88 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index ebc2f3c81cf5d..2177ec5e16ef5 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -575,7 +575,38 @@ final class KafkaMetadataLog private ( } object KafkaMetadataLog extends Logging { + def apply( + topicPartition: TopicPartition, + topicId: Uuid, + dataDir: File, + time: Time, + scheduler: Scheduler, + config: MetadataLogConfig + ): KafkaMetadataLog = { + val props: Properties = settingLogProperties(config) + 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 + 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.") + } + + // 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 + } + + // visible for testing + def internalApply( topicPartition: TopicPartition, topicId: Uuid, dataDir: File, @@ -583,35 +614,41 @@ 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) + props.remove(TopicConfig.SEGMENT_BYTES_CONFIG); + props.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) 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) + + // 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.") } + // 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, @@ -631,20 +668,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( diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 466fb954d133f..2b7d5c42a10c7 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -1127,7 +1127,7 @@ object KafkaMetadataLogTest { UnifiedLog.logDirName(KafkaRaftServer.MetadataPartition) ) - val metadataLog = KafkaMetadataLog( + val metadataLog = KafkaMetadataLog.internalApply( KafkaRaftServer.MetadataPartition, KafkaRaftServer.MetadataTopicId, logDir, diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 11d33787dd6e4..ad371353d08d8 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -538,7 +538,7 @@ class DumpLogSegmentsTest { setPartitionId(0).setIsr(util.Arrays.asList(0, 1, 2)), 0.toShort) ) - val metadataLog = KafkaMetadataLog( + val metadataLog = KafkaMetadataLog.internalApply( KafkaRaftServer.MetadataPartition, KafkaRaftServer.MetadataTopicId, logDir, From e2adfbe618a5b585fb7f6e1864179e975082f23b Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 11 Apr 2025 00:14:59 +0800 Subject: [PATCH 15/74] fix MetadataLog Test --- .../scala/kafka/raft/KafkaMetadataLog.scala | 35 ++++++++++--------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 2177ec5e16ef5..c0762e3049755 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -577,13 +577,13 @@ final class KafkaMetadataLog private ( object KafkaMetadataLog extends Logging { def apply( - topicPartition: TopicPartition, - topicId: Uuid, - dataDir: File, - time: Time, - scheduler: Scheduler, - config: MetadataLogConfig - ): KafkaMetadataLog = { + topicPartition: TopicPartition, + topicId: Uuid, + dataDir: File, + time: Time, + scheduler: Scheduler, + config: MetadataLogConfig + ): KafkaMetadataLog = { val props: Properties = settingLogProperties(config) LogConfig.validate(props) val defaultLogConfig = new LogConfig(props) @@ -592,11 +592,7 @@ object KafkaMetadataLog extends Logging { val metadataLog: KafkaMetadataLog = createKafkaMetadataLog(topicPartition, topicId, dataDir, time, scheduler, config, defaultLogConfig) - // 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.") - } + 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. @@ -604,7 +600,15 @@ object KafkaMetadataLog extends Logging { 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( topicPartition: TopicPartition, @@ -625,10 +629,7 @@ object KafkaMetadataLog extends Logging { val metadataLog: KafkaMetadataLog = createKafkaMetadataLog(topicPartition, topicId, dataDir, time, scheduler, config, defaultLogConfig) // 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.") - } + 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. From 3d8e12202225d5cd1e75452932c7c78a166ba574 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 11 Apr 2025 00:49:39 +0800 Subject: [PATCH 16/74] fix some test --- .../streams/processor/internals/InternalTopicManagerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java index 646e65aec50c0..14470db2efa8d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java @@ -923,7 +923,7 @@ public void shouldCreateTopicWhenTopicLeaderNotAvailableAndThenTopicNotFound() { new NewTopic(topic1, Optional.of(1), Optional.of((short) 1)) .configs(mkMap(mkEntry(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE), mkEntry(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, "CreateTime"), - mkEntry(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "52428800"), + mkEntry(TopicConfig.SEGMENT_BYTES_CONFIG, "52428800"), mkEntry(TopicConfig.RETENTION_MS_CONFIG, "-1")))))) .thenAnswer(answer -> new MockCreateTopicsResult(Collections.singletonMap(topic1, topicCreationFuture))); From 303ef391e48c83f385db33d951672bee6eea3784 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 11 Apr 2025 09:05:01 +0800 Subject: [PATCH 17/74] fix fail tests --- .../apache/kafka/clients/admin/StaticBrokerConfigTest.java | 2 +- .../kafka/server/DynamicBrokerReconfigurationTest.scala | 2 +- .../kafka/server/ControllerConfigurationValidatorTest.scala | 4 ++-- core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala | 2 +- .../org/apache/kafka/storage/internals/log/LogConfig.java | 1 + 5 files changed, 6 insertions(+), 5 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java index 8d981d1fb7a01..1b8ea247a342b 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java @@ -86,7 +86,7 @@ public void testTopicConfigsGetImpactedIfStaticConfigsAddToBroker(ClusterInstanc Admin adminUsingBootstrapController = cluster.admin(Map.of(), true) ) { ConfigEntry configEntry = admin.createTopics(List.of(new NewTopic(TOPIC, 1, (short) 1))) - .config(TOPIC).get().get(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG); + .config(TOPIC).get().get(TopicConfig.SEGMENT_BYTES_CONFIG); assertEquals(ConfigEntry.ConfigSource.DEFAULT_CONFIG, configEntry.source()); assertNotEquals(CUSTOM_VALUE, configEntry.value(), diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 8708c5bcce636..927652eb98431 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -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) => diff --git a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala index 671f8facd6ef1..35c4c05d994ad 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala @@ -20,7 +20,7 @@ package kafka.server import kafka.utils.TestUtils import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, BROKER_LOGGER, CLIENT_METRICS, GROUP, TOPIC} -import org.apache.kafka.common.config.TopicConfig.{REMOTE_LOG_STORAGE_ENABLE_CONFIG, INTERNAL_SEGMENT_BYTES_CONFIG, SEGMENT_JITTER_MS_CONFIG, SEGMENT_MS_CONFIG} +import org.apache.kafka.common.config.TopicConfig.{INTERNAL_SEGMENT_BYTES_CONFIG, REMOTE_LOG_STORAGE_ENABLE_CONFIG, SEGMENT_BYTES_CONFIG, SEGMENT_JITTER_MS_CONFIG, SEGMENT_MS_CONFIG} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, InvalidTopicException} import org.apache.kafka.coordinator.group.GroupConfig import org.apache.kafka.server.metrics.ClientMetricsConfigs @@ -62,7 +62,7 @@ class ControllerConfigurationValidatorTest { def testNullTopicConfigValue(): Unit = { val config = new util.TreeMap[String, String]() config.put(SEGMENT_JITTER_MS_CONFIG, "10") - config.put(INTERNAL_SEGMENT_BYTES_CONFIG, null) + config.put(SEGMENT_BYTES_CONFIG, null) config.put(SEGMENT_MS_CONFIG, null) assertEquals("Null value not supported for topic configs: segment.bytes,segment.ms", assertThrows(classOf[InvalidConfigurationException], () => validator.validate( diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 85ad67aaa8ed4..1f6c3755131cc 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1086,7 +1086,7 @@ class KafkaConfigTest { assertDynamic(kafkaConfigProp, "5", () => config.lz4CompressionLevel) case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertDynamic(kafkaConfigProp, "5", () => config.zstdCompressionLevel) - case TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG => + case TopicConfig.SEGMENT_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 1048576, () => config.logSegmentBytes) case TopicConfig.SEGMENT_MS_CONFIG => assertDynamic(kafkaConfigProp, 10001L, () => config.logRollTimeMillis) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index a60a79e9c59de..23a99ca68f031 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -148,6 +148,7 @@ public Optional serverConfigName(String configName) { TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, + TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG ); From 71dbb897088cab8264a3c6a03802fad485f634c5 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 11 Apr 2025 10:11:20 +0800 Subject: [PATCH 18/74] fix config def error --- .../kafka/server/DynamicBrokerReconfigurationTest.scala | 2 +- .../org/apache/kafka/storage/internals/log/LogConfig.java | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 927652eb98431..26fbce1ffa68b 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -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) => diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 23a99ca68f031..164bd40b8c292 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.record.LegacyRecord; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ConfigUtils; @@ -191,7 +190,7 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) - .defineInternal(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); + .defineInternal(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { @@ -259,7 +258,7 @@ public Optional serverConfigName(String configName) { TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) - .defineInternal(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); + .defineInternal(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); } public final Set overriddenConfigs; From 4c62f11c50f9ef37cf095e41a01716cf4e64b424 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 11 Apr 2025 14:39:00 +0800 Subject: [PATCH 19/74] fix fail test --- .../org/apache/kafka/clients/admin/StaticBrokerConfigTest.java | 2 +- core/src/test/scala/unit/kafka/log/LogConfigTest.scala | 1 + core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java index 1b8ea247a342b..58240c0711e62 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java @@ -53,7 +53,7 @@ public void testTopicConfigsGetImpactedIfStaticConfigsAddToController(ClusterIns Admin adminUsingBootstrapController = cluster.admin(Map.of(), true) ) { ConfigEntry configEntry = admin.createTopics(List.of(new NewTopic(TOPIC, 1, (short) 1))) - .config(TOPIC).get().get(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG); + .config(TOPIC).get().get(TopicConfig.SEGMENT_BYTES_CONFIG); assertEquals(ConfigEntry.ConfigSource.STATIC_BROKER_CONFIG, configEntry.source()); assertEquals(CUSTOM_VALUE, configEntry.value(), "Config value should be custom value since controller has related static config"); diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 1e26d653bbccf..cd8ef53eb8007 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -94,6 +94,7 @@ class LogConfigTest { case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1") case TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") + case TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // no op case _ => assertPropertyInvalid(name, "not_a_number", "-1") }) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 1f6c3755131cc..ae411a894c15d 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -897,6 +897,8 @@ class KafkaConfigTest { case ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", CompressionType.ZSTD.maxLevel() + 1) + case TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // ignore int + //SSL Configs case BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => case BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG => From 5e4fe4617c70b24e5a1f76c4450d56f26181446a Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 12 Apr 2025 08:51:49 +0800 Subject: [PATCH 20/74] Update LogCleaner file --- .../java/org/apache/kafka/storage/internals/log/Cleaner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java index b7d4ccedb2ec1..54ad6a4e79a97 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java @@ -171,7 +171,7 @@ public Map.Entry doClean(LogToClean cleanable, long currentT List> groupedSegments = groupSegmentsBySize( log.logSegments(0, endOffset), - log.config().segmentSize, + log.config().segmentSize(), log.config().maxIndexSize, cleanable.firstUncleanableOffset() ); From 9b9f4697d0234976b53b9af0b378b24ebe78d530 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 12 Apr 2025 08:58:29 +0800 Subject: [PATCH 21/74] change segmentSize modifier --- .../java/org/apache/kafka/storage/internals/log/LogConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 8f90adf01ce74..2eb1f419ddd2c 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -264,7 +264,7 @@ public Optional serverConfigName(String configName) { * Important note: Any configuration parameter that is passed along from KafkaConfig to LogConfig * should also be in `KafkaConfig#extractLogConfigMap`. */ - public final int segmentSize; + private final int segmentSize; public final long segmentMs; public final long segmentJitterMs; public final int maxIndexSize; From 8b7d366405e478a76c00b49259d08b5a03d5c61c Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 14 Apr 2025 21:26:48 +0800 Subject: [PATCH 22/74] addressed by comments --- .../kafka/common/config/TopicConfig.java | 3 - .../scala/kafka/raft/KafkaMetadataLog.scala | 2 +- .../api/PlaintextAdminIntegrationTest.scala | 2 +- .../api/SaslSslAdminIntegrationTest.scala | 5 +- .../kafka/cluster/AbstractPartitionTest.scala | 2 +- .../kafka/cluster/PartitionLockTest.scala | 2 +- .../AbstractLogCleanerIntegrationTest.scala | 2 +- .../kafka/log/LogCleanerManagerTest.scala | 18 ++--- .../scala/unit/kafka/log/LogCleanerTest.scala | 72 +++++++++---------- .../unit/kafka/log/LogConcurrencyTest.scala | 2 +- .../scala/unit/kafka/log/LogConfigTest.scala | 2 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 2 +- .../scala/unit/kafka/log/LogManagerTest.scala | 4 +- .../scala/unit/kafka/log/LogTestUtils.scala | 2 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 2 +- .../AlterReplicaLogDirsRequestTest.scala | 4 +- ...ControllerConfigurationValidatorTest.scala | 7 +- .../server/DynamicConfigChangeTest.scala | 4 +- .../unit/kafka/server/KafkaConfigTest.scala | 4 +- .../ReplicationControlManagerTest.java | 5 +- .../share/ShareCoordinatorServiceTest.java | 2 +- .../storage/internals/log/LogConfig.java | 10 +-- .../storage/internals/log/LocalLogTest.java | 3 +- .../PurgeRepartitionTopicIntegrationTest.java | 4 +- .../kafka/streams/StreamsConfigTest.java | 5 +- .../kafka/tools/GetOffsetShellTest.java | 3 +- 26 files changed, 85 insertions(+), 88 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 97e7506fb53b2..614689b17cc5c 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -215,9 +215,6 @@ public class TopicConfig { "configuration. If message.timestamp.type=CreateTime, the message will be rejected if the difference in " + "timestamps exceeds this specified threshold. This configuration is ignored if message.timestamp.type=LogAppendTime."; - // visible for testing - public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; - /** * @deprecated down-conversion is not possible in Apache Kafka 4.0 and newer, hence this configuration is a no-op, * and it is deprecated for removal in Apache Kafka 5.0. diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index c0762e3049755..4fe2cd073e03f 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -620,7 +620,7 @@ object KafkaMetadataLog extends Logging { ): KafkaMetadataLog = { val props: Properties = settingLogProperties(config) props.remove(TopicConfig.SEGMENT_BYTES_CONFIG); - props.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) + props.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) LogConfig.validate(props) val defaultLogConfig = new LogConfig(props) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index fe242203bfe63..f0353009b6d1b 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -1568,7 +1568,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @MethodSource(Array("getTestGroupProtocolParametersAll")) def testDeleteRecordsAfterCorruptRecords(groupProtocol: String): Unit = { val config = new Properties() - config.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "200") + config.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "200") createTopic(topic, numPartitions = 1, replicationFactor = 1, config) client = createAdminClient diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index ed8493da3e6c6..1766decb24d5c 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -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} @@ -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.INTERNAL_SEGMENT_BYTES_CONFIG -> "100000").asJava + val configsOverride = Map(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG -> "100000").asJava 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)) @@ -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.INTERNAL_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 diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index da4d042874920..fff1930a71836 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -100,7 +100,7 @@ class AbstractPartitionTest { def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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) } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index da899a36809c3..77b098cf68298 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -362,7 +362,7 @@ class PartitionLockTest extends Logging { private def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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) } diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index 950f8a1f00e39..b1e161b975340 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -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.INTERNAL_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) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index d47382950a258..8445baa7719fb 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -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.INTERNAL_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) @@ -370,7 +370,7 @@ class LogCleanerManagerTest extends Logging { // change cleanup policy from delete to compact val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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) @@ -548,7 +548,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsForNone(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -570,7 +570,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsActiveSegment(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -592,7 +592,7 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -625,7 +625,7 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForShortTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -667,7 +667,7 @@ class LogCleanerManagerTest extends Logging { def testUndecidedTransactionalDataNotCleanable(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -711,7 +711,7 @@ class LogCleanerManagerTest extends Logging { @Test def testDoneCleaning(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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) @@ -830,7 +830,7 @@ class LogCleanerManagerTest extends Logging { private def createLowRetentionLogConfig(segmentSize: Int, cleanupPolicy: String): LogConfig = { val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index df7aabf23ee8b..1bebfaa49e173 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -56,7 +56,7 @@ class LogCleanerTest extends Logging { val tmpdir = TestUtils.tempDir() val dir = TestUtils.randomPartitionLogDir(tmpdir) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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 = new LogConfig(logProps) @@ -148,7 +148,7 @@ class LogCleanerTest extends Logging { def testCleanSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -181,7 +181,7 @@ class LogCleanerTest extends Logging { // Construct a log instance. The replaceSegments() method of the log instance is overridden so that // it waits for another thread to execute deleteOldSegments() val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) @@ -271,7 +271,7 @@ class LogCleanerTest extends Logging { val originalMaxFileSize = 1024 val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact": java.lang.String) logProps.put(TopicConfig.PREALLOCATE_CONFIG, "true": java.lang.String) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -295,7 +295,7 @@ class LogCleanerTest extends Logging { def testDuplicateCheckAfterCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) var log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -464,7 +464,7 @@ class LogCleanerTest extends Logging { def testBasicTransactionAwareCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -497,7 +497,7 @@ class LogCleanerTest extends Logging { def testCleanWithTransactionsSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) val producerEpoch = 0.toShort @@ -543,7 +543,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -591,7 +591,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(capacity = Int.MaxValue, maxMessageSize = 100) val logProps = new Properties() logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, 100: java.lang.Integer) - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -619,7 +619,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -682,7 +682,7 @@ class LogCleanerTest extends Logging { def testCleanEmptyControlBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -714,7 +714,7 @@ class LogCleanerTest extends Logging { def testCommittedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -736,7 +736,7 @@ class LogCleanerTest extends Logging { def testAbortedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -765,7 +765,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -799,7 +799,7 @@ class LogCleanerTest extends Logging { val producerId = 1L val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, 0, AppendOrigin.REPLICATION) @@ -832,7 +832,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -896,7 +896,7 @@ class LogCleanerTest extends Logging { // Create cleaner with very small default max message size val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -968,7 +968,7 @@ class LogCleanerTest extends Logging { def createLogWithMessagesLargerThanMaxSize(largeMessageSize: Int): (UnifiedLog, FakeOffsetMap) = { val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -994,7 +994,7 @@ class LogCleanerTest extends Logging { def testCleaningWithDeletes(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -1021,7 +1021,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 3 messages in the map val cleaner = makeCleaner(4) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -1050,7 +1050,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsProducerLastSequence(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) log.appendAsLeader(record(0, 0), 0) // offset 0 @@ -1073,7 +1073,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsLastSequenceEvenIfTransactionAborted(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) val producerEpoch = 0.toShort @@ -1107,7 +1107,7 @@ class LogCleanerTest extends Logging { def testCleaningWithKeysConflictingWithTxnMarkerKeys(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) val leaderEpoch = 5 val producerEpoch = 0.toShort @@ -1151,7 +1151,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 1 message in the map val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -1182,7 +1182,7 @@ class LogCleanerTest extends Logging { def testCleaningWithUncleanableSection(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -1225,7 +1225,7 @@ class LogCleanerTest extends Logging { def testLogToClean(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1243,7 +1243,7 @@ class LogCleanerTest extends Logging { def testLogToCleanWithUncleanableSection(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1276,7 +1276,7 @@ class LogCleanerTest extends Logging { // create a log with compaction turned off so we can append unkeyed messages val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1334,7 +1334,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsWithAbort(): Unit = { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -1356,7 +1356,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsRetainingLastEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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)) @@ -1388,7 +1388,7 @@ class LogCleanerTest extends Logging { def testSegmentGrouping(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1489,7 +1489,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1541,7 +1541,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) //mimic the effect of loading an empty index file logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 400: java.lang.Integer) @@ -1666,7 +1666,7 @@ class LogCleanerTest extends Logging { def testRecoveryAfterCrash(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 10: java.lang.Integer) @@ -1797,7 +1797,7 @@ class LogCleanerTest extends Logging { def testBuildOffsetMapFakeLarge(): Unit = { val map = new FakeOffsetMap(1000) val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) @@ -1945,7 +1945,7 @@ class LogCleanerTest extends Logging { @Test def testCleaningBeyondMissingOffsets(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) val cleaner = makeCleaner(Int.MaxValue) diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 0661070f819c7..8c49978e6bd9f 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -60,7 +60,7 @@ class LogConcurrencyTest { @Test def testUncommittedDataNotConsumedFrequentSegmentRolls(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 237: Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 237: Integer) val logConfig = new LogConfig(logProps) testUncommittedDataNotConsumed(createLog(logConfig)) } diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index cd8ef53eb8007..752d41e54a9c1 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -94,7 +94,7 @@ class LogConfigTest { case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1") case TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") - case TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // no op + case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // no op case _ => assertPropertyInvalid(name, "not_a_number", "-1") }) diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index d942cc47ff945..1ef21ad81f407 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -245,7 +245,7 @@ class LogLoaderTest { @Test def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "640") + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "640") val logConfig = new LogConfig(logProps) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 7ff8fe6cf71fb..a4c68d170d48a 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -62,7 +62,7 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 60 * 1000 val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_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, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) val logConfig = new LogConfig(logProps) @@ -393,7 +393,7 @@ class LogManagerTest { logManager.shutdown() val segmentBytes = 10 * setSize val properties = new Properties() - properties.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes.toString) + properties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes.toString) properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString) val configRepository = MockConfigRepository.forTopic(name, properties) diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index fb809d235df0a..661bbf4006cb6 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -76,7 +76,7 @@ object LogTestUtils { remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs: java.lang.Long) logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 28161be15b4d8..17f8691154cc3 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -2733,7 +2733,7 @@ class UnifiedLogTest { @Test def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000") + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000") logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") val logConfig = new LogConfig(logProps) diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index 81887d492614d..78f5335caf42d 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AlterReplicaLogDirsRequest, AlterReplicaLogDirsResponse} import org.apache.kafka.server.config.ServerLogConfigs -import org.apache.kafka.storage.internals.log.LogFileUtils +import org.apache.kafka.storage.internals.log.{LogConfig, LogFileUtils} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -148,7 +148,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { // We don't want files with `.deleted` suffix are removed too fast, // so we can validate there will be orphan files and orphan files will be removed eventually. topicProperties.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, "10000") - topicProperties.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1024") + topicProperties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1024") createTopic(topic, partitionNum, 1, topicProperties) assertEquals(logDir1, brokers.head.logManager.getLog(tp).get.dir.getParent) diff --git a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala index 35c4c05d994ad..180d1bda70828 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala @@ -20,10 +20,11 @@ package kafka.server import kafka.utils.TestUtils import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, BROKER_LOGGER, CLIENT_METRICS, GROUP, TOPIC} -import org.apache.kafka.common.config.TopicConfig.{INTERNAL_SEGMENT_BYTES_CONFIG, REMOTE_LOG_STORAGE_ENABLE_CONFIG, SEGMENT_BYTES_CONFIG, SEGMENT_JITTER_MS_CONFIG, SEGMENT_MS_CONFIG} +import org.apache.kafka.common.config.TopicConfig.{REMOTE_LOG_STORAGE_ENABLE_CONFIG, SEGMENT_BYTES_CONFIG, SEGMENT_JITTER_MS_CONFIG, SEGMENT_MS_CONFIG} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, InvalidTopicException} import org.apache.kafka.coordinator.group.GroupConfig import org.apache.kafka.server.metrics.ClientMetricsConfigs +import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest @@ -73,7 +74,7 @@ class ControllerConfigurationValidatorTest { def testValidTopicConfig(): Unit = { val config = new util.TreeMap[String, String]() config.put(SEGMENT_JITTER_MS_CONFIG, "1000") - config.put(INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") + config.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") validator.validate(new ConfigResource(TOPIC, "foo"), config, emptyMap()) } @@ -81,7 +82,7 @@ class ControllerConfigurationValidatorTest { def testInvalidTopicConfig(): Unit = { val config = new util.TreeMap[String, String]() config.put(SEGMENT_JITTER_MS_CONFIG, "1000") - config.put(INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") + config.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") config.put("foobar", "abc") assertEquals("Unknown topic config name: foobar", assertThrows(classOf[InvalidConfigurationException], () => validator.validate( diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index fcc04dd8deb34..f72915a55c093 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -100,7 +100,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val tp = new TopicPartition("test", 0) val oldSegmentSize = 1000 val logProps = new Properties() - logProps.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) createTopic(tp.topic, 1, 1, logProps) TestUtils.retry(10000) { val logOpt = this.brokers.head.logManager.getLog(tp) @@ -112,7 +112,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), + val op = new AlterConfigOp(new ConfigEntry(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), OpType.SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get } finally { diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index ae411a894c15d..9c05c0f349945 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -896,9 +896,7 @@ class KafkaConfigTest { case ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", CompressionType.ZSTD.maxLevel() + 1) - - case TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // ignore int - + //SSL Configs case BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => case BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG => diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index f9c76d257c0e6..3ffb42b788bdf 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -126,7 +126,6 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import static org.apache.kafka.common.config.TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG; import static org.apache.kafka.common.metadata.MetadataRecordType.CLEAR_ELR_RECORD; import static org.apache.kafka.common.protocol.Errors.ELECTION_NOT_NEEDED; import static org.apache.kafka.common.protocol.Errors.ELIGIBLE_LEADERS_NOT_AVAILABLE; @@ -905,7 +904,7 @@ public void testCreateTopicsWithPolicy() { null, Map.of()), new CreateTopicPolicy.RequestMetadata("baz", null, null, Map.of(0, List.of(2, 1, 0)), - Map.of(INTERNAL_SEGMENT_BYTES_CONFIG, "12300000")), + Map.of("internal.segment.bytes", "12300000")), new CreateTopicPolicy.RequestMetadata("quux", null, null, Map.of(0, List.of(2, 1, 0)), Map.of()))); ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder(). @@ -916,7 +915,7 @@ public void testCreateTopicsWithPolicy() { ctx.createTestTopic("foo", 2, (short) 2, NONE.code()); ctx.createTestTopic("bar", 3, (short) 3, POLICY_VIOLATION.code()); ctx.createTestTopic("baz", new int[][] {new int[] {2, 1, 0}}, - Map.of(INTERNAL_SEGMENT_BYTES_CONFIG, "12300000"), NONE.code()); + Map.of("internal.segment.bytes", "12300000"), NONE.code()); ctx.createTestTopic("quux", new int[][] {new int[] {1, 2, 0}}, POLICY_VIOLATION.code()); } diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java index 178a8176f36db..bbcea152ffa80 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java @@ -1892,7 +1892,7 @@ public void testShareStateTopicConfigs() { List propNames = List.of( TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.COMPRESSION_TYPE_CONFIG, - TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, + "internal.segment.bytes", TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG ); Properties actual = service.shareGroupStateTopicConfigs(); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 2eb1f419ddd2c..cb3739cd7d81c 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -121,6 +121,8 @@ public Optional serverConfigName(String configName) { // Visible for testing public static final String SERVER_DEFAULT_HEADER_NAME = "Server Default Property"; + // visible for testing + public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final int DEFAULT_SEGMENT_BYTES = 1024 * 1024 * 1024; public static final long DEFAULT_SEGMENT_MS = 24 * 7 * 60 * 60 * 1000L; @@ -144,7 +146,7 @@ public Optional serverConfigName(String configName) { TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, - TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, + LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG ); @@ -187,7 +189,7 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) - .defineInternal(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); + .defineInternal(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { @@ -255,7 +257,7 @@ public Optional serverConfigName(String configName) { TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) - .defineInternal(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); + .defineInternal(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); } public final Set overriddenConfigs; @@ -309,7 +311,7 @@ public LogConfig(Map props, Set overriddenConfigs) { this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); - this.internalSegmentSize = getInt(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG); + this.internalSegmentSize = getInt(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG); this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index baa450fa011b0..d58c520af0972 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; -import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -198,7 +197,7 @@ public void testUpdateConfig() { assertEquals(oldConfig, log.config()); Properties props = new Properties(); - props.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); + props.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); LogConfig newConfig = new LogConfig(props); log.updateConfig(newConfig); assertEquals(newConfig, log.config()); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index 4c6f42544c104..be85c9fd0af86 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -114,7 +114,7 @@ public final boolean conditionMet() { .get(); return config.get(TopicConfig.CLEANUP_POLICY_CONFIG).value().equals(TopicConfig.CLEANUP_POLICY_DELETE) && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(PURGE_INTERVAL_MS.toString()) - && config.get(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); + && config.get("internal.segment.bytes").value().equals(PURGE_SEGMENT_BYTES.toString()); } catch (final Exception e) { return false; } @@ -171,7 +171,7 @@ public void setup() { streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); - streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); + streamsConfiguration.put(StreamsConfig.topicPrefix("internal.segment.bytes"), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size final StreamsBuilder builder = new StreamsBuilder(); diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index e19da26f67a01..5aa556e5acd8c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; @@ -248,7 +247,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { props.put(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, 99_999L); props.put(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, 7L); props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "dummy:host"); - props.put(StreamsConfig.topicPrefix(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG), 100); + props.put(StreamsConfig.topicPrefix("internal.segment.bytes"), 100); final StreamsConfig streamsConfig = new StreamsConfig(props); final Map returnedProps = streamsConfig.getMainConsumerConfigs(groupId, clientId, threadIdx); @@ -263,7 +262,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { ); assertEquals(7L, returnedProps.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG)); assertEquals("dummy:host", returnedProps.get(StreamsConfig.APPLICATION_SERVER_CONFIG)); - assertEquals(100, returnedProps.get(StreamsConfig.topicPrefix(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG))); + assertEquals(100, returnedProps.get(StreamsConfig.topicPrefix("internal.segment.bytes"))); } @Test diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index b85f25f341b9e..5ee3b38564032 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -40,6 +40,7 @@ import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig; import org.apache.kafka.server.log.remote.storage.LocalTieredStorage; import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig; +import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.test.TestUtils; import java.time.Duration; @@ -97,7 +98,7 @@ private void setUpRemoteLogTopics() { Map rlsConfigs = new HashMap<>(); rlsConfigs.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"); rlsConfigs.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "1"); - rlsConfigs.put(TopicConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "100"); + rlsConfigs.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "100"); setupTopics(this::getRemoteLogStorageEnabledTopicName, rlsConfigs); sendProducerRecords(this::getRemoteLogStorageEnabledTopicName); } From 5bbdf1fd43c72fe67ab8db1c2f81fb651c179f33 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 14 Apr 2025 21:29:49 +0800 Subject: [PATCH 23/74] addressed by comments --- .../main/java/org/apache/kafka/common/config/TopicConfig.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 614689b17cc5c..f3157bb1b1a43 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -228,5 +228,4 @@ public class TopicConfig { @Deprecated public static final String MESSAGE_DOWNCONVERSION_ENABLE_DOC = "Down-conversion is not possible in Apache Kafka 4.0 and newer, " + "hence this configuration is no-op and it is deprecated for removal in Apache Kafka 5.0."; - } From 2a3db85f9a6afca153333e9a53968971f8073356 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 14 Apr 2025 21:37:25 +0800 Subject: [PATCH 24/74] update the test --- core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 9c05c0f349945..f6f476bcd0c33 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -40,11 +40,10 @@ import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfig, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.MetricConfigs -import org.apache.kafka.storage.internals.log.CleanerConfig +import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.junit.jupiter.api.function.Executable - import org.apache.kafka.common.test.{TestUtils => JTestUtils} import scala.jdk.CollectionConverters._ @@ -897,6 +896,7 @@ class KafkaConfigTest { case ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", CompressionType.ZSTD.maxLevel() + 1) + case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // ignore int //SSL Configs case BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => case BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG => From a3320721cd30bf81c10511089673968e167de3b0 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 16 Apr 2025 00:43:26 +0800 Subject: [PATCH 25/74] remove unused import --- core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala | 1 - core/src/test/scala/unit/kafka/log/LogLoaderTest.scala | 1 - 2 files changed, 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 8c49978e6bd9f..854be39808661 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -20,7 +20,6 @@ package kafka.log import java.util.{Optional, Properties} import java.util.concurrent.{Callable, Executors} import kafka.utils.TestUtils -import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 1ef21ad81f407..0c465cf2138df 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -21,7 +21,6 @@ import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.compress.Compression -import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, SimpleRecord, TimestampType} import org.apache.kafka.common.utils.{Time, Utils} From 6876739befc749f28820600b7008c76a1dc1e72e Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 20 Apr 2025 16:09:01 +0800 Subject: [PATCH 26/74] move other internal config --- .../scala/kafka/raft/KafkaMetadataLog.scala | 108 +++++++++++++----- .../kafka/raft/KafkaMetadataLogTest.scala | 4 +- .../server/DynamicBrokerConfigTest.scala | 6 +- .../apache/kafka/raft/MetadataLogConfig.java | 20 +--- .../storage/internals/log/LogConfig.java | 19 ++- 5 files changed, 105 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 01f69b374bc3d..057f372ff9e36 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -585,32 +585,60 @@ object KafkaMetadataLog extends Logging { config: MetadataLogConfig, nodeId: Int ): 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) + val props: Properties = settingLogProperties(config) + LogConfig.validate(props) + val defaultLogConfig = new LogConfig(props) - // Disable time and byte retention when deleting segments - props.setProperty(TopicConfig.RETENTION_MS_CONFIG, "-1") - props.setProperty(TopicConfig.RETENTION_BYTES_CONFIG, "-1") + validateConfig(defaultLogConfig) + + val metadataLog: KafkaMetadataLog = createMetadataLog(topicPartition, topicId, dataDir, time, scheduler, config, nodeId, defaultLogConfig) + + printInternalConfigWarningMessage(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 + } + + // only for testing + def internalApply( + topicPartition: TopicPartition, + topicId: Uuid, + dataDir: File, + time: Time, + scheduler: Scheduler, + config: MetadataLogConfig, + nodeId: Int + ): 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) - if (config.logSegmentBytes < config.logSegmentMinBytes) { - throw new InvalidConfigurationException( - s"Cannot set ${MetadataLogConfig.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(defaultLogConfig) + val metadataLog: KafkaMetadataLog = createMetadataLog(topicPartition, topicId, dataDir, time, scheduler, config, nodeId, defaultLogConfig) + + printInternalConfigWarningMessage(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 createMetadataLog( + topicPartition: TopicPartition, + topicId: Uuid, + dataDir: File, + time: Time, + scheduler: Scheduler, + config: MetadataLogConfig, + nodeId: Int, + defaultLogConfig: LogConfig + ) = { val log = UnifiedLog.create( dataDir, defaultLogConfig, @@ -636,18 +664,40 @@ object KafkaMetadataLog extends Logging { config, nodeId ) + metadataLog + } + 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) + + // 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 validateConfig(defaultLogConfig: LogConfig): Unit = { + 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}." + ) + } + } + + private def printInternalConfigWarningMessage(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 ${MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG} is only supported for testing. Setting " + + if (config.logSegmentBytes() != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) { + metadataLog.error(s"Overriding ${LogConfig.INTERNAL_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.") } - - // 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 recoverSnapshots( diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 812d88fc6491d..f3a2fe94b6d6c 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -84,7 +84,7 @@ final class KafkaMetadataLogTest { buildMetadataLog(tempDir, mockTime, metadataConfig) }) - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) + props.put(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) @@ -689,7 +689,7 @@ final class KafkaMetadataLogTest { val recordSize = 64 val config = new MetadataLogConfig( DefaultMetadataLogConfig.logSegmentBytes, - DefaultMetadataLogConfig.logSegmentMinBytes, + DefaultMetadataLogConfig.logSegmentBytes, DefaultMetadataLogConfig.logSegmentMillis, DefaultMetadataLogConfig.retentionMaxBytes, DefaultMetadataLogConfig.retentionMillis, diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 17ad2200dcc3c..89e070eb0021a 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -673,11 +673,11 @@ class DynamicBrokerConfigTest { @Test def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = { val props = TestUtils.createBrokerConfig(0, port = 8181) - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, "1024") + props.put(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, "1024") val config = new KafkaConfig(props) - assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) + assertFalse(config.nonInternalValues.containsKey(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) config.updateCurrentConfig(new KafkaConfig(props)) - assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) + assertFalse(config.nonInternalValues.containsKey(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) } @Test diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index b0a6f9f045ad6..46faa42985494 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -52,10 +52,6 @@ public class MetadataLogConfig { "configuration. The Kafka node will generate a snapshot when either the maximum time interval is reached or the " + "maximum bytes limit is reached."; - public static final String METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG = "metadata.log.segment.min.bytes"; - public static final String METADATA_LOG_SEGMENT_MIN_BYTES_DOC = "Override the minimum size for a single metadata log file. This should be used for testing only."; - public static final int METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT = 8 * 1024 * 1024; - public static final String METADATA_LOG_SEGMENT_BYTES_CONFIG = "metadata.log.segment.bytes"; public static final String METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file."; public static final int METADATA_LOG_SEGMENT_BYTES_DEFAULT = 1024 * 1024 * 1024; @@ -85,14 +81,13 @@ public class MetadataLogConfig { .define(METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC) .define(METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, METADATA_LOG_DIR_DOC) .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) - .defineInternal(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_MIN_BYTES_DOC) .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); private final int logSegmentBytes; - private final int logSegmentMinBytes; + private final Integer internalLogSegmentMinBytes; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -103,7 +98,7 @@ public class MetadataLogConfig { /** * Configuration for the metadata log * @param logSegmentBytes The maximum size of a single metadata log file - * @param logSegmentMinBytes The minimum size of a single metadata log file + * @param internalLogSegmentMinBytes The minimum size of a single metadata log file * @param logSegmentMillis The maximum time before a new metadata log file is rolled out * @param retentionMaxBytes The size of the metadata log and snapshots before deleting old snapshots and log files * @param retentionMillis The time to keep a metadata log file or snapshot before deleting it @@ -112,7 +107,7 @@ public class MetadataLogConfig { * @param deleteDelayMillis The amount of time to wait before deleting a file from the filesystem */ public MetadataLogConfig(int logSegmentBytes, - int logSegmentMinBytes, + int internalLogSegmentMinBytes, long logSegmentMillis, long retentionMaxBytes, long retentionMillis, @@ -120,7 +115,7 @@ public MetadataLogConfig(int logSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = logSegmentBytes; - this.logSegmentMinBytes = logSegmentMinBytes; + this.internalLogSegmentMinBytes = internalLogSegmentMinBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; @@ -131,7 +126,7 @@ public MetadataLogConfig(int logSegmentBytes, public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); - this.logSegmentMinBytes = config.getInt(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG); + this.internalLogSegmentMinBytes = config.getInt("metadata.log.segment.min.bytes"); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); @@ -141,13 +136,10 @@ public MetadataLogConfig(AbstractConfig config) { } public int logSegmentBytes() { + if (internalLogSegmentMinBytes != null) return internalLogSegmentMinBytes; return logSegmentBytes; } - public int logSegmentMinBytes() { - return logSegmentMinBytes; - } - public long logSegmentMillis() { return logSegmentMillis; } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index cb3739cd7d81c..4596c73220423 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -121,8 +121,14 @@ public Optional serverConfigName(String configName) { // Visible for testing public static final String SERVER_DEFAULT_HEADER_NAME = "Server Default Property"; - // visible for testing + // only for testing public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; + public static final String LOG_SEGMENT_BYTES_DOC = "Setting the maximum size of a single log file. This should be used for testing only."; + // only for testing + public static final String INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG = "metadata.log.segment.min.bytes"; + private static final String INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_DOC = + "Override the minimum size for a single metadata log file. This should be used for testing only."; + private static final int METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT = 8 * 1024 * 1024; public static final int DEFAULT_SEGMENT_BYTES = 1024 * 1024 * 1024; public static final long DEFAULT_SEGMENT_MS = 24 * 7 * 60 * 60 * 1000L; @@ -146,7 +152,8 @@ public Optional serverConfigName(String configName) { TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, - LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, + INTERNAL_SEGMENT_BYTES_CONFIG, + INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG ); @@ -189,7 +196,9 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) - .defineInternal(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); + .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, LOG_SEGMENT_BYTES_DOC) + .defineInternal(INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_DOC); + private static final LogConfigDef CONFIG = new LogConfigDef(); static { @@ -257,7 +266,9 @@ public Optional serverConfigName(String configName) { TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) - .defineInternal(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC); + .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, LOG_SEGMENT_BYTES_DOC) + .defineInternal(INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_DOC); + } public final Set overriddenConfigs; From 34a794fcfdddac4ae7154ce9dc92ee696e79f744 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 20 Apr 2025 16:31:52 +0800 Subject: [PATCH 27/74] update KafkaMetadataLog apply flow --- .../scala/kafka/raft/KafkaMetadataLog.scala | 34 ++++++++----------- 1 file changed, 14 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 057f372ff9e36..600ccdc57ce8d 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -593,7 +593,11 @@ object KafkaMetadataLog extends Logging { val metadataLog: KafkaMetadataLog = createMetadataLog(topicPartition, topicId, dataDir, time, scheduler, config, nodeId, defaultLogConfig) - printInternalConfigWarningMessage(config, metadataLog) + // Print a warning if users have overridden the internal config + if (config.logSegmentBytes() != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) { + metadataLog.error(s"Overriding ${LogConfig.INTERNAL_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.") + } // 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() @@ -620,8 +624,6 @@ object KafkaMetadataLog extends Logging { validateConfig(defaultLogConfig) val metadataLog: KafkaMetadataLog = createMetadataLog(topicPartition, topicId, dataDir, time, scheduler, config, nodeId, defaultLogConfig) - - printInternalConfigWarningMessage(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() @@ -667,11 +669,11 @@ object KafkaMetadataLog extends Logging { metadataLog } - private def settingLogProperties(config: MetadataLogConfig) = { + private def settingLogProperties(metadataLogConfig: 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.MAX_MESSAGE_BYTES_CONFIG, metadataLogConfig.maxBatchSizeInBytes.toString) + props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, metadataLogConfig.logSegmentBytes.toString) + props.setProperty(TopicConfig.SEGMENT_MS_CONFIG, metadataLogConfig.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 @@ -680,26 +682,18 @@ object KafkaMetadataLog extends Logging { props } - private def validateConfig(defaultLogConfig: LogConfig): Unit = { - if (defaultLogConfig.retentionMs >= 0) { + private def validateConfig(logConfig: LogConfig): Unit = { + if (logConfig.retentionMs >= 0) { throw new InvalidConfigurationException( - s"Cannot set ${TopicConfig.RETENTION_MS_CONFIG} above -1: ${defaultLogConfig.retentionMs}." + s"Cannot set ${TopicConfig.RETENTION_MS_CONFIG} above -1: ${logConfig.retentionMs}." ) - } else if (defaultLogConfig.retentionSize >= 0) { + } else if (logConfig.retentionSize >= 0) { throw new InvalidConfigurationException( - s"Cannot set ${TopicConfig.RETENTION_BYTES_CONFIG} above -1: ${defaultLogConfig.retentionSize}." + s"Cannot set ${TopicConfig.RETENTION_BYTES_CONFIG} above -1: ${logConfig.retentionSize}." ) } } - private def printInternalConfigWarningMessage(config: MetadataLogConfig, metadataLog: KafkaMetadataLog): Unit = { - // Print a warning if users have overridden the internal config - if (config.logSegmentBytes() != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) { - metadataLog.error(s"Overriding ${LogConfig.INTERNAL_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 recoverSnapshots( log: UnifiedLog ): mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]] = { From c10c9cb9365ef6580276655afaa2040ae5f6f1d3 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 20 Apr 2025 20:41:30 +0800 Subject: [PATCH 28/74] fix compile error --- .../test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 89e070eb0021a..75b633c61c3c1 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.internals.Plugin import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig} +import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.server.DynamicThreadPool import org.apache.kafka.server.authorizer._ From ea981e308baebe4bc9229b971063a284d3b6589c Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 21 Apr 2025 11:21:57 +0800 Subject: [PATCH 29/74] fix fail test --- build.gradle | 5 +++-- checkstyle/import-control.xml | 3 ++- .../kafka/api/SaslSslAdminIntegrationTest.scala | 7 +++++-- .../src/test/scala/kafka/raft/KafkaMetadataLogTest.scala | 8 +------- core/src/test/scala/unit/kafka/log/LogConfigTest.scala | 1 + .../test/scala/unit/kafka/server/KafkaConfigTest.scala | 5 +++-- .../java/org/apache/kafka/raft/MetadataLogConfig.java | 3 ++- .../apache/kafka/storage/internals/log/LogConfig.java | 9 ++++----- 8 files changed, 21 insertions(+), 20 deletions(-) diff --git a/build.gradle b/build.gradle index 2e35057165c53..f2db8a973ef79 100644 --- a/build.gradle +++ b/build.gradle @@ -159,7 +159,7 @@ ext { libs.log4j2Api, libs.log4j2Core ] - + } allprojects { @@ -552,7 +552,7 @@ subprojects { maxFailures = userMaxTestRetryFailures } } - + finalizedBy("copyTestXml") } @@ -2010,6 +2010,7 @@ project(':raft') { dependencies { implementation project(':server-common') implementation project(':clients') + implementation project(':storage') implementation libs.jacksonDatabind implementation libs.slf4jApi diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index ead4111389bb3..4f8d007472053 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -46,7 +46,7 @@ - + @@ -480,6 +480,7 @@ + diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index f684d98aa0197..bd900d14f9db0 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -568,7 +568,10 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu client.createAcls(List(denyAcl).asJava, new CreateAclsOptions()).all().get() val topics = Seq(topic1, topic2) - val configsOverride = Map(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG -> "100000").asJava + val configsOverride = util.Map.of( + LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "100000", + LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, 8 * 1024 * 1024 + "", + ) 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)) @@ -602,7 +605,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu assertNotEquals(Uuid.ZERO_UUID, createResult.topicId(topic1).get()) assertEquals(topicIds(topic1), createResult.topicId(topic1).get()) assertFutureThrows(classOf[TopicAuthorizationException], createResult.topicId(topic2)) - + val createResponseConfig = createResult.config(topic1).get().entries.asScala val describeResponseConfig = describeConfigs(topic1) diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index f3a2fe94b6d6c..506ce03420a25 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -20,7 +20,7 @@ import kafka.server.{KafkaConfig, KafkaRaftServer} import kafka.utils.TestUtils import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.errors.CorruptRecordException -import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException} +import org.apache.kafka.common.errors.RecordTooLargeException import org.apache.kafka.common.protocol import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable} import org.apache.kafka.common.record.ArbitraryMemoryRecords @@ -78,12 +78,6 @@ final class KafkaMetadataLogTest { props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240)) props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024)) - assertThrows(classOf[InvalidConfigurationException], () => { - val kafkaConfig = KafkaConfig.fromProps(props) - val metadataConfig = new MetadataLogConfig(kafkaConfig) - buildMetadataLog(tempDir, mockTime, metadataConfig) - }) - props.put(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 752d41e54a9c1..b3fe7407bd14b 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -95,6 +95,7 @@ class LogConfigTest { case TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // no op + case LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG => // no op case _ => assertPropertyInvalid(name, "not_a_number", "-1") }) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index c356d63a22d18..091ab0c8caa30 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -895,8 +895,9 @@ class KafkaConfigTest { case ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", CompressionType.ZSTD.maxLevel() + 1) - - case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // ignore int + + case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // ignore int + case LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG => // ignore int //SSL Configs case BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => case BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG => diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index 46faa42985494..f90c7c4e33ceb 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.record.Records; import org.apache.kafka.server.config.ServerLogConfigs; +import org.apache.kafka.storage.internals.log.LogConfig; import java.util.concurrent.TimeUnit; @@ -126,7 +127,7 @@ public MetadataLogConfig(int logSegmentBytes, public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); - this.internalLogSegmentMinBytes = config.getInt("metadata.log.segment.min.bytes"); + this.internalLogSegmentMinBytes = config.getInt(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 4596c73220423..2397ae1d34d66 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -121,14 +121,13 @@ public Optional serverConfigName(String configName) { // Visible for testing public static final String SERVER_DEFAULT_HEADER_NAME = "Server Default Property"; - // only for testing + // only for testing public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String LOG_SEGMENT_BYTES_DOC = "Setting the maximum size of a single log file. This should be used for testing only."; - // only for testing + // only for testing public static final String INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG = "metadata.log.segment.min.bytes"; - private static final String INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_DOC = + private static final String INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_DOC = "Override the minimum size for a single metadata log file. This should be used for testing only."; - private static final int METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT = 8 * 1024 * 1024; public static final int DEFAULT_SEGMENT_BYTES = 1024 * 1024 * 1024; public static final long DEFAULT_SEGMENT_MS = 24 * 7 * 60 * 60 * 1000L; @@ -198,7 +197,7 @@ public Optional serverConfigName(String configName) { .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, LOG_SEGMENT_BYTES_DOC) .defineInternal(INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_DOC); - + private static final LogConfigDef CONFIG = new LogConfigDef(); static { From 934ac37a39f92a351d7a858457cecc9a3a983c2c Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 21 Apr 2025 11:26:42 +0800 Subject: [PATCH 30/74] fix fail test --- build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index f2db8a973ef79..068ec94aa5533 100644 --- a/build.gradle +++ b/build.gradle @@ -159,7 +159,7 @@ ext { libs.log4j2Api, libs.log4j2Core ] - + } allprojects { @@ -552,7 +552,7 @@ subprojects { maxFailures = userMaxTestRetryFailures } } - + finalizedBy("copyTestXml") } From 77898ed26ba334841e1a1765724972ac6c66c4e6 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 21 Apr 2025 11:27:35 +0800 Subject: [PATCH 31/74] revert unused change --- checkstyle/import-control.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 4f8d007472053..8d6801e4e450a 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -46,7 +46,7 @@ - + From 1756fee7a1dd1955350c2c427fbe4f506d94eed1 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 21 Apr 2025 11:28:23 +0800 Subject: [PATCH 32/74] revert unused change --- checkstyle/import-control.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 8d6801e4e450a..02a83587686e7 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -46,7 +46,7 @@ - + @@ -480,7 +480,6 @@ - @@ -489,6 +488,7 @@ + From 1a3f737eca75526381da3ff284ac767d3c190c89 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 22 Apr 2025 23:27:23 +0800 Subject: [PATCH 33/74] temp --- .../src/main/scala/kafka/raft/KafkaMetadataLog.scala | 8 +------- .../org/apache/kafka/raft/MetadataLogConfig.java | 12 ++++++------ .../kafka/storage/internals/log/LogConfig.java | 1 + 3 files changed, 8 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 600ccdc57ce8d..9fa51879e936c 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record.{MemoryRecords, Records} import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} -import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} +import org.apache.kafka.raft.{Isolation, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.Scheduler @@ -592,12 +592,6 @@ object KafkaMetadataLog extends Logging { validateConfig(defaultLogConfig) val metadataLog: KafkaMetadataLog = createMetadataLog(topicPartition, topicId, dataDir, time, scheduler, config, nodeId, defaultLogConfig) - - // Print a warning if users have overridden the internal config - if (config.logSegmentBytes() != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) { - metadataLog.error(s"Overriding ${LogConfig.INTERNAL_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.") - } // 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() diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index f90c7c4e33ceb..98d4613188d10 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -88,7 +88,7 @@ public class MetadataLogConfig { .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); private final int logSegmentBytes; - private final Integer internalLogSegmentMinBytes; + private final Integer internalLogSegmentBytes; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -99,7 +99,7 @@ public class MetadataLogConfig { /** * Configuration for the metadata log * @param logSegmentBytes The maximum size of a single metadata log file - * @param internalLogSegmentMinBytes The minimum size of a single metadata log file + * @param internalLogSegmentBytes the internal log segment size for the metadata log * @param logSegmentMillis The maximum time before a new metadata log file is rolled out * @param retentionMaxBytes The size of the metadata log and snapshots before deleting old snapshots and log files * @param retentionMillis The time to keep a metadata log file or snapshot before deleting it @@ -108,7 +108,7 @@ public class MetadataLogConfig { * @param deleteDelayMillis The amount of time to wait before deleting a file from the filesystem */ public MetadataLogConfig(int logSegmentBytes, - int internalLogSegmentMinBytes, + int internalLogSegmentBytes, long logSegmentMillis, long retentionMaxBytes, long retentionMillis, @@ -116,7 +116,7 @@ public MetadataLogConfig(int logSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = logSegmentBytes; - this.internalLogSegmentMinBytes = internalLogSegmentMinBytes; + this.internalLogSegmentBytes = internalLogSegmentBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; @@ -127,7 +127,7 @@ public MetadataLogConfig(int logSegmentBytes, public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); - this.internalLogSegmentMinBytes = config.getInt(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG); + this.internalLogSegmentBytes = config.getInt(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); @@ -137,7 +137,7 @@ public MetadataLogConfig(AbstractConfig config) { } public int logSegmentBytes() { - if (internalLogSegmentMinBytes != null) return internalLogSegmentMinBytes; + if (internalLogSegmentBytes != null) return internalLogSegmentBytes; return logSegmentBytes; } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 2397ae1d34d66..b66819f90002d 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -195,6 +195,7 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) + // only for testing .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, LOG_SEGMENT_BYTES_DOC) .defineInternal(INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_DOC); From 7483a5ddcbd0b1bb5578d010ae808b618540b1c8 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 25 Apr 2025 10:00:20 +0800 Subject: [PATCH 34/74] completed the feature --- build.gradle | 1 - checkstyle/import-control.xml | 1 - .../scala/kafka/raft/KafkaMetadataLog.scala | 96 +++++-------------- .../api/SaslSslAdminIntegrationTest.scala | 5 +- .../kafka/raft/KafkaMetadataLogTest.scala | 11 +-- .../scala/unit/kafka/log/LogConfigTest.scala | 1 - .../server/DynamicBrokerConfigTest.scala | 8 +- .../unit/kafka/server/KafkaConfigTest.scala | 3 +- .../kafka/tools/DumpLogSegmentsTest.scala | 3 +- .../apache/kafka/raft/MetadataLogConfig.java | 22 +++-- .../storage/internals/log/LogConfig.java | 18 +--- 11 files changed, 52 insertions(+), 117 deletions(-) diff --git a/build.gradle b/build.gradle index f08330b6fab1d..036682572e48b 100644 --- a/build.gradle +++ b/build.gradle @@ -2008,7 +2008,6 @@ project(':raft') { dependencies { implementation project(':server-common') implementation project(':clients') - implementation project(':storage') implementation libs.jacksonDatabind implementation libs.slf4jApi diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 02a83587686e7..ead4111389bb3 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -488,7 +488,6 @@ - diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 9fa51879e936c..f44f30efdb14b 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -585,56 +585,31 @@ object KafkaMetadataLog extends Logging { config: MetadataLogConfig, nodeId: Int ): KafkaMetadataLog = { - val props: Properties = settingLogProperties(config) - LogConfig.validate(props) - val defaultLogConfig = new LogConfig(props) - - validateConfig(defaultLogConfig) - - val metadataLog: KafkaMetadataLog = createMetadataLog(topicPartition, topicId, dataDir, time, scheduler, config, nodeId, defaultLogConfig) - // 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 - } + val props = new Properties() + props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString) + if (config.internalLogSegmentBytes == null) + props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) + else + 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) - // only for testing - def internalApply( - topicPartition: TopicPartition, - topicId: Uuid, - dataDir: File, - time: Time, - scheduler: Scheduler, - config: MetadataLogConfig, - nodeId: Int - ): KafkaMetadataLog = { - val props: Properties = settingLogProperties(config) - props.remove(TopicConfig.SEGMENT_BYTES_CONFIG) - props.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) + // Disable time and byte retention when deleting segments + props.setProperty(TopicConfig.RETENTION_MS_CONFIG, "-1") + props.setProperty(TopicConfig.RETENTION_BYTES_CONFIG, "-1") LogConfig.validate(props) val defaultLogConfig = new LogConfig(props) + + 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(defaultLogConfig) - - val metadataLog: KafkaMetadataLog = createMetadataLog(topicPartition, topicId, dataDir, time, scheduler, config, nodeId, defaultLogConfig) - // 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 createMetadataLog( - topicPartition: TopicPartition, - topicId: Uuid, - dataDir: File, - time: Time, - scheduler: Scheduler, - config: MetadataLogConfig, - nodeId: Int, - defaultLogConfig: LogConfig - ) = { val log = UnifiedLog.create( dataDir, defaultLogConfig, @@ -660,32 +635,11 @@ object KafkaMetadataLog extends Logging { config, nodeId ) - metadataLog - } - - private def settingLogProperties(metadataLogConfig: MetadataLogConfig) = { - val props = new Properties() - props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, metadataLogConfig.maxBatchSizeInBytes.toString) - props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, metadataLogConfig.logSegmentBytes.toString) - props.setProperty(TopicConfig.SEGMENT_MS_CONFIG, metadataLogConfig.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") - props - } + // 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 validateConfig(logConfig: LogConfig): Unit = { - if (logConfig.retentionMs >= 0) { - throw new InvalidConfigurationException( - s"Cannot set ${TopicConfig.RETENTION_MS_CONFIG} above -1: ${logConfig.retentionMs}." - ) - } else if (logConfig.retentionSize >= 0) { - throw new InvalidConfigurationException( - s"Cannot set ${TopicConfig.RETENTION_BYTES_CONFIG} above -1: ${logConfig.retentionSize}." - ) - } + metadataLog } private def recoverSnapshots( diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index bd900d14f9db0..c74d957974360 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -568,10 +568,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu client.createAcls(List(denyAcl).asJava, new CreateAclsOptions()).all().get() val topics = Seq(topic1, topic2) - val configsOverride = util.Map.of( - LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "100000", - LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, 8 * 1024 * 1024 + "", - ) + val configsOverride = Map(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG -> "100000").asJava 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)) diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 506ce03420a25..52e9189d381b5 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -76,9 +76,8 @@ final class KafkaMetadataLogTest { props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9093") props.put(KRaftConfigs.NODE_ID_CONFIG, Int.box(2)) props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240)) props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024)) - props.put(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) + props.put(MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240)) val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) @@ -682,7 +681,6 @@ final class KafkaMetadataLogTest { val maxBatchSizeInBytes = 16384 val recordSize = 64 val config = new MetadataLogConfig( - DefaultMetadataLogConfig.logSegmentBytes, DefaultMetadataLogConfig.logSegmentBytes, DefaultMetadataLogConfig.logSegmentMillis, DefaultMetadataLogConfig.retentionMaxBytes, @@ -902,7 +900,6 @@ final class KafkaMetadataLogTest { @Test def testAdvanceLogStartOffsetAfterCleaning(): Unit = { val config = new MetadataLogConfig( - 512, 512, 10 * 1000, 256, @@ -939,7 +936,6 @@ final class KafkaMetadataLogTest { def testDeleteSnapshots(): Unit = { // Generate some logs and a few snapshots, set retention low and verify that cleaning occurs val config = new MetadataLogConfig( - 1024, 1024, 10 * 1000, 1024, @@ -973,7 +969,6 @@ final class KafkaMetadataLogTest { def testSoftRetentionLimit(): Unit = { // Set retention equal to the segment size and generate slightly more than one segment of logs val config = new MetadataLogConfig( - 10240, 10240, 10 * 1000, 10240, @@ -1017,7 +1012,6 @@ final class KafkaMetadataLogTest { @Test def testSegmentsLessThanLatestSnapshot(): Unit = { val config = new MetadataLogConfig( - 10240, 10240, 10 * 1000, 10240, @@ -1076,7 +1070,6 @@ object KafkaMetadataLogTest { } val DefaultMetadataLogConfig = new MetadataLogConfig( - 100 * 1024, 100 * 1024, 10 * 1000, 100 * 1024, @@ -1097,7 +1090,7 @@ object KafkaMetadataLogTest { UnifiedLog.logDirName(KafkaRaftServer.MetadataPartition) ) - val metadataLog = KafkaMetadataLog.internalApply( + val metadataLog = KafkaMetadataLog.apply( KafkaRaftServer.MetadataPartition, KafkaRaftServer.MetadataTopicId, logDir, diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index b3fe7407bd14b..752d41e54a9c1 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -95,7 +95,6 @@ class LogConfigTest { case TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // no op - case LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG => // no op case _ => assertPropertyInvalid(name, "not_a_number", "-1") }) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 75b633c61c3c1..3888ec8aa5d48 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.internals.Plugin import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.raft.QuorumConfig +import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.server.DynamicThreadPool import org.apache.kafka.server.authorizer._ @@ -673,11 +673,11 @@ class DynamicBrokerConfigTest { @Test def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = { val props = TestUtils.createBrokerConfig(0, port = 8181) - props.put(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, "1024") + props.put(MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, "1024") val config = new KafkaConfig(props) - assertFalse(config.nonInternalValues.containsKey(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) + assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG)) config.updateCurrentConfig(new KafkaConfig(props)) - assertFalse(config.nonInternalValues.containsKey(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) + assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG)) } @Test diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 70776b377de22..489f1fdf2fde2 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -897,7 +897,8 @@ class KafkaConfigTest { case ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", CompressionType.ZSTD.maxLevel() + 1) case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // ignore int - case LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG => // ignore int + case MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG => // ignore int + //SSL Configs case BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => case BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG => diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index e50820a105800..b86b25f9b1a15 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -538,14 +538,13 @@ class DumpLogSegmentsTest { setPartitionId(0).setIsr(util.Arrays.asList(0, 1, 2)), 0.toShort) ) - val metadataLog = KafkaMetadataLog.internalApply( + val metadataLog = KafkaMetadataLog.apply( KafkaRaftServer.MetadataPartition, KafkaRaftServer.MetadataTopicId, logDir, time, time.scheduler, new MetadataLogConfig( - 100 * 1024, 100 * 1024, 10 * 1000, 100 * 1024, diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index 98d4613188d10..e6d53e02c832f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -18,9 +18,7 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.record.Records; import org.apache.kafka.server.config.ServerLogConfigs; -import org.apache.kafka.storage.internals.log.LogConfig; import java.util.concurrent.TimeUnit; @@ -57,6 +55,10 @@ public class MetadataLogConfig { public static final String METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file."; public static final int METADATA_LOG_SEGMENT_BYTES_DEFAULT = 1024 * 1024 * 1024; + public static final String INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG = "internal.metadata.log.segment.bytes"; + private static final String INTERNAL_METADATA_LOG_SEGMENT_BYTES_DOC = + "Override the minimum size for a single metadata log file. This should be used for testing only."; + public static final String METADATA_LOG_SEGMENT_MILLIS_CONFIG = "metadata.log.segment.ms"; public static final String METADATA_LOG_SEGMENT_MILLIS_DOC = "The maximum time before a new metadata log file is rolled out (in milliseconds)."; public static final long METADATA_LOG_SEGMENT_MILLIS_DEFAULT = 24 * 7 * 60 * 60 * 1000L; @@ -81,14 +83,15 @@ public class MetadataLogConfig { .define(METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG, LONG, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_DOC) .define(METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC) .define(METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, METADATA_LOG_DIR_DOC) - .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) + .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(8 * 1024 * 1024), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) - .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); + .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC) + .defineInternal(INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_METADATA_LOG_SEGMENT_BYTES_DOC); private final int logSegmentBytes; - private final Integer internalLogSegmentBytes; + private Integer internalLogSegmentBytes = null; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -99,7 +102,6 @@ public class MetadataLogConfig { /** * Configuration for the metadata log * @param logSegmentBytes The maximum size of a single metadata log file - * @param internalLogSegmentBytes the internal log segment size for the metadata log * @param logSegmentMillis The maximum time before a new metadata log file is rolled out * @param retentionMaxBytes The size of the metadata log and snapshots before deleting old snapshots and log files * @param retentionMillis The time to keep a metadata log file or snapshot before deleting it @@ -108,7 +110,6 @@ public class MetadataLogConfig { * @param deleteDelayMillis The amount of time to wait before deleting a file from the filesystem */ public MetadataLogConfig(int logSegmentBytes, - int internalLogSegmentBytes, long logSegmentMillis, long retentionMaxBytes, long retentionMillis, @@ -116,7 +117,6 @@ public MetadataLogConfig(int logSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = logSegmentBytes; - this.internalLogSegmentBytes = internalLogSegmentBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; @@ -127,7 +127,7 @@ public MetadataLogConfig(int logSegmentBytes, public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); - this.internalLogSegmentBytes = config.getInt(LogConfig.INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG); + this.internalLogSegmentBytes = config.getInt(INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); @@ -141,6 +141,10 @@ public int logSegmentBytes() { return logSegmentBytes; } + public Integer internalLogSegmentBytes() { + return internalLogSegmentBytes; + } + public long logSegmentMillis() { return logSegmentMillis; } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index b66819f90002d..d762e570845f6 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -121,13 +121,9 @@ public Optional serverConfigName(String configName) { // Visible for testing public static final String SERVER_DEFAULT_HEADER_NAME = "Server Default Property"; - // only for testing + public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; - public static final String LOG_SEGMENT_BYTES_DOC = "Setting the maximum size of a single log file. This should be used for testing only."; - // only for testing - public static final String INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG = "metadata.log.segment.min.bytes"; - private static final String INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_DOC = - "Override the minimum size for a single metadata log file. This should be used for testing only."; + public static final String INTERNAL_LOG_SEGMENT_BYTES_DOC = "Setting the maximum size of a single log file. This should be used for testing only."; public static final int DEFAULT_SEGMENT_BYTES = 1024 * 1024 * 1024; public static final long DEFAULT_SEGMENT_MS = 24 * 7 * 60 * 60 * 1000L; @@ -152,7 +148,6 @@ public Optional serverConfigName(String configName) { TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, INTERNAL_SEGMENT_BYTES_CONFIG, - INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG ); @@ -194,11 +189,7 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) - .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) - // only for testing - .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, LOG_SEGMENT_BYTES_DOC) - .defineInternal(INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_DOC); - + .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { @@ -266,8 +257,7 @@ public Optional serverConfigName(String configName) { TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) - .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, LOG_SEGMENT_BYTES_DOC) - .defineInternal(INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_METADATA_LOG_SEGMENT_MIN_BYTES_DOC); + .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_LOG_SEGMENT_BYTES_DOC); } From 048e052f68b2cc9529ca030c32f1a8fb68c79964 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 25 Apr 2025 12:55:38 +0800 Subject: [PATCH 35/74] fix fail test --- core/src/main/scala/kafka/raft/KafkaMetadataLog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index f44f30efdb14b..051feeca3e4c5 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -587,7 +587,7 @@ object KafkaMetadataLog extends Logging { ): KafkaMetadataLog = { val props = new Properties() props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString) - if (config.internalLogSegmentBytes == null) + if (config.internalLogSegmentBytes != null) props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) else props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) From 76bd28710ea61ef906faaa475781de43b88d83bd Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 25 Apr 2025 13:49:31 +0800 Subject: [PATCH 36/74] fix fail test --- raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java | 1 + 1 file changed, 1 insertion(+) diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index e6d53e02c832f..b09d0411cc45b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -117,6 +117,7 @@ public MetadataLogConfig(int logSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = logSegmentBytes; + this.internalLogSegmentBytes = logSegmentBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; From ce644aeec1e7fbebd317fb493ef4f875dcaf1ada Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 27 Apr 2025 08:17:48 +0800 Subject: [PATCH 37/74] addressed by comments --- .../integration/kafka/api/SaslSslAdminIntegrationTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index c74d957974360..9ee334e31a494 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -568,7 +568,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu client.createAcls(List(denyAcl).asJava, new CreateAclsOptions()).all().get() val topics = Seq(topic1, topic2) - val configsOverride = Map(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG -> "100000").asJava + val configsOverride = Map(TopicConfig.SEGMENT_BYTES_CONFIG -> "100000").asJava 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)) @@ -602,7 +602,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu assertNotEquals(Uuid.ZERO_UUID, createResult.topicId(topic1).get()) assertEquals(topicIds(topic1), createResult.topicId(topic1).get()) assertFutureThrows(classOf[TopicAuthorizationException], createResult.topicId(topic2)) - + val createResponseConfig = createResult.config(topic1).get().entries.asScala val describeResponseConfig = describeConfigs(topic1) From a4d3f56d1a5ecab91057e0bb6bf7b85aff73b626 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 27 Apr 2025 16:21:24 +0800 Subject: [PATCH 38/74] remove metadataLogConfig --- .../scala/kafka/raft/KafkaMetadataLogTest.scala | 2 +- .../kafka/server/DynamicBrokerConfigTest.scala | 10 ---------- .../unit/kafka/server/KafkaConfigTest.scala | 3 +-- .../apache/kafka/raft/MetadataLogConfig.java | 17 ++--------------- 4 files changed, 4 insertions(+), 28 deletions(-) diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 52e9189d381b5..0c478078ce699 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -77,7 +77,7 @@ final class KafkaMetadataLogTest { props.put(KRaftConfigs.NODE_ID_CONFIG, Int.box(2)) props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024)) - props.put(MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240)) + props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(8 * 1024 * 1024)) val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 3888ec8aa5d48..98c33d81ead84 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -670,16 +670,6 @@ class DynamicBrokerConfigTest { assertTrue(m.currentReporters.isEmpty) } - @Test - def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = { - val props = TestUtils.createBrokerConfig(0, port = 8181) - props.put(MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, "1024") - val config = new KafkaConfig(props) - assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG)) - config.updateCurrentConfig(new KafkaConfig(props)) - assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG)) - } - @Test def testDynamicLogLocalRetentionMsConfig(): Unit = { val props = TestUtils.createBrokerConfig(0, port = 8181) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 489f1fdf2fde2..8b3035a4ca407 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -897,8 +897,7 @@ class KafkaConfigTest { case ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", CompressionType.ZSTD.maxLevel() + 1) case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // ignore int - case MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG => // ignore int - + //SSL Configs case BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => case BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG => diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index b09d0411cc45b..3901443af69b2 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -54,11 +54,7 @@ public class MetadataLogConfig { public static final String METADATA_LOG_SEGMENT_BYTES_CONFIG = "metadata.log.segment.bytes"; public static final String METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file."; public static final int METADATA_LOG_SEGMENT_BYTES_DEFAULT = 1024 * 1024 * 1024; - - public static final String INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG = "internal.metadata.log.segment.bytes"; - private static final String INTERNAL_METADATA_LOG_SEGMENT_BYTES_DOC = - "Override the minimum size for a single metadata log file. This should be used for testing only."; - + public static final String METADATA_LOG_SEGMENT_MILLIS_CONFIG = "metadata.log.segment.ms"; public static final String METADATA_LOG_SEGMENT_MILLIS_DOC = "The maximum time before a new metadata log file is rolled out (in milliseconds)."; public static final long METADATA_LOG_SEGMENT_MILLIS_DEFAULT = 24 * 7 * 60 * 60 * 1000L; @@ -87,11 +83,9 @@ public class MetadataLogConfig { .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) - .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC) - .defineInternal(INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_METADATA_LOG_SEGMENT_BYTES_DOC); + .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); private final int logSegmentBytes; - private Integer internalLogSegmentBytes = null; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -117,7 +111,6 @@ public MetadataLogConfig(int logSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = logSegmentBytes; - this.internalLogSegmentBytes = logSegmentBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; @@ -128,7 +121,6 @@ public MetadataLogConfig(int logSegmentBytes, public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); - this.internalLogSegmentBytes = config.getInt(INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); @@ -138,14 +130,9 @@ public MetadataLogConfig(AbstractConfig config) { } public int logSegmentBytes() { - if (internalLogSegmentBytes != null) return internalLogSegmentBytes; return logSegmentBytes; } - public Integer internalLogSegmentBytes() { - return internalLogSegmentBytes; - } - public long logSegmentMillis() { return logSegmentMillis; } From 95d8e28cd62c65315707cb443c8a7aaccc7349b9 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 27 Apr 2025 20:00:31 +0800 Subject: [PATCH 39/74] complete --- .../scala/kafka/raft/KafkaMetadataLog.scala | 5 +- .../api/PlaintextAdminIntegrationTest.scala | 2 +- .../api/SaslSslAdminIntegrationTest.scala | 2 +- .../kafka/cluster/AbstractPartitionTest.scala | 4 +- .../kafka/cluster/PartitionLockTest.scala | 4 +- .../AbstractLogCleanerIntegrationTest.scala | 3 +- .../kafka/log/LogCleanerManagerTest.scala | 19 ++--- .../scala/unit/kafka/log/LogCleanerTest.scala | 73 ++++++++++--------- .../unit/kafka/log/LogConcurrencyTest.scala | 3 +- .../scala/unit/kafka/log/LogConfigTest.scala | 1 - .../scala/unit/kafka/log/LogLoaderTest.scala | 3 +- .../scala/unit/kafka/log/LogManagerTest.scala | 7 +- .../scala/unit/kafka/log/LogTestUtils.scala | 2 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 5 +- .../AlterReplicaLogDirsRequestTest.scala | 2 +- ...ControllerConfigurationValidatorTest.scala | 5 +- .../server/DynamicBrokerConfigTest.scala | 2 +- .../server/DynamicConfigChangeTest.scala | 4 +- .../unit/kafka/server/KafkaConfigTest.scala | 4 +- .../ReplicationControlManagerTest.java | 4 +- .../kafka/server/config/ServerLogConfigs.java | 3 + .../share/ShareCoordinatorServiceTest.java | 3 +- .../storage/internals/log/LogConfig.java | 11 +-- .../storage/internals/log/LocalLogTest.java | 3 +- .../PurgeRepartitionTopicIntegrationTest.java | 5 +- .../kafka/tools/GetOffsetShellTest.java | 3 +- 26 files changed, 92 insertions(+), 90 deletions(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 051feeca3e4c5..ea1259e29d5e6 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -587,10 +587,7 @@ object KafkaMetadataLog extends Logging { ): KafkaMetadataLog = { val props = new Properties() props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString) - if (config.internalLogSegmentBytes != null) - props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) - else - props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.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) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index f0353009b6d1b..6bc77a193a74d 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -1568,7 +1568,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @MethodSource(Array("getTestGroupProtocolParametersAll")) def testDeleteRecordsAfterCorruptRecords(groupProtocol: String): Unit = { val config = new Properties() - config.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "200") + config.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "200") createTopic(topic, numPartitions = 1, replicationFactor = 1, config) client = createAdminClient diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 9ee334e31a494..33bb2e161f40d 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -581,7 +581,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 == LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG).get + val segmentBytesConfig = topicConfigs.find(_.name == ServerLogConfigs.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 diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index fff1930a71836..9bf6e2f025c58 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.requests.LeaderAndIsrRequest import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.{MetadataCache, MockConfigRepository} import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpoints import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} @@ -100,7 +100,7 @@ class AbstractPartitionTest { def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 512: java.lang.Integer) + logProps.put(ServerLogConfigs.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) } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 77b098cf68298..e4b026e2889ef 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.{LeaderAndIsr, MetadataCache, MockConfigRepository} import org.apache.kafka.server.common.{RequestLocal, TopicIdPartition} -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpoints @@ -362,7 +362,7 @@ class PartitionLockTest extends Logging { private def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 512: java.lang.Integer) + logProps.put(ServerLogConfigs.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) } diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index b1e161b975340..493bbccd3f792 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion} import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.transaction.TransactionLogConfig +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{CleanerConfig, LogCleaner, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, UnifiedLog} import org.apache.kafka.storage.log.metrics.BrokerTopicStats @@ -71,7 +72,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(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentSize: java.lang.Integer) + props.put(ServerLogConfigs.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) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 8445baa7719fb..c85eab16ee834 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -27,6 +27,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.coordinator.transaction.TransactionLogConfig +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.LogCleaningState.{LOG_CLEANING_ABORTED, LOG_CLEANING_IN_PROGRESS} import org.apache.kafka.storage.internals.log.{AppendOrigin, LocalLog, LogCleanerManager, LogCleaningException, LogCleaningState, LogConfig, LogDirFailureChannel, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, PreCleanStats, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} @@ -53,7 +54,7 @@ class LogCleanerManagerTest extends Logging { val topicPartition = new TopicPartition("log", 0) val topicPartition2 = new TopicPartition("log2", 0) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.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) @@ -370,7 +371,7 @@ class LogCleanerManagerTest extends Logging { // change cleanup policy from delete to compact val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, log.config.segmentSize(): Integer) + logProps.put(ServerLogConfigs.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) @@ -548,7 +549,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsForNone(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -570,7 +571,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsActiveSegment(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -592,7 +593,7 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.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)) @@ -625,7 +626,7 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForShortTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.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)) @@ -667,7 +668,7 @@ class LogCleanerManagerTest extends Logging { def testUndecidedTransactionalDataNotCleanable(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.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)) @@ -711,7 +712,7 @@ class LogCleanerManagerTest extends Logging { @Test def testDoneCleaning(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.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) @@ -830,7 +831,7 @@ class LogCleanerManagerTest extends Logging { private def createLowRetentionLogConfig(segmentSize: Int, cleanupPolicy: String): LogConfig = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentSize: Integer) + logProps.put(ServerLogConfigs.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 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 1bebfaa49e173..4c2a08d9f4acd 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.errors.CorruptRecordException import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanedTransactionMetadata, Cleaner, CleanerConfig, CleanerStats, LocalLog, LogAppendInfo, LogCleaner, LogCleanerManager, LogCleaningAbortedException, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetMap, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} @@ -56,7 +57,7 @@ class LogCleanerTest extends Logging { val tmpdir = TestUtils.tempDir() val dir = TestUtils.randomPartitionLogDir(tmpdir) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.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 = new LogConfig(logProps) @@ -148,7 +149,7 @@ class LogCleanerTest extends Logging { def testCleanSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -181,7 +182,7 @@ class LogCleanerTest extends Logging { // Construct a log instance. The replaceSegments() method of the log instance is overridden so that // it waits for another thread to execute deleteOldSegments() val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) @@ -271,7 +272,7 @@ class LogCleanerTest extends Logging { val originalMaxFileSize = 1024 val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact": java.lang.String) logProps.put(TopicConfig.PREALLOCATE_CONFIG, "true": java.lang.String) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -295,7 +296,7 @@ class LogCleanerTest extends Logging { def testDuplicateCheckAfterCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) var log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -464,7 +465,7 @@ class LogCleanerTest extends Logging { def testBasicTransactionAwareCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -497,7 +498,7 @@ class LogCleanerTest extends Logging { def testCleanWithTransactionsSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -543,7 +544,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -591,7 +592,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(capacity = Int.MaxValue, maxMessageSize = 100) val logProps = new Properties() logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, 100: java.lang.Integer) - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -619,7 +620,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -682,7 +683,7 @@ class LogCleanerTest extends Logging { def testCleanEmptyControlBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -714,7 +715,7 @@ class LogCleanerTest extends Logging { def testCommittedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -736,7 +737,7 @@ class LogCleanerTest extends Logging { def testAbortedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -765,7 +766,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -799,7 +800,7 @@ class LogCleanerTest extends Logging { val producerId = 1L val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, 0, AppendOrigin.REPLICATION) @@ -832,7 +833,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -896,7 +897,7 @@ class LogCleanerTest extends Logging { // Create cleaner with very small default max message size val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -968,7 +969,7 @@ class LogCleanerTest extends Logging { def createLogWithMessagesLargerThanMaxSize(largeMessageSize: Int): (UnifiedLog, FakeOffsetMap) = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -994,7 +995,7 @@ class LogCleanerTest extends Logging { def testCleaningWithDeletes(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1021,7 +1022,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 3 messages in the map val cleaner = makeCleaner(4) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1050,7 +1051,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsProducerLastSequence(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) log.appendAsLeader(record(0, 0), 0) // offset 0 @@ -1073,7 +1074,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsLastSequenceEvenIfTransactionAborted(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -1107,7 +1108,7 @@ class LogCleanerTest extends Logging { def testCleaningWithKeysConflictingWithTxnMarkerKeys(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val leaderEpoch = 5 val producerEpoch = 0.toShort @@ -1151,7 +1152,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 1 message in the map val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1182,7 +1183,7 @@ class LogCleanerTest extends Logging { def testCleaningWithUncleanableSection(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1225,7 +1226,7 @@ class LogCleanerTest extends Logging { def testLogToClean(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1243,7 +1244,7 @@ class LogCleanerTest extends Logging { def testLogToCleanWithUncleanableSection(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1276,7 +1277,7 @@ class LogCleanerTest extends Logging { // create a log with compaction turned off so we can append unkeyed messages val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1334,7 +1335,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsWithAbort(): Unit = { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1356,7 +1357,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsRetainingLastEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1388,7 +1389,7 @@ class LogCleanerTest extends Logging { def testSegmentGrouping(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1489,7 +1490,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1541,7 +1542,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) //mimic the effect of loading an empty index file logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 400: java.lang.Integer) @@ -1666,7 +1667,7 @@ class LogCleanerTest extends Logging { def testRecoveryAfterCrash(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 10: java.lang.Integer) @@ -1797,7 +1798,7 @@ class LogCleanerTest extends Logging { def testBuildOffsetMapFakeLarge(): Unit = { val map = new FakeOffsetMap(1000) val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) @@ -1945,7 +1946,7 @@ class LogCleanerTest extends Logging { @Test def testCleaningBeyondMissingOffsets(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) val cleaner = makeCleaner(Int.MaxValue) diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 854be39808661..a7e162dd76e7f 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -23,6 +23,7 @@ import kafka.utils.TestUtils import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.KafkaScheduler import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, UnifiedLog} @@ -59,7 +60,7 @@ class LogConcurrencyTest { @Test def testUncommittedDataNotConsumedFrequentSegmentRolls(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 237: Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 237: Integer) val logConfig = new LogConfig(logProps) testUncommittedDataNotConsumed(createLog(logConfig)) } diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 752d41e54a9c1..1e26d653bbccf 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -94,7 +94,6 @@ class LogConfigTest { case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1") case TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") - case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // no op case _ => assertPropertyInvalid(name, "not_a_number", "-1") }) diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 0c465cf2138df..974fdf00bcd5d 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -26,6 +26,7 @@ import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, Me import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.MockConfigRepository +import org.apache.kafka.server.config.ServerLogConfigs 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, LocalLog, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetMetadata, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile, UnifiedLog} @@ -244,7 +245,7 @@ class LogLoaderTest { @Test def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "640") + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "640") val logConfig = new LogConfig(logProps) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 04f4acca5debd..04aeeff4c45d8 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uui import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.{ConfigRepository, MockConfigRepository} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} +import org.apache.kafka.server.config.ServerLogConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any @@ -43,7 +44,7 @@ import java.util.{Collections, Optional, OptionalLong, Properties} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.{FileLock, KafkaScheduler, MockTime, Scheduler} -import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogManager => JLogManager, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} +import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog, LogManager => JLogManager} import org.apache.kafka.storage.internals.checkpoint.{CleanShutdownFileHandler, OffsetCheckpointFile} import org.apache.kafka.storage.log.metrics.BrokerTopicStats import org.junit.jupiter.api.function.Executable @@ -60,7 +61,7 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) val logConfig = new LogConfig(logProps) @@ -391,7 +392,7 @@ class LogManagerTest { logManager.shutdown() val segmentBytes = 10 * setSize val properties = new Properties() - properties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes.toString) + properties.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes.toString) properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString) val configRepository = MockConfigRepository.forTopic(name, properties) diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index 0ff68988d76fb..340b1b26e6e4e 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -75,7 +75,7 @@ object LogTestUtils { remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs: java.lang.Long) logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 42c813074aa9a..93f29e01d70d7 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -52,11 +52,10 @@ import org.junit.jupiter.params.provider.{EnumSource, ValueSource} import org.mockito.ArgumentMatchers import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{doAnswer, doThrow, spy} - import net.jqwik.api.AfterFailureMode import net.jqwik.api.ForAll import net.jqwik.api.Property -import org.apache.kafka.server.config.KRaftConfigs +import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} import java.io._ import java.nio.ByteBuffer @@ -2734,7 +2733,7 @@ class UnifiedLogTest { @Test def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000") + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "1000") logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") val logConfig = new LogConfig(logProps) diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index 78f5335caf42d..a0463182daca6 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -148,7 +148,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { // We don't want files with `.deleted` suffix are removed too fast, // so we can validate there will be orphan files and orphan files will be removed eventually. topicProperties.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, "10000") - topicProperties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1024") + topicProperties.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "1024") createTopic(topic, partitionNum, 1, topicProperties) assertEquals(logDir1, brokers.head.logManager.getLog(tp).get.dir.getParent) diff --git a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala index 180d1bda70828..ccbe0d043669a 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala @@ -23,6 +23,7 @@ import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, BROKER_LOGGER import org.apache.kafka.common.config.TopicConfig.{REMOTE_LOG_STORAGE_ENABLE_CONFIG, SEGMENT_BYTES_CONFIG, SEGMENT_JITTER_MS_CONFIG, SEGMENT_MS_CONFIG} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, InvalidTopicException} import org.apache.kafka.coordinator.group.GroupConfig +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.metrics.ClientMetricsConfigs import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} @@ -74,7 +75,7 @@ class ControllerConfigurationValidatorTest { def testValidTopicConfig(): Unit = { val config = new util.TreeMap[String, String]() config.put(SEGMENT_JITTER_MS_CONFIG, "1000") - config.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") + config.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") validator.validate(new ConfigResource(TOPIC, "foo"), config, emptyMap()) } @@ -82,7 +83,7 @@ class ControllerConfigurationValidatorTest { def testInvalidTopicConfig(): Unit = { val config = new util.TreeMap[String, String]() config.put(SEGMENT_JITTER_MS_CONFIG, "1000") - config.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") + config.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") config.put("foobar", "abc") assertEquals("Unknown topic config name: foobar", assertThrows(classOf[InvalidConfigurationException], () => validator.validate( diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 98c33d81ead84..ae06e2f90d3e2 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.internals.Plugin import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig} +import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.server.DynamicThreadPool import org.apache.kafka.server.authorizer._ diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 8f858183941c5..ef9287de349e6 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -101,7 +101,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val tp = new TopicPartition("test", 0) val oldSegmentSize = 1000 val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) + logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) createTopic(tp.topic, 1, 1, logProps) TestUtils.retry(10000) { val logOpt = this.brokers.head.logManager.getLog(tp) @@ -113,7 +113,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), + val op = new AlterConfigOp(new ConfigEntry(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), OpType.SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get } finally { diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 8b3035a4ca407..57d9d3bec0651 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -40,7 +40,7 @@ import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig} import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfig, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.MetricConfigs -import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} +import org.apache.kafka.storage.internals.log.CleanerConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.junit.jupiter.api.function.Executable @@ -895,8 +895,6 @@ class KafkaConfigTest { case ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", CompressionType.ZSTD.maxLevel() + 1) - - case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // ignore int //SSL Configs case BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index 3ffb42b788bdf..9a8c05ee5fc85 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -904,7 +904,7 @@ public void testCreateTopicsWithPolicy() { null, Map.of()), new CreateTopicPolicy.RequestMetadata("baz", null, null, Map.of(0, List.of(2, 1, 0)), - Map.of("internal.segment.bytes", "12300000")), + Map.of("segment.bytes", "12300000")), new CreateTopicPolicy.RequestMetadata("quux", null, null, Map.of(0, List.of(2, 1, 0)), Map.of()))); ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder(). @@ -915,7 +915,7 @@ public void testCreateTopicsWithPolicy() { ctx.createTestTopic("foo", 2, (short) 2, NONE.code()); ctx.createTestTopic("bar", 3, (short) 3, POLICY_VIOLATION.code()); ctx.createTestTopic("baz", new int[][] {new int[] {2, 1, 0}}, - Map.of("internal.segment.bytes", "12300000"), NONE.code()); + Map.of("segment.bytes", "12300000"), NONE.code()); ctx.createTestTopic("quux", new int[][] {new int[] {1, 2, 0}}, POLICY_VIOLATION.code()); } diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 2c2d44ef012e5..18e28932f444f 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -153,6 +153,9 @@ public class ServerLogConfigs { public static final String LOG_INITIAL_TASK_DELAY_MS_DOC = "The initial task delay in millisecond when initializing " + "tasks in LogManager. This should be used for testing only."; + public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; + public static final String INTERNAL_SEGMENT_BYTES_DOC = "Setting the maximum size of a single log file. This should be used for testing only."; + public static final String LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG = LOG_PREFIX + "dir.failure.timeout.ms"; public static final Long LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT = 30000L; public static final String LOG_DIR_FAILURE_TIMEOUT_MS_DOC = "If the broker is unable to successfully communicate to the controller that some log " + diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java index fe6194e2424a9..049c9172901a1 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java @@ -44,6 +44,7 @@ import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime; import org.apache.kafka.coordinator.common.runtime.PartitionWriter; import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics; +import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.share.SharePartitionKey; import org.apache.kafka.server.util.FutureUtils; import org.apache.kafka.server.util.MockTime; @@ -1999,7 +2000,7 @@ public void testShareStateTopicConfigs() { List propNames = List.of( TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.COMPRESSION_TYPE_CONFIG, - "internal.segment.bytes", + ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG ); Properties actual = service.shareGroupStateTopicConfigs(); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index d762e570845f6..8bea615d82e99 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -122,9 +122,6 @@ public Optional serverConfigName(String configName) { // Visible for testing public static final String SERVER_DEFAULT_HEADER_NAME = "Server Default Property"; - public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; - public static final String INTERNAL_LOG_SEGMENT_BYTES_DOC = "Setting the maximum size of a single log file. This should be used for testing only."; - public static final int DEFAULT_SEGMENT_BYTES = 1024 * 1024 * 1024; public static final long DEFAULT_SEGMENT_MS = 24 * 7 * 60 * 60 * 1000L; public static final long DEFAULT_SEGMENT_JITTER_MS = 0; @@ -147,7 +144,6 @@ public Optional serverConfigName(String configName) { TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, - INTERNAL_SEGMENT_BYTES_CONFIG, QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG ); @@ -189,7 +185,8 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) - .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC); + .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) + .defineInternal(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.INTERNAL_SEGMENT_BYTES_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { @@ -257,7 +254,7 @@ public Optional serverConfigName(String configName) { TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) - .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_LOG_SEGMENT_BYTES_DOC); + .defineInternal(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.INTERNAL_SEGMENT_BYTES_DOC); } @@ -312,7 +309,7 @@ public LogConfig(Map props, Set overriddenConfigs) { this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); - this.internalSegmentSize = getInt(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG); + this.internalSegmentSize = getInt(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG); this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index d58c520af0972..85b2712cc7886 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.util.MockTime; import org.apache.kafka.test.TestUtils; @@ -197,7 +198,7 @@ public void testUpdateConfig() { assertEquals(oldConfig, log.config()); Properties props = new Properties(); - props.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); + props.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); LogConfig newConfig = new LogConfig(props); log.updateConfig(newConfig); assertEquals(newConfig, log.config()); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index be85c9fd0af86..bb0af9a96c950 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; @@ -114,7 +115,7 @@ public final boolean conditionMet() { .get(); return config.get(TopicConfig.CLEANUP_POLICY_CONFIG).value().equals(TopicConfig.CLEANUP_POLICY_DELETE) && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(PURGE_INTERVAL_MS.toString()) - && config.get("internal.segment.bytes").value().equals(PURGE_SEGMENT_BYTES.toString()); + && config.get(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); } catch (final Exception e) { return false; } @@ -171,7 +172,7 @@ public void setup() { streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); - streamsConfiguration.put(StreamsConfig.topicPrefix("internal.segment.bytes"), PURGE_SEGMENT_BYTES); + streamsConfiguration.put(StreamsConfig.topicPrefix(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size final StreamsBuilder builder = new StreamsBuilder(); diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 5ee3b38564032..1d75581cf2a43 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -40,7 +40,6 @@ import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig; import org.apache.kafka.server.log.remote.storage.LocalTieredStorage; import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig; -import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.test.TestUtils; import java.time.Duration; @@ -98,7 +97,7 @@ private void setUpRemoteLogTopics() { Map rlsConfigs = new HashMap<>(); rlsConfigs.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"); rlsConfigs.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "1"); - rlsConfigs.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "100"); + rlsConfigs.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "100"); setupTopics(this::getRemoteLogStorageEnabledTopicName, rlsConfigs); sendProducerRecords(this::getRemoteLogStorageEnabledTopicName); } From 0f3548006d82d0d48eadb63a8fda06ddd1a283f9 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 28 Apr 2025 19:55:37 +0800 Subject: [PATCH 40/74] revert to trunk --- .../scala/kafka/raft/KafkaMetadataLog.scala | 17 ++++- .../api/PlaintextAdminIntegrationTest.scala | 2 +- .../api/SaslSslAdminIntegrationTest.scala | 3 +- .../DynamicBrokerReconfigurationTest.scala | 2 +- .../kafka/raft/KafkaMetadataLogTest.scala | 19 ++++- .../kafka/cluster/AbstractPartitionTest.scala | 4 +- .../kafka/cluster/PartitionLockTest.scala | 4 +- .../AbstractLogCleanerIntegrationTest.scala | 3 +- .../kafka/log/LogCleanerManagerTest.scala | 19 +++-- .../scala/unit/kafka/log/LogCleanerTest.scala | 73 +++++++++---------- .../unit/kafka/log/LogConcurrencyTest.scala | 4 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 4 +- .../scala/unit/kafka/log/LogManagerTest.scala | 7 +- .../scala/unit/kafka/log/LogTestUtils.scala | 2 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 5 +- .../AlterReplicaLogDirsRequestTest.scala | 4 +- ...ControllerConfigurationValidatorTest.scala | 6 +- .../server/DynamicBrokerConfigTest.scala | 12 ++- .../server/DynamicConfigChangeTest.scala | 8 +- .../unit/kafka/server/KafkaConfigTest.scala | 2 +- .../kafka/tools/DumpLogSegmentsTest.scala | 3 +- .../ReplicationControlManagerTest.java | 5 +- .../apache/kafka/raft/MetadataLogConfig.java | 19 ++++- .../kafka/server/config/ServerLogConfigs.java | 3 - .../share/ShareCoordinatorServiceTest.java | 3 +- .../kafka/storage/internals/log/Cleaner.java | 2 +- .../storage/internals/log/LogConfig.java | 25 ++----- .../storage/internals/log/UnifiedLog.java | 8 +- .../storage/internals/log/LocalLogTest.java | 4 +- .../PurgeRepartitionTopicIntegrationTest.java | 5 +- .../kafka/streams/StreamsConfigTest.java | 5 +- .../tools/ConfigCommandIntegrationTest.java | 32 -------- .../kafka/tools/GetOffsetShellTest.java | 2 +- 33 files changed, 158 insertions(+), 158 deletions(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index ea1259e29d5e6..01f69b374bc3d 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record.{MemoryRecords, Records} import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} -import org.apache.kafka.raft.{Isolation, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} +import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.Scheduler @@ -596,8 +596,12 @@ object KafkaMetadataLog extends Logging { props.setProperty(TopicConfig.RETENTION_BYTES_CONFIG, "-1") LogConfig.validate(props) val defaultLogConfig = new LogConfig(props) - - if (defaultLogConfig.retentionMs >= 0) { + + if (config.logSegmentBytes < config.logSegmentMinBytes) { + throw new InvalidConfigurationException( + s"Cannot set ${MetadataLogConfig.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}." ) @@ -632,6 +636,13 @@ object KafkaMetadataLog extends Logging { config, nodeId ) + + // Print a warning if users have overridden the internal config + if (config.logSegmentMinBytes != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) { + metadataLog.error(s"Overriding ${MetadataLogConfig.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.") + } + // 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() diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 6bc77a193a74d..0ab2328c53f79 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -1568,7 +1568,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @MethodSource(Array("getTestGroupProtocolParametersAll")) def testDeleteRecordsAfterCorruptRecords(groupProtocol: String): Unit = { val config = new Properties() - config.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "200") + config.put(TopicConfig.SEGMENT_BYTES_CONFIG, "200") createTopic(topic, numPartitions = 1, replicationFactor = 1, config) client = createAdminClient diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 33bb2e161f40d..66fca0db9badc 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -34,7 +34,6 @@ 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} @@ -581,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 == ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG).get + val segmentBytesConfig = topicConfigs.find(_.name == TopicConfig.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 diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index c42972473bec9..3c5cd9396bdba 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -653,7 +653,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) => diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 0c478078ce699..1d3b3493cce8e 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -20,7 +20,7 @@ import kafka.server.{KafkaConfig, KafkaRaftServer} import kafka.utils.TestUtils import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.errors.CorruptRecordException -import org.apache.kafka.common.errors.RecordTooLargeException +import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException} import org.apache.kafka.common.protocol import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable} import org.apache.kafka.common.record.ArbitraryMemoryRecords @@ -76,8 +76,15 @@ final class KafkaMetadataLogTest { props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9093") props.put(KRaftConfigs.NODE_ID_CONFIG, Int.box(2)) props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") + props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240)) props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024)) - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(8 * 1024 * 1024)) + assertThrows(classOf[InvalidConfigurationException], () => { + val kafkaConfig = KafkaConfig.fromProps(props) + val metadataConfig = new MetadataLogConfig(kafkaConfig) + buildMetadataLog(tempDir, mockTime, metadataConfig) + }) + + props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) @@ -682,6 +689,7 @@ final class KafkaMetadataLogTest { val recordSize = 64 val config = new MetadataLogConfig( DefaultMetadataLogConfig.logSegmentBytes, + DefaultMetadataLogConfig.logSegmentMinBytes, DefaultMetadataLogConfig.logSegmentMillis, DefaultMetadataLogConfig.retentionMaxBytes, DefaultMetadataLogConfig.retentionMillis, @@ -900,6 +908,7 @@ final class KafkaMetadataLogTest { @Test def testAdvanceLogStartOffsetAfterCleaning(): Unit = { val config = new MetadataLogConfig( + 512, 512, 10 * 1000, 256, @@ -936,6 +945,7 @@ final class KafkaMetadataLogTest { def testDeleteSnapshots(): Unit = { // Generate some logs and a few snapshots, set retention low and verify that cleaning occurs val config = new MetadataLogConfig( + 1024, 1024, 10 * 1000, 1024, @@ -969,6 +979,7 @@ final class KafkaMetadataLogTest { def testSoftRetentionLimit(): Unit = { // Set retention equal to the segment size and generate slightly more than one segment of logs val config = new MetadataLogConfig( + 10240, 10240, 10 * 1000, 10240, @@ -1012,6 +1023,7 @@ final class KafkaMetadataLogTest { @Test def testSegmentsLessThanLatestSnapshot(): Unit = { val config = new MetadataLogConfig( + 10240, 10240, 10 * 1000, 10240, @@ -1070,6 +1082,7 @@ object KafkaMetadataLogTest { } val DefaultMetadataLogConfig = new MetadataLogConfig( + 100 * 1024, 100 * 1024, 10 * 1000, 100 * 1024, @@ -1090,7 +1103,7 @@ object KafkaMetadataLogTest { UnifiedLog.logDirName(KafkaRaftServer.MetadataPartition) ) - val metadataLog = KafkaMetadataLog.apply( + val metadataLog = KafkaMetadataLog( KafkaRaftServer.MetadataPartition, KafkaRaftServer.MetadataTopicId, logDir, diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index 9bf6e2f025c58..d9eaa7b2aacc0 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.requests.LeaderAndIsrRequest import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.{MetadataCache, MockConfigRepository} import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpoints import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} @@ -100,7 +100,7 @@ class AbstractPartitionTest { def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 512: java.lang.Integer) + logProps.put(TopicConfig.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) } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index e4b026e2889ef..4a3051ddc9567 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.{LeaderAndIsr, MetadataCache, MockConfigRepository} import org.apache.kafka.server.common.{RequestLocal, TopicIdPartition} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpoints @@ -362,7 +362,7 @@ class PartitionLockTest extends Logging { private def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 512: java.lang.Integer) + logProps.put(TopicConfig.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) } diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index 493bbccd3f792..acf21e69ec377 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -24,7 +24,6 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion} import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{CleanerConfig, LogCleaner, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, UnifiedLog} import org.apache.kafka.storage.log.metrics.BrokerTopicStats @@ -72,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(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, segmentSize: java.lang.Integer) + props.put(TopicConfig.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) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index c85eab16ee834..f93d703f07777 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -27,7 +27,6 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.LogCleaningState.{LOG_CLEANING_ABORTED, LOG_CLEANING_IN_PROGRESS} import org.apache.kafka.storage.internals.log.{AppendOrigin, LocalLog, LogCleanerManager, LogCleaningException, LogCleaningState, LogConfig, LogDirFailureChannel, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, PreCleanStats, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} @@ -54,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(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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) @@ -371,7 +370,7 @@ class LogCleanerManagerTest extends Logging { // change cleanup policy from delete to compact val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, log.config.segmentSize(): Integer) + logProps.put(TopicConfig.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) @@ -549,7 +548,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsForNone(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -571,7 +570,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsActiveSegment(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -593,7 +592,7 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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)) @@ -626,7 +625,7 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForShortTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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)) @@ -668,7 +667,7 @@ class LogCleanerManagerTest extends Logging { def testUndecidedTransactionalDataNotCleanable(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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)) @@ -712,7 +711,7 @@ class LogCleanerManagerTest extends Logging { @Test def testDoneCleaning(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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) @@ -831,7 +830,7 @@ class LogCleanerManagerTest extends Logging { private def createLowRetentionLogConfig(segmentSize: Int, cleanupPolicy: String): LogConfig = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, segmentSize: Integer) + logProps.put(TopicConfig.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 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 4c2a08d9f4acd..09a9d1c40f140 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -26,7 +26,6 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanedTransactionMetadata, Cleaner, CleanerConfig, CleanerStats, LocalLog, LogAppendInfo, LogCleaner, LogCleanerManager, LogCleaningAbortedException, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetMap, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} @@ -57,7 +56,7 @@ class LogCleanerTest extends Logging { val tmpdir = TestUtils.tempDir() val dir = TestUtils.randomPartitionLogDir(tmpdir) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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 = new LogConfig(logProps) @@ -149,7 +148,7 @@ class LogCleanerTest extends Logging { def testCleanSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -182,7 +181,7 @@ class LogCleanerTest extends Logging { // Construct a log instance. The replaceSegments() method of the log instance is overridden so that // it waits for another thread to execute deleteOldSegments() val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) @@ -272,7 +271,7 @@ class LogCleanerTest extends Logging { val originalMaxFileSize = 1024 val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact": java.lang.String) logProps.put(TopicConfig.PREALLOCATE_CONFIG, "true": java.lang.String) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -296,7 +295,7 @@ class LogCleanerTest extends Logging { def testDuplicateCheckAfterCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) var log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -465,7 +464,7 @@ class LogCleanerTest extends Logging { def testBasicTransactionAwareCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -498,7 +497,7 @@ class LogCleanerTest extends Logging { def testCleanWithTransactionsSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -544,7 +543,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -592,7 +591,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(capacity = Int.MaxValue, maxMessageSize = 100) val logProps = new Properties() logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, 100: java.lang.Integer) - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -620,7 +619,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -683,7 +682,7 @@ class LogCleanerTest extends Logging { def testCleanEmptyControlBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -715,7 +714,7 @@ class LogCleanerTest extends Logging { def testCommittedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -737,7 +736,7 @@ class LogCleanerTest extends Logging { def testAbortedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -766,7 +765,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -800,7 +799,7 @@ class LogCleanerTest extends Logging { val producerId = 1L val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, 0, AppendOrigin.REPLICATION) @@ -833,7 +832,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -897,7 +896,7 @@ class LogCleanerTest extends Logging { // Create cleaner with very small default max message size val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -969,7 +968,7 @@ class LogCleanerTest extends Logging { def createLogWithMessagesLargerThanMaxSize(largeMessageSize: Int): (UnifiedLog, FakeOffsetMap) = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -995,7 +994,7 @@ class LogCleanerTest extends Logging { def testCleaningWithDeletes(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1022,7 +1021,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 3 messages in the map val cleaner = makeCleaner(4) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1051,7 +1050,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsProducerLastSequence(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) log.appendAsLeader(record(0, 0), 0) // offset 0 @@ -1074,7 +1073,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsLastSequenceEvenIfTransactionAborted(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -1108,7 +1107,7 @@ class LogCleanerTest extends Logging { def testCleaningWithKeysConflictingWithTxnMarkerKeys(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val leaderEpoch = 5 val producerEpoch = 0.toShort @@ -1152,7 +1151,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 1 message in the map val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1183,7 +1182,7 @@ class LogCleanerTest extends Logging { def testCleaningWithUncleanableSection(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1226,7 +1225,7 @@ class LogCleanerTest extends Logging { def testLogToClean(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1244,7 +1243,7 @@ class LogCleanerTest extends Logging { def testLogToCleanWithUncleanableSection(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1277,7 +1276,7 @@ class LogCleanerTest extends Logging { // create a log with compaction turned off so we can append unkeyed messages val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1335,7 +1334,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsWithAbort(): Unit = { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1357,7 +1356,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsRetainingLastEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1389,7 +1388,7 @@ class LogCleanerTest extends Logging { def testSegmentGrouping(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1490,7 +1489,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1542,7 +1541,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) //mimic the effect of loading an empty index file logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 400: java.lang.Integer) @@ -1667,7 +1666,7 @@ class LogCleanerTest extends Logging { def testRecoveryAfterCrash(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 10: java.lang.Integer) @@ -1798,7 +1797,7 @@ class LogCleanerTest extends Logging { def testBuildOffsetMapFakeLarge(): Unit = { val map = new FakeOffsetMap(1000) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) @@ -1946,7 +1945,7 @@ class LogCleanerTest extends Logging { @Test def testCleaningBeyondMissingOffsets(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) val cleaner = makeCleaner(Int.MaxValue) diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index a7e162dd76e7f..0da8366f443ee 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -20,10 +20,10 @@ package kafka.log import java.util.{Optional, Properties} import java.util.concurrent.{Callable, Executors} import kafka.utils.TestUtils +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.KafkaScheduler import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, UnifiedLog} @@ -60,7 +60,7 @@ class LogConcurrencyTest { @Test def testUncommittedDataNotConsumedFrequentSegmentRolls(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 237: Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 237: Integer) val logConfig = new LogConfig(logProps) testUncommittedDataNotConsumed(createLog(logConfig)) } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 974fdf00bcd5d..8e417a695ee36 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -21,12 +21,12 @@ import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.compress.Compression +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, SimpleRecord, TimestampType} import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.MockConfigRepository -import org.apache.kafka.server.config.ServerLogConfigs 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, LocalLog, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetMetadata, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile, UnifiedLog} @@ -245,7 +245,7 @@ class LogLoaderTest { @Test def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "640") + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "640") val logConfig = new LogConfig(logProps) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 04aeeff4c45d8..67880e0ced54c 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -27,7 +27,6 @@ import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uui import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.{ConfigRepository, MockConfigRepository} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} -import org.apache.kafka.server.config.ServerLogConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any @@ -44,7 +43,7 @@ import java.util.{Collections, Optional, OptionalLong, Properties} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.{FileLock, KafkaScheduler, MockTime, Scheduler} -import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog, LogManager => JLogManager} +import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogManager => JLogManager, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} import org.apache.kafka.storage.internals.checkpoint.{CleanShutdownFileHandler, OffsetCheckpointFile} import org.apache.kafka.storage.log.metrics.BrokerTopicStats import org.junit.jupiter.api.function.Executable @@ -61,7 +60,7 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 60 * 1000 val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) val logConfig = new LogConfig(logProps) @@ -392,7 +391,7 @@ class LogManagerTest { logManager.shutdown() val segmentBytes = 10 * setSize val properties = new Properties() - properties.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes.toString) + properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes.toString) properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString) val configRepository = MockConfigRepository.forTopic(name, properties) diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index 340b1b26e6e4e..04c91741037bb 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -75,7 +75,7 @@ object LogTestUtils { remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs: java.lang.Long) logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 93f29e01d70d7..3f73f8f731a71 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -52,10 +52,11 @@ import org.junit.jupiter.params.provider.{EnumSource, ValueSource} import org.mockito.ArgumentMatchers import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{doAnswer, doThrow, spy} + import net.jqwik.api.AfterFailureMode import net.jqwik.api.ForAll import net.jqwik.api.Property -import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.KRaftConfigs import java.io._ import java.nio.ByteBuffer @@ -2733,7 +2734,7 @@ class UnifiedLogTest { @Test def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "1000") + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000") logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") val logConfig = new LogConfig(logProps) diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index a0463182daca6..8e2698b0842cf 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AlterReplicaLogDirsRequest, AlterReplicaLogDirsResponse} import org.apache.kafka.server.config.ServerLogConfigs -import org.apache.kafka.storage.internals.log.{LogConfig, LogFileUtils} +import org.apache.kafka.storage.internals.log.LogFileUtils import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -148,7 +148,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { // We don't want files with `.deleted` suffix are removed too fast, // so we can validate there will be orphan files and orphan files will be removed eventually. topicProperties.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, "10000") - topicProperties.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "1024") + topicProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1024") createTopic(topic, partitionNum, 1, topicProperties) assertEquals(logDir1, brokers.head.logManager.getLog(tp).get.dir.getParent) diff --git a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala index ccbe0d043669a..3056753f53bf4 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerConfigurationValidatorTest.scala @@ -23,9 +23,7 @@ import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, BROKER_LOGGER import org.apache.kafka.common.config.TopicConfig.{REMOTE_LOG_STORAGE_ENABLE_CONFIG, SEGMENT_BYTES_CONFIG, SEGMENT_JITTER_MS_CONFIG, SEGMENT_MS_CONFIG} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, InvalidTopicException} import org.apache.kafka.coordinator.group.GroupConfig -import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.metrics.ClientMetricsConfigs -import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest @@ -75,7 +73,7 @@ class ControllerConfigurationValidatorTest { def testValidTopicConfig(): Unit = { val config = new util.TreeMap[String, String]() config.put(SEGMENT_JITTER_MS_CONFIG, "1000") - config.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") + config.put(SEGMENT_BYTES_CONFIG, "67108864") validator.validate(new ConfigResource(TOPIC, "foo"), config, emptyMap()) } @@ -83,7 +81,7 @@ class ControllerConfigurationValidatorTest { def testInvalidTopicConfig(): Unit = { val config = new util.TreeMap[String, String]() config.put(SEGMENT_JITTER_MS_CONFIG, "1000") - config.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "67108864") + config.put(SEGMENT_BYTES_CONFIG, "67108864") config.put("foobar", "abc") assertEquals("Unknown topic config name: foobar", assertThrows(classOf[InvalidConfigurationException], () => validator.validate( diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index ae06e2f90d3e2..17ad2200dcc3c 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.internals.Plugin import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.raft.QuorumConfig +import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.server.DynamicThreadPool import org.apache.kafka.server.authorizer._ @@ -670,6 +670,16 @@ class DynamicBrokerConfigTest { assertTrue(m.currentReporters.isEmpty) } + @Test + def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = { + val props = TestUtils.createBrokerConfig(0, port = 8181) + props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, "1024") + val config = new KafkaConfig(props) + assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) + config.updateCurrentConfig(new KafkaConfig(props)) + assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) + } + @Test def testDynamicLogLocalRetentionMsConfig(): Unit = { val props = TestUtils.createBrokerConfig(0, port = 8181) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index ef9287de349e6..28c8d694f9d51 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -101,19 +101,19 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val tp = new TopicPartition("test", 0) val oldSegmentSize = 1000 val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) createTopic(tp.topic, 1, 1, logProps) TestUtils.retry(10000) { val logOpt = this.brokers.head.logManager.getLog(tp) assertTrue(logOpt.isDefined) - assertEquals(oldSegmentSize, logOpt.get.config.segmentSize()) + assertEquals(oldSegmentSize, logOpt.get.config.segmentSize) } val newSegmentSize = 2000 val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), + val op = new AlterConfigOp(new ConfigEntry(TopicConfig.SEGMENT_BYTES_CONFIG, newSegmentSize.toString), OpType.SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get } finally { @@ -121,7 +121,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } val log = brokers.head.logManager.getLog(tp).get TestUtils.retry(10000) { - assertEquals(newSegmentSize, log.config.segmentSize()) + assertEquals(newSegmentSize, log.config.segmentSize) } (1 to 50).foreach(i => TestUtils.produceMessage(brokers, tp.topic, i.toString)) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 57d9d3bec0651..517741cf2d869 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -895,7 +895,7 @@ class KafkaConfigTest { case ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", CompressionType.ZSTD.maxLevel() + 1) - + //SSL Configs case BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => case BrokerSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG => diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index b86b25f9b1a15..5b3e9abe1112b 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -538,13 +538,14 @@ class DumpLogSegmentsTest { setPartitionId(0).setIsr(util.Arrays.asList(0, 1, 2)), 0.toShort) ) - val metadataLog = KafkaMetadataLog.apply( + val metadataLog = KafkaMetadataLog( KafkaRaftServer.MetadataPartition, KafkaRaftServer.MetadataTopicId, logDir, time, time.scheduler, new MetadataLogConfig( + 100 * 1024, 100 * 1024, 10 * 1000, 100 * 1024, diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index 9a8c05ee5fc85..1b5bf4fb29d24 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -126,6 +126,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.apache.kafka.common.config.TopicConfig.SEGMENT_BYTES_CONFIG; import static org.apache.kafka.common.metadata.MetadataRecordType.CLEAR_ELR_RECORD; import static org.apache.kafka.common.protocol.Errors.ELECTION_NOT_NEEDED; import static org.apache.kafka.common.protocol.Errors.ELIGIBLE_LEADERS_NOT_AVAILABLE; @@ -904,7 +905,7 @@ public void testCreateTopicsWithPolicy() { null, Map.of()), new CreateTopicPolicy.RequestMetadata("baz", null, null, Map.of(0, List.of(2, 1, 0)), - Map.of("segment.bytes", "12300000")), + Map.of(SEGMENT_BYTES_CONFIG, "12300000")), new CreateTopicPolicy.RequestMetadata("quux", null, null, Map.of(0, List.of(2, 1, 0)), Map.of()))); ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder(). @@ -915,7 +916,7 @@ public void testCreateTopicsWithPolicy() { ctx.createTestTopic("foo", 2, (short) 2, NONE.code()); ctx.createTestTopic("bar", 3, (short) 3, POLICY_VIOLATION.code()); ctx.createTestTopic("baz", new int[][] {new int[] {2, 1, 0}}, - Map.of("segment.bytes", "12300000"), NONE.code()); + Map.of(SEGMENT_BYTES_CONFIG, "12300000"), NONE.code()); ctx.createTestTopic("quux", new int[][] {new int[] {1, 2, 0}}, POLICY_VIOLATION.code()); } diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index 3901443af69b2..b0a6f9f045ad6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.record.Records; import org.apache.kafka.server.config.ServerLogConfigs; import java.util.concurrent.TimeUnit; @@ -51,10 +52,14 @@ public class MetadataLogConfig { "configuration. The Kafka node will generate a snapshot when either the maximum time interval is reached or the " + "maximum bytes limit is reached."; + public static final String METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG = "metadata.log.segment.min.bytes"; + public static final String METADATA_LOG_SEGMENT_MIN_BYTES_DOC = "Override the minimum size for a single metadata log file. This should be used for testing only."; + public static final int METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT = 8 * 1024 * 1024; + public static final String METADATA_LOG_SEGMENT_BYTES_CONFIG = "metadata.log.segment.bytes"; public static final String METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file."; public static final int METADATA_LOG_SEGMENT_BYTES_DEFAULT = 1024 * 1024 * 1024; - + public static final String METADATA_LOG_SEGMENT_MILLIS_CONFIG = "metadata.log.segment.ms"; public static final String METADATA_LOG_SEGMENT_MILLIS_DOC = "The maximum time before a new metadata log file is rolled out (in milliseconds)."; public static final long METADATA_LOG_SEGMENT_MILLIS_DEFAULT = 24 * 7 * 60 * 60 * 1000L; @@ -79,13 +84,15 @@ public class MetadataLogConfig { .define(METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG, LONG, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_DOC) .define(METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC) .define(METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, METADATA_LOG_DIR_DOC) - .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(8 * 1024 * 1024), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) + .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) + .defineInternal(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_MIN_BYTES_DOC) .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); private final int logSegmentBytes; + private final int logSegmentMinBytes; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -96,6 +103,7 @@ public class MetadataLogConfig { /** * Configuration for the metadata log * @param logSegmentBytes The maximum size of a single metadata log file + * @param logSegmentMinBytes The minimum size of a single metadata log file * @param logSegmentMillis The maximum time before a new metadata log file is rolled out * @param retentionMaxBytes The size of the metadata log and snapshots before deleting old snapshots and log files * @param retentionMillis The time to keep a metadata log file or snapshot before deleting it @@ -104,6 +112,7 @@ public class MetadataLogConfig { * @param deleteDelayMillis The amount of time to wait before deleting a file from the filesystem */ public MetadataLogConfig(int logSegmentBytes, + int logSegmentMinBytes, long logSegmentMillis, long retentionMaxBytes, long retentionMillis, @@ -111,6 +120,7 @@ public MetadataLogConfig(int logSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = logSegmentBytes; + this.logSegmentMinBytes = logSegmentMinBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; @@ -121,6 +131,7 @@ public MetadataLogConfig(int logSegmentBytes, public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); + this.logSegmentMinBytes = config.getInt(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); @@ -133,6 +144,10 @@ public int logSegmentBytes() { return logSegmentBytes; } + public int logSegmentMinBytes() { + return logSegmentMinBytes; + } + public long logSegmentMillis() { return logSegmentMillis; } diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 18e28932f444f..2c2d44ef012e5 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -153,9 +153,6 @@ public class ServerLogConfigs { public static final String LOG_INITIAL_TASK_DELAY_MS_DOC = "The initial task delay in millisecond when initializing " + "tasks in LogManager. This should be used for testing only."; - public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; - public static final String INTERNAL_SEGMENT_BYTES_DOC = "Setting the maximum size of a single log file. This should be used for testing only."; - public static final String LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG = LOG_PREFIX + "dir.failure.timeout.ms"; public static final Long LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT = 30000L; public static final String LOG_DIR_FAILURE_TIMEOUT_MS_DOC = "If the broker is unable to successfully communicate to the controller that some log " + diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java index 049c9172901a1..a4b316d9443c8 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java @@ -44,7 +44,6 @@ import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime; import org.apache.kafka.coordinator.common.runtime.PartitionWriter; import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics; -import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.share.SharePartitionKey; import org.apache.kafka.server.util.FutureUtils; import org.apache.kafka.server.util.MockTime; @@ -2000,7 +1999,7 @@ public void testShareStateTopicConfigs() { List propNames = List.of( TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.COMPRESSION_TYPE_CONFIG, - ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, + TopicConfig.SEGMENT_BYTES_CONFIG, TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG ); Properties actual = service.shareGroupStateTopicConfigs(); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java index 54ad6a4e79a97..b7d4ccedb2ec1 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java @@ -171,7 +171,7 @@ public Map.Entry doClean(LogToClean cleanable, long currentT List> groupedSegments = groupSegmentsBySize( log.logSegments(0, endOffset), - log.config().segmentSize(), + log.config().segmentSize, log.config().maxIndexSize, cleanable.firstUncleanableOffset() ); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 8bea615d82e99..21c92cd84dff4 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.LegacyRecord; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ConfigUtils; import org.apache.kafka.common.utils.Utils; @@ -121,7 +122,7 @@ public Optional serverConfigName(String configName) { // Visible for testing public static final String SERVER_DEFAULT_HEADER_NAME = "Server Default Property"; - + public static final int DEFAULT_SEGMENT_BYTES = 1024 * 1024 * 1024; public static final long DEFAULT_SEGMENT_MS = 24 * 7 * 60 * 60 * 1000L; public static final long DEFAULT_SEGMENT_JITTER_MS = 0; @@ -185,13 +186,12 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) - .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) - .defineInternal(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.INTERNAL_SEGMENT_BYTES_DOC); + .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { CONFIG. - define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(1024 * 1024), MEDIUM, + define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), MEDIUM, TopicConfig.SEGMENT_BYTES_DOC) .define(TopicConfig.SEGMENT_MS_CONFIG, LONG, DEFAULT_SEGMENT_MS, atLeast(1), MEDIUM, TopicConfig.SEGMENT_MS_DOC) .define(TopicConfig.SEGMENT_JITTER_MS_CONFIG, LONG, DEFAULT_SEGMENT_JITTER_MS, atLeast(0), MEDIUM, @@ -253,9 +253,7 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) - .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) - .defineInternal(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, ServerLogConfigs.INTERNAL_SEGMENT_BYTES_DOC); - + .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC); } public final Set overriddenConfigs; @@ -264,7 +262,7 @@ public Optional serverConfigName(String configName) { * Important note: Any configuration parameter that is passed along from KafkaConfig to LogConfig * should also be in `KafkaConfig#extractLogConfigMap`. */ - private final int segmentSize; + public final int segmentSize; public final long segmentMs; public final long segmentJitterMs; public final int maxIndexSize; @@ -296,7 +294,6 @@ public Optional serverConfigName(String configName) { private final RemoteLogConfig remoteLogConfig; private final int maxMessageSize; private final Map props; - private final Integer internalSegmentSize; public LogConfig(Map props) { this(props, Set.of()); @@ -309,7 +306,6 @@ public LogConfig(Map props, Set overriddenConfigs) { this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); - this.internalSegmentSize = getInt(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG); this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); @@ -392,7 +388,7 @@ public long maxSegmentMs() { public int initFileSize() { if (preallocate) - return segmentSize(); + return segmentSize; else return 0; } @@ -458,11 +454,6 @@ public static Map configKeys() { return Collections.unmodifiableMap(CONFIG.configKeys()); } - public int segmentSize() { - if (internalSegmentSize != null) return internalSegmentSize; - return segmentSize; - } - /** * Check that property names are valid */ @@ -637,7 +628,7 @@ public static void validate(Map existingConfigs, @Override public String toString() { return "LogConfig{" + - "segmentSize=" + segmentSize() + + "segmentSize=" + segmentSize + ", segmentMs=" + segmentMs + ", segmentJitterMs=" + segmentJitterMs + ", maxIndexSize=" + maxIndexSize + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index ff689898ecd62..dcef6929d1902 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1177,9 +1177,9 @@ private LogAppendInfo append(MemoryRecords records, }); // check messages size does not exceed config.segmentSize - if (validRecords.sizeInBytes() > config().segmentSize()) { + if (validRecords.sizeInBytes() > config().segmentSize) { throw new RecordBatchTooLargeException("Message batch size is " + validRecords.sizeInBytes() + " bytes in append " + - "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize() + "."); + "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize + "."); } // maybe roll the log if this segment is full @@ -2034,12 +2034,12 @@ private LogSegment maybeRoll(int messagesSize, LogAppendInfo appendInfo) throws long maxTimestampInMessages = appendInfo.maxTimestamp(); long maxOffsetInMessages = appendInfo.lastOffset(); - if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize(), appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { + if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize, appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { logger.debug("Rolling new log segment (log_size = {}/{}}, " + "offset_index_size = {}/{}, " + "time_index_size = {}/{}, " + "inactive_time_ms = {}/{}).", - segment.size(), config().segmentSize(), + segment.size(), config().segmentSize, segment.offsetIndex().entries(), segment.offsetIndex().maxEntries(), segment.timeIndex().entries(), segment.timeIndex().maxEntries(), segment.timeWaitedForRoll(now, maxTimestampInMessages), config().segmentMs - segment.rollJitterMs()); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index 85b2712cc7886..7dfba8f7a5954 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -19,12 +19,12 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.util.MockTime; import org.apache.kafka.test.TestUtils; @@ -198,7 +198,7 @@ public void testUpdateConfig() { assertEquals(oldConfig, log.config()); Properties props = new Properties(); - props.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); + props.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldConfig.segmentSize + 1); LogConfig newConfig = new LogConfig(props); log.updateConfig(newConfig); assertEquals(newConfig, log.config()); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index bb0af9a96c950..6023706847f15 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -28,7 +28,6 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; @@ -115,7 +114,7 @@ public final boolean conditionMet() { .get(); return config.get(TopicConfig.CLEANUP_POLICY_CONFIG).value().equals(TopicConfig.CLEANUP_POLICY_DELETE) && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(PURGE_INTERVAL_MS.toString()) - && config.get(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); + && config.get(TopicConfig.SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); } catch (final Exception e) { return false; } @@ -172,7 +171,7 @@ public void setup() { streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); - streamsConfiguration.put(StreamsConfig.topicPrefix(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); + streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size final StreamsBuilder builder = new StreamsBuilder(); diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index 5aa556e5acd8c..a7f657ec54b70 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; @@ -247,7 +248,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { props.put(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, 99_999L); props.put(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, 7L); props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "dummy:host"); - props.put(StreamsConfig.topicPrefix("internal.segment.bytes"), 100); + props.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), 100); final StreamsConfig streamsConfig = new StreamsConfig(props); final Map returnedProps = streamsConfig.getMainConsumerConfigs(groupId, clientId, threadIdx); @@ -262,7 +263,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { ); assertEquals(7L, returnedProps.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG)); assertEquals("dummy:host", returnedProps.get(StreamsConfig.APPLICATION_SERVER_CONFIG)); - assertEquals(100, returnedProps.get(StreamsConfig.topicPrefix("internal.segment.bytes"))); + assertEquals(100, returnedProps.get(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG))); } @Test diff --git a/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java index 04b1bffc97fbf..a5c91a632f7d8 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java @@ -471,38 +471,6 @@ public void testUnsupportedVersionException() { } } - @ClusterTest( - serverProperties = {@ClusterConfigProperty(key = "log.segment.bytes", value = "1048577")} - ) - public void testAlterInvalidLogSegmentBytes() throws Exception { - try (Admin client = cluster.admin()) { - cluster.createTopic("test", 1, (short) 1); - TestUtils.waitForCondition( - () -> cluster.brokerSocketServers() - .stream() - .allMatch(broker -> broker.config().getInt("log.segment.bytes") == 1048577), - "Timeout waiting for topic config propagating to broker" - ); - - ConfigCommand.ConfigCommandOptions command = new ConfigCommand.ConfigCommandOptions( - toArray(asList("--bootstrap-server", cluster.bootstrapServers(), - "--alter", - "--topic", "test", - "--add-config", "segment.bytes=1000" - )) - ); - - Throwable exception = - assertThrows(ExecutionException.class, () -> ConfigCommand.alterConfig(client, command)).getCause(); - - assertInstanceOf(InvalidConfigurationException.class, exception); - assertEquals( - "Invalid value 1000 for configuration segment.bytes: Value must be at least 1048576", - exception.getMessage() - ); - } - } - private void assertNonZeroStatusExit(Stream args, Consumer checkErrOut) { AtomicReference exitStatus = new AtomicReference<>(); Exit.setExitProcedure((status, __) -> { diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 1d75581cf2a43..5bb23cabdd98a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -97,7 +97,7 @@ private void setUpRemoteLogTopics() { Map rlsConfigs = new HashMap<>(); rlsConfigs.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"); rlsConfigs.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "1"); - rlsConfigs.put(ServerLogConfigs.INTERNAL_SEGMENT_BYTES_CONFIG, "100"); + rlsConfigs.put(TopicConfig.SEGMENT_BYTES_CONFIG, "100"); setupTopics(this::getRemoteLogStorageEnabledTopicName, rlsConfigs); sendProducerRecords(this::getRemoteLogStorageEnabledTopicName); } From f30c259c65737e2537b66109c1424f8e4a15ca1c Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 28 Apr 2025 20:56:58 +0800 Subject: [PATCH 41/74] using the ServerLogConfigs instead of Topic level --- .../src/main/scala/kafka/log/LogManager.scala | 2 +- .../scala/kafka/raft/KafkaMetadataLog.scala | 19 +++------- .../main/scala/kafka/raft/RaftManager.scala | 3 +- .../kafka/server/DynamicBrokerConfig.scala | 2 +- .../kafka/raft/KafkaMetadataLogTest.scala | 35 +++++++++---------- .../server/DynamicBrokerConfigTest.scala | 12 +------ .../kafka/tools/DumpLogSegmentsTest.scala | 3 +- .../apache/kafka/raft/MetadataLogConfig.java | 17 +-------- .../kafka/server/config/ServerLogConfigs.java | 3 ++ .../server/config/AbstractKafkaConfig.java | 4 +++ .../kafka/storage/internals/log/Cleaner.java | 2 +- .../storage/internals/log/LogConfig.java | 27 ++++++++++---- .../storage/internals/log/UnifiedLog.java | 8 ++--- .../storage/internals/log/LocalLogTest.java | 2 +- 14 files changed, 62 insertions(+), 77 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index c116bc398cfa6..e22a78a08dd1d 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -1528,7 +1528,7 @@ object LogManager { val defaultProps = config.extractLogConfigMap LogConfig.validateBrokerLogConfigValues(defaultProps, config.remoteLogManagerConfig.isRemoteStorageSystemEnabled) - val defaultLogConfig = new LogConfig(defaultProps) + val defaultLogConfig = new LogConfig(defaultProps, config.internalSegmentSize()) val cleanerConfig = new CleanerConfig(config) val transactionLogConfig = new TransactionLogConfig(config) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 01f69b374bc3d..afd2101608f80 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record.{MemoryRecords, Records} import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} -import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} +import org.apache.kafka.raft.{Isolation, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.Scheduler @@ -583,7 +583,8 @@ object KafkaMetadataLog extends Logging { time: Time, scheduler: Scheduler, config: MetadataLogConfig, - nodeId: Int + nodeId: Int, + internalSegmentSize: Integer = null ): KafkaMetadataLog = { val props = new Properties() props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString) @@ -595,13 +596,9 @@ object KafkaMetadataLog extends Logging { props.setProperty(TopicConfig.RETENTION_MS_CONFIG, "-1") props.setProperty(TopicConfig.RETENTION_BYTES_CONFIG, "-1") LogConfig.validate(props) - val defaultLogConfig = new LogConfig(props) + val defaultLogConfig = new LogConfig(props, internalSegmentSize) - if (config.logSegmentBytes < config.logSegmentMinBytes) { - throw new InvalidConfigurationException( - s"Cannot set ${MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG} below ${config.logSegmentMinBytes}: ${config.logSegmentBytes}" - ) - } else if (defaultLogConfig.retentionMs >= 0) { + if (defaultLogConfig.retentionMs >= 0) { throw new InvalidConfigurationException( s"Cannot set ${TopicConfig.RETENTION_MS_CONFIG} above -1: ${defaultLogConfig.retentionMs}." ) @@ -637,12 +634,6 @@ object KafkaMetadataLog extends Logging { nodeId ) - // Print a warning if users have overridden the internal config - if (config.logSegmentMinBytes != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) { - metadataLog.error(s"Overriding ${MetadataLogConfig.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.") - } - // 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() diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index 0727c660fe406..266fe1317ce20 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -230,7 +230,8 @@ class KafkaRaftManager[T]( time, scheduler, config = new MetadataLogConfig(config), - config.nodeId + config.nodeId, + config.internalSegmentSize() ) } diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 27b5c8e16d417..87fb551309134 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -705,7 +705,7 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { val newBrokerDefaults = new util.HashMap[String, Object](newConfig.extractLogConfigMap) - logManager.reconfigureDefaultLogConfig(new LogConfig(newBrokerDefaults)) + logManager.reconfigureDefaultLogConfig(new LogConfig(newBrokerDefaults, newConfig.internalSegmentSize())) updateLogsConfig(newBrokerDefaults.asScala) } diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 1d3b3493cce8e..35cef2cfed40b 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -20,7 +20,7 @@ import kafka.server.{KafkaConfig, KafkaRaftServer} import kafka.utils.TestUtils import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.errors.CorruptRecordException -import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException} +import org.apache.kafka.common.errors.RecordTooLargeException import org.apache.kafka.common.protocol import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable} import org.apache.kafka.common.record.ArbitraryMemoryRecords @@ -40,10 +40,10 @@ 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.ArgumentsSource - import net.jqwik.api.AfterFailureMode import net.jqwik.api.ForAll import net.jqwik.api.Property +import org.apache.kafka.common.config.ConfigException import java.io.File import java.nio.ByteBuffer @@ -78,13 +78,13 @@ final class KafkaMetadataLogTest { props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240)) props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024)) - assertThrows(classOf[InvalidConfigurationException], () => { + assertThrows(classOf[ConfigException], () => { val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) }) - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) + props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10 * 1024 * 1024)) val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) @@ -689,7 +689,6 @@ final class KafkaMetadataLogTest { val recordSize = 64 val config = new MetadataLogConfig( DefaultMetadataLogConfig.logSegmentBytes, - DefaultMetadataLogConfig.logSegmentMinBytes, DefaultMetadataLogConfig.logSegmentMillis, DefaultMetadataLogConfig.retentionMaxBytes, DefaultMetadataLogConfig.retentionMillis, @@ -908,8 +907,7 @@ final class KafkaMetadataLogTest { @Test def testAdvanceLogStartOffsetAfterCleaning(): Unit = { val config = new MetadataLogConfig( - 512, - 512, + 1024 * 1024, 10 * 1000, 256, 60 * 1000, @@ -945,8 +943,7 @@ final class KafkaMetadataLogTest { def testDeleteSnapshots(): Unit = { // Generate some logs and a few snapshots, set retention low and verify that cleaning occurs val config = new MetadataLogConfig( - 1024, - 1024, + 1024 * 1024, 10 * 1000, 1024, 60 * 1000, @@ -979,8 +976,7 @@ final class KafkaMetadataLogTest { def testSoftRetentionLimit(): Unit = { // Set retention equal to the segment size and generate slightly more than one segment of logs val config = new MetadataLogConfig( - 10240, - 10240, + 1024 * 1024, 10 * 1000, 10240, 60 * 1000, @@ -1023,8 +1019,7 @@ final class KafkaMetadataLogTest { @Test def testSegmentsLessThanLatestSnapshot(): Unit = { val config = new MetadataLogConfig( - 10240, - 10240, + 1024 * 1024, 10 * 1000, 10240, 60 * 1000, @@ -1032,7 +1027,7 @@ final class KafkaMetadataLogTest { DefaultMetadataLogConfig.maxFetchSizeInBytes, DefaultMetadataLogConfig.deleteDelayMillis ) - val log = buildMetadataLog(tempDir, mockTime, config) + val log = buildMetadataLog(tempDir, mockTime, config, 10240) // Generate enough data to cause a segment roll for (_ <- 0 to 2000) { @@ -1082,8 +1077,7 @@ object KafkaMetadataLogTest { } val DefaultMetadataLogConfig = new MetadataLogConfig( - 100 * 1024, - 100 * 1024, + 1024 * 1024, 10 * 1000, 100 * 1024, 60 * 1000, @@ -1095,7 +1089,8 @@ object KafkaMetadataLogTest { def buildMetadataLogAndDir( tempDir: File, time: MockTime, - metadataLogConfig: MetadataLogConfig = DefaultMetadataLogConfig + metadataLogConfig: MetadataLogConfig = DefaultMetadataLogConfig, + internalSegmentSize: Integer = null, ): (Path, KafkaMetadataLog, MetadataLogConfig) = { val logDir = createLogDirectory( @@ -1110,7 +1105,8 @@ object KafkaMetadataLogTest { time, time.scheduler, metadataLogConfig, - 1 + 1, + internalSegmentSize ) (logDir.toPath, metadataLog, metadataLogConfig) @@ -1120,8 +1116,9 @@ object KafkaMetadataLogTest { tempDir: File, time: MockTime, metadataLogConfig: MetadataLogConfig = DefaultMetadataLogConfig, + internalSegmentSize: Integer = null, ): KafkaMetadataLog = { - val (_, log, _) = buildMetadataLogAndDir(tempDir, time, metadataLogConfig) + val (_, log, _) = buildMetadataLogAndDir(tempDir, time, metadataLogConfig, internalSegmentSize) log } diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 17ad2200dcc3c..ae06e2f90d3e2 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.internals.Plugin import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig} +import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.server.DynamicThreadPool import org.apache.kafka.server.authorizer._ @@ -670,16 +670,6 @@ class DynamicBrokerConfigTest { assertTrue(m.currentReporters.isEmpty) } - @Test - def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = { - val props = TestUtils.createBrokerConfig(0, port = 8181) - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, "1024") - val config = new KafkaConfig(props) - assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) - config.updateCurrentConfig(new KafkaConfig(props)) - assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) - } - @Test def testDynamicLogLocalRetentionMsConfig(): Unit = { val props = TestUtils.createBrokerConfig(0, port = 8181) diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 5b3e9abe1112b..d9de803cc76f4 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -545,8 +545,7 @@ class DumpLogSegmentsTest { time, time.scheduler, new MetadataLogConfig( - 100 * 1024, - 100 * 1024, + 1024 * 1024, 10 * 1000, 100 * 1024, 60 * 1000, diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index b0a6f9f045ad6..cd276f4c799e4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.record.Records; import org.apache.kafka.server.config.ServerLogConfigs; import java.util.concurrent.TimeUnit; @@ -52,10 +51,6 @@ public class MetadataLogConfig { "configuration. The Kafka node will generate a snapshot when either the maximum time interval is reached or the " + "maximum bytes limit is reached."; - public static final String METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG = "metadata.log.segment.min.bytes"; - public static final String METADATA_LOG_SEGMENT_MIN_BYTES_DOC = "Override the minimum size for a single metadata log file. This should be used for testing only."; - public static final int METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT = 8 * 1024 * 1024; - public static final String METADATA_LOG_SEGMENT_BYTES_CONFIG = "metadata.log.segment.bytes"; public static final String METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file."; public static final int METADATA_LOG_SEGMENT_BYTES_DEFAULT = 1024 * 1024 * 1024; @@ -84,15 +79,13 @@ public class MetadataLogConfig { .define(METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG, LONG, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_DOC) .define(METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC) .define(METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, METADATA_LOG_DIR_DOC) - .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) - .defineInternal(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_MIN_BYTES_DOC) + .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(8 * 1024 * 1024), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); private final int logSegmentBytes; - private final int logSegmentMinBytes; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -103,7 +96,6 @@ public class MetadataLogConfig { /** * Configuration for the metadata log * @param logSegmentBytes The maximum size of a single metadata log file - * @param logSegmentMinBytes The minimum size of a single metadata log file * @param logSegmentMillis The maximum time before a new metadata log file is rolled out * @param retentionMaxBytes The size of the metadata log and snapshots before deleting old snapshots and log files * @param retentionMillis The time to keep a metadata log file or snapshot before deleting it @@ -112,7 +104,6 @@ public class MetadataLogConfig { * @param deleteDelayMillis The amount of time to wait before deleting a file from the filesystem */ public MetadataLogConfig(int logSegmentBytes, - int logSegmentMinBytes, long logSegmentMillis, long retentionMaxBytes, long retentionMillis, @@ -120,7 +111,6 @@ public MetadataLogConfig(int logSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = logSegmentBytes; - this.logSegmentMinBytes = logSegmentMinBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; @@ -131,7 +121,6 @@ public MetadataLogConfig(int logSegmentBytes, public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); - this.logSegmentMinBytes = config.getInt(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); @@ -144,10 +133,6 @@ public int logSegmentBytes() { return logSegmentBytes; } - public int logSegmentMinBytes() { - return logSegmentMinBytes; - } - public long logSegmentMillis() { return logSegmentMillis; } diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 2c2d44ef012e5..f735370ce8d9a 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -43,6 +43,9 @@ public class ServerLogConfigs { public static final String LOG_SEGMENT_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); public static final String LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file"; + public static final String INTERNAL_LOG_SEGMENT_BYTES_CONFIG = "internal." + ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); + public static final String INTERNAL_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; + public static final String LOG_ROLL_TIME_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_MS_CONFIG); public static final String LOG_ROLL_TIME_HOURS_CONFIG = LOG_PREFIX + "roll.hours"; public static final String LOG_ROLL_TIME_MILLIS_DOC = "The maximum time before a new log segment is rolled out (in milliseconds). If not set, the value in " + LOG_ROLL_TIME_HOURS_CONFIG + " is used"; diff --git a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java index ef61fee154283..7a612db1478c5 100644 --- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java @@ -82,6 +82,10 @@ public int numRecoveryThreadsPerDataDir() { return getInt(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG); } + public Integer internalSegmentSize() { + return getInt(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG); + } + public int backgroundThreads() { return getInt(ServerConfigs.BACKGROUND_THREADS_CONFIG); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java index b7d4ccedb2ec1..54ad6a4e79a97 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java @@ -171,7 +171,7 @@ public Map.Entry doClean(LogToClean cleanable, long currentT List> groupedSegments = groupSegmentsBySize( log.logSegments(0, endOffset), - log.config().segmentSize, + log.config().segmentSize(), log.config().maxIndexSize, cleanable.firstUncleanableOffset() ); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 21c92cd84dff4..946ac7490e6bc 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.record.LegacyRecord; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ConfigUtils; import org.apache.kafka.common.utils.Utils; @@ -186,12 +185,13 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) - .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC); + .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) + .defineInternal(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { CONFIG. - define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), MEDIUM, + define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(1024 * 1024), MEDIUM, TopicConfig.SEGMENT_BYTES_DOC) .define(TopicConfig.SEGMENT_MS_CONFIG, LONG, DEFAULT_SEGMENT_MS, atLeast(1), MEDIUM, TopicConfig.SEGMENT_MS_DOC) .define(TopicConfig.SEGMENT_JITTER_MS_CONFIG, LONG, DEFAULT_SEGMENT_JITTER_MS, atLeast(0), MEDIUM, @@ -262,7 +262,8 @@ public Optional serverConfigName(String configName) { * Important note: Any configuration parameter that is passed along from KafkaConfig to LogConfig * should also be in `KafkaConfig#extractLogConfigMap`. */ - public final int segmentSize; + private final int segmentSize; + private final Integer internalSegmentSize; public final long segmentMs; public final long segmentJitterMs; public final int maxIndexSize; @@ -296,16 +297,25 @@ public Optional serverConfigName(String configName) { private final Map props; public LogConfig(Map props) { - this(props, Set.of()); + this(props, Set.of(), null); } - @SuppressWarnings({"this-escape"}) public LogConfig(Map props, Set overriddenConfigs) { + this(props, overriddenConfigs, null); + } + + public LogConfig(Map props, Integer internalSegmentSize) { + this(props, Set.of(), internalSegmentSize); + } + + @SuppressWarnings({"this-escape"}) + public LogConfig(Map props, Set overriddenConfigs, Integer internalSegmentSize) { super(CONFIG, props, false); this.props = Collections.unmodifiableMap(props); this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); + this.internalSegmentSize = internalSegmentSize; this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); @@ -367,6 +377,11 @@ private Optional getCompression() { } } + public int segmentSize() { + if (internalSegmentSize != null) return internalSegmentSize; + return segmentSize; + } + // Exposed as a method so it can be mocked public int maxMessageSize() { return maxMessageSize; diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index dcef6929d1902..ff689898ecd62 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1177,9 +1177,9 @@ private LogAppendInfo append(MemoryRecords records, }); // check messages size does not exceed config.segmentSize - if (validRecords.sizeInBytes() > config().segmentSize) { + if (validRecords.sizeInBytes() > config().segmentSize()) { throw new RecordBatchTooLargeException("Message batch size is " + validRecords.sizeInBytes() + " bytes in append " + - "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize + "."); + "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize() + "."); } // maybe roll the log if this segment is full @@ -2034,12 +2034,12 @@ private LogSegment maybeRoll(int messagesSize, LogAppendInfo appendInfo) throws long maxTimestampInMessages = appendInfo.maxTimestamp(); long maxOffsetInMessages = appendInfo.lastOffset(); - if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize, appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { + if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize(), appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { logger.debug("Rolling new log segment (log_size = {}/{}}, " + "offset_index_size = {}/{}, " + "time_index_size = {}/{}, " + "inactive_time_ms = {}/{}).", - segment.size(), config().segmentSize, + segment.size(), config().segmentSize(), segment.offsetIndex().entries(), segment.offsetIndex().maxEntries(), segment.timeIndex().entries(), segment.timeIndex().maxEntries(), segment.timeWaitedForRoll(now, maxTimestampInMessages), config().segmentMs - segment.rollJitterMs()); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index 7dfba8f7a5954..86ff278012e87 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -198,7 +198,7 @@ public void testUpdateConfig() { assertEquals(oldConfig, log.config()); Properties props = new Properties(); - props.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldConfig.segmentSize + 1); + props.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); LogConfig newConfig = new LogConfig(props); log.updateConfig(newConfig); assertEquals(newConfig, log.config()); From 197561caf4438b112908568e1f3b578c732a2ced Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 28 Apr 2025 22:19:16 +0800 Subject: [PATCH 42/74] fix test --- .../clients/admin/StaticBrokerConfigTest.java | 6 +- .../api/PlaintextAdminIntegrationTest.scala | 2 +- .../api/SaslSslAdminIntegrationTest.scala | 4 +- .../kafka/cluster/AbstractPartitionTest.scala | 6 +- .../kafka/cluster/PartitionLockTest.scala | 4 +- .../AbstractLogCleanerIntegrationTest.scala | 3 +- .../kafka/log/LogCleanerManagerTest.scala | 19 ++--- .../scala/unit/kafka/log/LogCleanerTest.scala | 73 ++++++++++--------- .../unit/kafka/log/LogConcurrencyTest.scala | 3 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 3 +- .../scala/unit/kafka/log/LogManagerTest.scala | 7 +- .../scala/unit/kafka/log/LogTestUtils.scala | 2 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 5 +- .../AlterReplicaLogDirsRequestTest.scala | 2 +- .../server/DynamicConfigChangeTest.scala | 4 +- .../group/GroupCoordinatorServiceTest.java | 3 +- .../storage/internals/log/LogConfig.java | 5 +- .../storage/internals/log/LocalLogTest.java | 3 +- .../PurgeRepartitionTopicIntegrationTest.java | 5 +- .../kafka/tools/GetOffsetShellTest.java | 2 +- 20 files changed, 86 insertions(+), 75 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java index 58240c0711e62..b6822eb95b6c0 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java @@ -18,12 +18,12 @@ import org.apache.kafka.common.config.ConfigResource; -import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; +import org.apache.kafka.server.config.ServerLogConfigs; import java.util.List; import java.util.Map; @@ -53,7 +53,7 @@ public void testTopicConfigsGetImpactedIfStaticConfigsAddToController(ClusterIns Admin adminUsingBootstrapController = cluster.admin(Map.of(), true) ) { ConfigEntry configEntry = admin.createTopics(List.of(new NewTopic(TOPIC, 1, (short) 1))) - .config(TOPIC).get().get(TopicConfig.SEGMENT_BYTES_CONFIG); + .config(TOPIC).get().get(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG); assertEquals(ConfigEntry.ConfigSource.STATIC_BROKER_CONFIG, configEntry.source()); assertEquals(CUSTOM_VALUE, configEntry.value(), "Config value should be custom value since controller has related static config"); @@ -86,7 +86,7 @@ public void testTopicConfigsGetImpactedIfStaticConfigsAddToBroker(ClusterInstanc Admin adminUsingBootstrapController = cluster.admin(Map.of(), true) ) { ConfigEntry configEntry = admin.createTopics(List.of(new NewTopic(TOPIC, 1, (short) 1))) - .config(TOPIC).get().get(TopicConfig.SEGMENT_BYTES_CONFIG); + .config(TOPIC).get().get(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG); assertEquals(ConfigEntry.ConfigSource.DEFAULT_CONFIG, configEntry.source()); assertNotEquals(CUSTOM_VALUE, configEntry.value(), diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 0ab2328c53f79..27865e29b0989 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -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(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "200") createTopic(topic, numPartitions = 1, replicationFactor = 1, config) client = createAdminClient diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 66fca0db9badc..9facc4f41a3b7 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -567,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(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG -> "100000").asJava 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)) @@ -580,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 == ServerLogConfigs.INTERNAL_LOG_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 diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index d9eaa7b2aacc0..65d50434e0efa 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.requests.LeaderAndIsrRequest import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.{MetadataCache, MockConfigRepository} import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpoints import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} @@ -69,7 +69,7 @@ class AbstractPartitionTest { TestUtils.clearYammerMetrics() val logProps = createLogProperties(Map.empty) - logConfig = new LogConfig(logProps) + logConfig = new LogConfig(logProps, 1024) configRepository = MockConfigRepository.forTopic(topicPartition.topic, logProps) tmpDir = TestUtils.tempDir() @@ -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(ServerLogConfigs.INTERNAL_LOG_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) } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 4a3051ddc9567..390d457ab4583 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.{LeaderAndIsr, MetadataCache, MockConfigRepository} import org.apache.kafka.server.common.{RequestLocal, TopicIdPartition} -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpoints @@ -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(ServerLogConfigs.INTERNAL_LOG_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) } diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index acf21e69ec377..9c00a348f14ec 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion} import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.transaction.TransactionLogConfig +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{CleanerConfig, LogCleaner, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, UnifiedLog} import org.apache.kafka.storage.log.metrics.BrokerTopicStats @@ -71,7 +72,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(ServerLogConfigs.INTERNAL_LOG_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) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index f93d703f07777..01ad7f34abf1d 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -27,6 +27,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.coordinator.transaction.TransactionLogConfig +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.LogCleaningState.{LOG_CLEANING_ABORTED, LOG_CLEANING_IN_PROGRESS} import org.apache.kafka.storage.internals.log.{AppendOrigin, LocalLog, LogCleanerManager, LogCleaningException, LogCleaningState, LogConfig, LogDirFailureChannel, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, PreCleanStats, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} @@ -53,7 +54,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(ServerLogConfigs.INTERNAL_LOG_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) @@ -370,7 +371,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(ServerLogConfigs.INTERNAL_LOG_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) @@ -548,7 +549,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(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -570,7 +571,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(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -592,7 +593,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(ServerLogConfigs.INTERNAL_LOG_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)) @@ -625,7 +626,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(ServerLogConfigs.INTERNAL_LOG_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)) @@ -667,7 +668,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(ServerLogConfigs.INTERNAL_LOG_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)) @@ -711,7 +712,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(ServerLogConfigs.INTERNAL_LOG_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) @@ -830,7 +831,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(ServerLogConfigs.INTERNAL_LOG_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 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 09a9d1c40f140..6c3688f497794 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.errors.CorruptRecordException import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanedTransactionMetadata, Cleaner, CleanerConfig, CleanerStats, LocalLog, LogAppendInfo, LogCleaner, LogCleanerManager, LogCleaningAbortedException, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetMap, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} @@ -56,7 +57,7 @@ class LogCleanerTest extends Logging { val tmpdir = TestUtils.tempDir() val dir = TestUtils.randomPartitionLogDir(tmpdir) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_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 = new LogConfig(logProps) @@ -148,7 +149,7 @@ class LogCleanerTest extends Logging { def testCleanSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -181,7 +182,7 @@ class LogCleanerTest extends Logging { // Construct a log instance. The replaceSegments() method of the log instance is overridden so that // it waits for another thread to execute deleteOldSegments() val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) @@ -271,7 +272,7 @@ class LogCleanerTest extends Logging { val originalMaxFileSize = 1024 val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact": java.lang.String) logProps.put(TopicConfig.PREALLOCATE_CONFIG, "true": java.lang.String) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -295,7 +296,7 @@ class LogCleanerTest extends Logging { def testDuplicateCheckAfterCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) var log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -464,7 +465,7 @@ class LogCleanerTest extends Logging { def testBasicTransactionAwareCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -497,7 +498,7 @@ class LogCleanerTest extends Logging { def testCleanWithTransactionsSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -543,7 +544,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -591,7 +592,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(capacity = Int.MaxValue, maxMessageSize = 100) val logProps = new Properties() logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, 100: java.lang.Integer) - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -619,7 +620,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -682,7 +683,7 @@ class LogCleanerTest extends Logging { def testCleanEmptyControlBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -714,7 +715,7 @@ class LogCleanerTest extends Logging { def testCommittedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -736,7 +737,7 @@ class LogCleanerTest extends Logging { def testAbortedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -765,7 +766,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -799,7 +800,7 @@ class LogCleanerTest extends Logging { val producerId = 1L val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, 0, AppendOrigin.REPLICATION) @@ -832,7 +833,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -896,7 +897,7 @@ class LogCleanerTest extends Logging { // Create cleaner with very small default max message size val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -968,7 +969,7 @@ class LogCleanerTest extends Logging { def createLogWithMessagesLargerThanMaxSize(largeMessageSize: Int): (UnifiedLog, FakeOffsetMap) = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -994,7 +995,7 @@ class LogCleanerTest extends Logging { def testCleaningWithDeletes(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1021,7 +1022,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 3 messages in the map val cleaner = makeCleaner(4) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1050,7 +1051,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsProducerLastSequence(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) log.appendAsLeader(record(0, 0), 0) // offset 0 @@ -1073,7 +1074,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsLastSequenceEvenIfTransactionAborted(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -1107,7 +1108,7 @@ class LogCleanerTest extends Logging { def testCleaningWithKeysConflictingWithTxnMarkerKeys(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val leaderEpoch = 5 val producerEpoch = 0.toShort @@ -1151,7 +1152,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 1 message in the map val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1182,7 +1183,7 @@ class LogCleanerTest extends Logging { def testCleaningWithUncleanableSection(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1225,7 +1226,7 @@ class LogCleanerTest extends Logging { def testLogToClean(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1243,7 +1244,7 @@ class LogCleanerTest extends Logging { def testLogToCleanWithUncleanableSection(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1276,7 +1277,7 @@ class LogCleanerTest extends Logging { // create a log with compaction turned off so we can append unkeyed messages val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1334,7 +1335,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsWithAbort(): Unit = { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1356,7 +1357,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsRetainingLastEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1388,7 +1389,7 @@ class LogCleanerTest extends Logging { def testSegmentGrouping(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1489,7 +1490,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1541,7 +1542,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) //mimic the effect of loading an empty index file logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 400: java.lang.Integer) @@ -1666,7 +1667,7 @@ class LogCleanerTest extends Logging { def testRecoveryAfterCrash(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 10: java.lang.Integer) @@ -1797,7 +1798,7 @@ class LogCleanerTest extends Logging { def testBuildOffsetMapFakeLarge(): Unit = { val map = new FakeOffsetMap(1000) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) @@ -1945,7 +1946,7 @@ class LogCleanerTest extends Logging { @Test def testCleaningBeyondMissingOffsets(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) val cleaner = makeCleaner(Int.MaxValue) diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 0da8366f443ee..a6276c91f2243 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.KafkaScheduler import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, UnifiedLog} @@ -60,7 +61,7 @@ class LogConcurrencyTest { @Test def testUncommittedDataNotConsumedFrequentSegmentRolls(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 237: Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 237: Integer) val logConfig = new LogConfig(logProps) testUncommittedDataNotConsumed(createLog(logConfig)) } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 8e417a695ee36..ee1e98a22c398 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, Me import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.MockConfigRepository +import org.apache.kafka.server.config.ServerLogConfigs 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, LocalLog, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetMetadata, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile, UnifiedLog} @@ -245,7 +246,7 @@ class LogLoaderTest { @Test def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "640") + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "640") val logConfig = new LogConfig(logProps) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 67880e0ced54c..d97e6f523fead 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uui import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.{ConfigRepository, MockConfigRepository} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} +import org.apache.kafka.server.config.ServerLogConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any @@ -43,7 +44,7 @@ import java.util.{Collections, Optional, OptionalLong, Properties} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.{FileLock, KafkaScheduler, MockTime, Scheduler} -import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogManager => JLogManager, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} +import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog, LogManager => JLogManager} import org.apache.kafka.storage.internals.checkpoint.{CleanShutdownFileHandler, OffsetCheckpointFile} import org.apache.kafka.storage.log.metrics.BrokerTopicStats import org.junit.jupiter.api.function.Executable @@ -60,7 +61,7 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 60 * 1000 val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) val logConfig = new LogConfig(logProps) @@ -391,7 +392,7 @@ class LogManagerTest { logManager.shutdown() val segmentBytes = 10 * setSize val properties = new Properties() - properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes.toString) + properties.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, segmentBytes.toString) properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString) val configRepository = MockConfigRepository.forTopic(name, properties) diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index 04c91741037bb..dea67024a6107 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -75,7 +75,7 @@ object LogTestUtils { remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes: Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs: java.lang.Long) logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 3f73f8f731a71..015d2232fcffd 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -52,11 +52,10 @@ import org.junit.jupiter.params.provider.{EnumSource, ValueSource} import org.mockito.ArgumentMatchers import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{doAnswer, doThrow, spy} - import net.jqwik.api.AfterFailureMode import net.jqwik.api.ForAll import net.jqwik.api.Property -import org.apache.kafka.server.config.KRaftConfigs +import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} import java.io._ import java.nio.ByteBuffer @@ -2734,7 +2733,7 @@ class UnifiedLogTest { @Test def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000") + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1000") logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") val logConfig = new LogConfig(logProps) diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index 8e2698b0842cf..a3de06a393bbd 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -148,7 +148,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { // We don't want files with `.deleted` suffix are removed too fast, // so we can validate there will be orphan files and orphan files will be removed eventually. topicProperties.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, "10000") - topicProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1024") + topicProperties.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1024") createTopic(topic, partitionNum, 1, topicProperties) assertEquals(logDir1, brokers.head.logManager.getLog(tp).get.dir.getParent) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 28c8d694f9d51..ea8bc400decea 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -101,7 +101,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val tp = new TopicPartition("test", 0) val oldSegmentSize = 1000 val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) createTopic(tp.topic, 1, 1, logProps) TestUtils.retry(10000) { val logOpt = this.brokers.head.logManager.getLog(tp) @@ -113,7 +113,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(TopicConfig.SEGMENT_BYTES_CONFIG, newSegmentSize.toString), + val op = new AlterConfigOp(new ConfigEntry(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), OpType.SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get } finally { diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 0fd4d1270e282..0deba0b98c79e 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -92,6 +92,7 @@ import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.TopicsImage; import org.apache.kafka.server.authorizer.AuthorizableRequestContext; +import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.record.BrokerCompressionType; import org.apache.kafka.server.share.persister.DefaultStatePersister; import org.apache.kafka.server.share.persister.DeleteShareGroupStateParameters; @@ -900,7 +901,7 @@ public void testGroupMetadataTopicConfigs() { Properties expectedProperties = new Properties(); expectedProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); expectedProperties.put(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.PRODUCER.name); - expectedProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000"); + expectedProperties.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1000"); assertEquals(expectedProperties, service.groupMetadataTopicConfigs()); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 946ac7490e6bc..867ec6adb9e56 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -314,7 +314,10 @@ public LogConfig(Map props, Set overriddenConfigs, Integer interna this.props = Collections.unmodifiableMap(props); this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); - this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); + if (internalSegmentSize == null) + this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); + else + this.segmentSize = internalSegmentSize; this.internalSegmentSize = internalSegmentSize; this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index 86ff278012e87..1dfc247ac1351 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.util.MockTime; import org.apache.kafka.test.TestUtils; @@ -198,7 +199,7 @@ public void testUpdateConfig() { assertEquals(oldConfig, log.config()); Properties props = new Properties(); - props.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); + props.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); LogConfig newConfig = new LogConfig(props); log.updateConfig(newConfig); assertEquals(newConfig, log.config()); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index 6023706847f15..25506f03e3937 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; @@ -114,7 +115,7 @@ public final boolean conditionMet() { .get(); return config.get(TopicConfig.CLEANUP_POLICY_CONFIG).value().equals(TopicConfig.CLEANUP_POLICY_DELETE) && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(PURGE_INTERVAL_MS.toString()) - && config.get(TopicConfig.SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); + && config.get(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); } catch (final Exception e) { return false; } @@ -171,7 +172,7 @@ public void setup() { streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); - streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); + streamsConfiguration.put(StreamsConfig.topicPrefix(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size final StreamsBuilder builder = new StreamsBuilder(); diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 5bb23cabdd98a..1e96f1d361fea 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -97,7 +97,7 @@ private void setUpRemoteLogTopics() { Map rlsConfigs = new HashMap<>(); rlsConfigs.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"); rlsConfigs.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "1"); - rlsConfigs.put(TopicConfig.SEGMENT_BYTES_CONFIG, "100"); + rlsConfigs.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "100"); setupTopics(this::getRemoteLogStorageEnabledTopicName, rlsConfigs); sendProducerRecords(this::getRemoteLogStorageEnabledTopicName); } From 9d5a0bd17505f2dcc47e856dd69e5f447055b5ff Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 28 Apr 2025 22:27:10 +0800 Subject: [PATCH 43/74] fix test --- .../apache/kafka/clients/admin/StaticBrokerConfigTest.java | 6 +++--- .../org/apache/kafka/storage/internals/log/LogConfig.java | 5 +---- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java index b6822eb95b6c0..58240c0711e62 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/StaticBrokerConfigTest.java @@ -18,12 +18,12 @@ import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; -import org.apache.kafka.server.config.ServerLogConfigs; import java.util.List; import java.util.Map; @@ -53,7 +53,7 @@ public void testTopicConfigsGetImpactedIfStaticConfigsAddToController(ClusterIns Admin adminUsingBootstrapController = cluster.admin(Map.of(), true) ) { ConfigEntry configEntry = admin.createTopics(List.of(new NewTopic(TOPIC, 1, (short) 1))) - .config(TOPIC).get().get(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG); + .config(TOPIC).get().get(TopicConfig.SEGMENT_BYTES_CONFIG); assertEquals(ConfigEntry.ConfigSource.STATIC_BROKER_CONFIG, configEntry.source()); assertEquals(CUSTOM_VALUE, configEntry.value(), "Config value should be custom value since controller has related static config"); @@ -86,7 +86,7 @@ public void testTopicConfigsGetImpactedIfStaticConfigsAddToBroker(ClusterInstanc Admin adminUsingBootstrapController = cluster.admin(Map.of(), true) ) { ConfigEntry configEntry = admin.createTopics(List.of(new NewTopic(TOPIC, 1, (short) 1))) - .config(TOPIC).get().get(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG); + .config(TOPIC).get().get(TopicConfig.SEGMENT_BYTES_CONFIG); assertEquals(ConfigEntry.ConfigSource.DEFAULT_CONFIG, configEntry.source()); assertNotEquals(CUSTOM_VALUE, configEntry.value(), diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 867ec6adb9e56..946ac7490e6bc 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -314,10 +314,7 @@ public LogConfig(Map props, Set overriddenConfigs, Integer interna this.props = Collections.unmodifiableMap(props); this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); - if (internalSegmentSize == null) - this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); - else - this.segmentSize = internalSegmentSize; + this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); this.internalSegmentSize = internalSegmentSize; this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); From 2597bac689db1c4eb352204951ecd9cd5e61cfdf Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 28 Apr 2025 22:36:13 +0800 Subject: [PATCH 44/74] fix test --- .../kafka/coordinator/group/GroupCoordinatorServiceTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 0deba0b98c79e..cb3bd90203e0c 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -92,7 +92,6 @@ import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.TopicsImage; import org.apache.kafka.server.authorizer.AuthorizableRequestContext; -import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.record.BrokerCompressionType; import org.apache.kafka.server.share.persister.DefaultStatePersister; import org.apache.kafka.server.share.persister.DeleteShareGroupStateParameters; @@ -111,6 +110,7 @@ import org.apache.kafka.server.util.FutureUtils; import org.apache.kafka.server.util.timer.MockTimer; +import org.apache.kafka.storage.internals.log.LogConfig; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -901,7 +901,7 @@ public void testGroupMetadataTopicConfigs() { Properties expectedProperties = new Properties(); expectedProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); expectedProperties.put(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.PRODUCER.name); - expectedProperties.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1000"); + expectedProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000"); assertEquals(expectedProperties, service.groupMetadataTopicConfigs()); } From c70f9914f002d2386fad1232c7a27d009d4ea9b9 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 28 Apr 2025 23:28:12 +0800 Subject: [PATCH 45/74] temp test --- .../integration/PurgeRepartitionTopicIntegrationTest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index 25506f03e3937..93075ab17b13d 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -76,6 +76,7 @@ public class PurgeRepartitionTopicIntegrationTest { put("log.retention.check.interval.ms", PURGE_INTERVAL_MS); put("log.initial.task.delay.ms", INITIAL_TASK_DELAY_MS); put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 0); + put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, PURGE_SEGMENT_BYTES); } }); @@ -114,8 +115,7 @@ public final boolean conditionMet() { .get(resource) .get(); return config.get(TopicConfig.CLEANUP_POLICY_CONFIG).value().equals(TopicConfig.CLEANUP_POLICY_DELETE) - && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(PURGE_INTERVAL_MS.toString()) - && config.get(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); + && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(PURGE_INTERVAL_MS.toString()); } catch (final Exception e) { return false; } @@ -172,7 +172,6 @@ public void setup() { streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); - streamsConfiguration.put(StreamsConfig.topicPrefix(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size final StreamsBuilder builder = new StreamsBuilder(); From 6e612998a223d12f7a6e7c07cc7e9bbaeb0f4e1f Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 29 Apr 2025 17:22:01 +0800 Subject: [PATCH 46/74] fix PurgeRepartitionTopicIntegrationTest test --- .../PurgeRepartitionTopicIntegrationTest.java | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index 93075ab17b13d..1c47061565ff2 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -24,11 +24,10 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.config.TopicConfig; -import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; @@ -50,6 +49,7 @@ import java.io.IOException; import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -68,7 +68,7 @@ public class PurgeRepartitionTopicIntegrationTest { private static Admin adminClient; private static KafkaStreams kafkaStreams; private static final Integer PURGE_INTERVAL_MS = 10; - private static final Integer PURGE_SEGMENT_BYTES = 2000; + private static final Integer PURGE_SEGMENT_BYTES = 1024 * 1024; private static final Integer INITIAL_TASK_DELAY_MS = 0; public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS, new Properties() { @@ -76,7 +76,6 @@ public class PurgeRepartitionTopicIntegrationTest { put("log.retention.check.interval.ms", PURGE_INTERVAL_MS); put("log.initial.task.delay.ms", INITIAL_TASK_DELAY_MS); put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 0); - put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, PURGE_SEGMENT_BYTES); } }); @@ -115,7 +114,8 @@ public final boolean conditionMet() { .get(resource) .get(); return config.get(TopicConfig.CLEANUP_POLICY_CONFIG).value().equals(TopicConfig.CLEANUP_POLICY_DELETE) - && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(PURGE_INTERVAL_MS.toString()); + && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(PURGE_INTERVAL_MS.toString()) + && config.get(TopicConfig.SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); } catch (final Exception e) { return false; } @@ -168,10 +168,11 @@ public void setup() { streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, PURGE_INTERVAL_MS); streamsConfiguration.put(StreamsConfig.REPARTITION_PURGE_INTERVAL_MS_CONFIG, PURGE_INTERVAL_MS); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); - streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); + streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.ByteArray().getClass()); + streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.ByteArray().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); + streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size final StreamsBuilder builder = new StreamsBuilder(); @@ -195,15 +196,17 @@ public void shutdown() { @Test public void shouldRestoreState() throws Exception { // produce some data to input topic - final List> messages = new ArrayList<>(); + final List> messages = new ArrayList<>(); for (int i = 0; i < 1000; i++) { - messages.add(new KeyValue<>(i, i)); + final byte[] data = new byte[1024]; + Arrays.fill(data, (byte) i); + messages.add(new KeyValue<>(data, data)); } IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(INPUT_TOPIC, messages, TestUtils.producerConfig(CLUSTER.bootstrapServers(), - IntegerSerializer.class, - IntegerSerializer.class), + ByteArraySerializer.class, + ByteArraySerializer.class), time.milliseconds()); kafkaStreams.start(); From c69973d4d48ead85a35eb37918310cc47e541ad1 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 29 Apr 2025 17:27:54 +0800 Subject: [PATCH 47/74] spotless --- .../kafka/coordinator/group/GroupCoordinatorServiceTest.java | 1 - .../org/apache/kafka/storage/internals/log/LocalLogTest.java | 1 - 2 files changed, 2 deletions(-) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index cb3bd90203e0c..0fd4d1270e282 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -110,7 +110,6 @@ import org.apache.kafka.server.util.FutureUtils; import org.apache.kafka.server.util.timer.MockTimer; -import org.apache.kafka.storage.internals.log.LogConfig; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index 1dfc247ac1351..8bbf525df0ba8 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; -import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; From a4af39c60283745acfd46e24fd188dd9cf1bd8ee Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 29 Apr 2025 17:48:40 +0800 Subject: [PATCH 48/74] spotless --- core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala | 1 - core/src/test/scala/unit/kafka/log/LogLoaderTest.scala | 1 - 2 files changed, 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index a6276c91f2243..16a02a274dd2e 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -20,7 +20,6 @@ package kafka.log import java.util.{Optional, Properties} import java.util.concurrent.{Callable, Executors} import kafka.utils.TestUtils -import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index ee1e98a22c398..faf9bb7d22c58 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -21,7 +21,6 @@ import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.compress.Compression -import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, SimpleRecord, TimestampType} import org.apache.kafka.common.utils.{Time, Utils} From 569b59939d9575fab1a44d2f3a254f882bf3a0fd Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 29 Apr 2025 21:05:53 +0800 Subject: [PATCH 49/74] fix test --- .../src/main/scala/kafka/log/LogManager.scala | 2 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 4 +- .../scala/unit/kafka/log/LogManagerTest.scala | 23 +++++++--- .../scala/unit/kafka/log/LogTestUtils.scala | 5 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 46 +++++++++---------- .../server/DynamicConfigChangeTest.scala | 8 ++-- .../unit/kafka/server/KafkaConfigTest.scala | 1 + .../storage/internals/log/LogConfig.java | 18 +++++--- 8 files changed, 61 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index e22a78a08dd1d..d8a2501d407b4 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -582,7 +582,7 @@ class LogManager(logDirs: Seq[File], val overrides = configRepository.topicConfig(topicName) // save memory by only including configs for topics with overrides if (!overrides.isEmpty) { - val logConfig = LogConfig.fromProps(defaultProps, overrides) + val logConfig = LogConfig.fromProps(defaultProps, overrides, defaultConfig.segmentSize()) topicConfigOverrides(topicName) = logConfig } } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index faf9bb7d22c58..673d651848c6f 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -26,7 +26,6 @@ import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, Me import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.MockConfigRepository -import org.apache.kafka.server.config.ServerLogConfigs 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, LocalLog, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetMetadata, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile, UnifiedLog} @@ -245,8 +244,7 @@ class LogLoaderTest { @Test def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "640") - val logConfig = new LogConfig(logProps) + val logConfig = new LogConfig(logProps, 640) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index d97e6f523fead..9be056bc37896 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -27,7 +27,6 @@ import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uui import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.{ConfigRepository, MockConfigRepository} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} -import org.apache.kafka.server.config.ServerLogConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any @@ -61,10 +60,9 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 60 * 1000 val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) - val logConfig = new LogConfig(logProps) + val logConfig = new LogConfig(logProps, 1024) var logDir: File = _ var logManager: LogManager = _ val name = "kafka" @@ -392,11 +390,17 @@ class LogManagerTest { logManager.shutdown() val segmentBytes = 10 * setSize val properties = new Properties() - properties.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, segmentBytes.toString) properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString) val configRepository = MockConfigRepository.forTopic(name, properties) - logManager = createLogManager(configRepository = configRepository) + logManager = TestUtils.createLogManager( + defaultConfig = new LogConfig(logProps, segmentBytes), + configRepository = configRepository, + logDirs = Seq(this.logDir), + time = this.time, + recoveryThreadsPerDataDir = 1, + initialTaskDelayMs = initialTaskDelayMs + ) logManager.startup(Set.empty) // create a log @@ -450,7 +454,14 @@ class LogManagerTest { logManager.shutdown() val configRepository = MockConfigRepository.forTopic(name, TopicConfig.CLEANUP_POLICY_CONFIG, policy) - logManager = createLogManager(configRepository = configRepository) + logManager = TestUtils.createLogManager( + defaultConfig = new LogConfig(logProps, 1024), + configRepository = configRepository, + logDirs = Seq(this.logDir), + time = this.time, + recoveryThreadsPerDataDir = 1, + initialTaskDelayMs = initialTaskDelayMs + ) val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = Optional.empty) var offset = 0L for (_ <- 0 until 200) { diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index dea67024a6107..77d5d8bd17cb3 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -72,7 +72,8 @@ object LogTestUtils { fileDeleteDelayMs: Long = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, remoteLogStorageEnable: Boolean = LogConfig.DEFAULT_REMOTE_STORAGE_ENABLE, remoteLogCopyDisable: Boolean = DEFAULT_REMOTE_LOG_COPY_DISABLE_CONFIG, - remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { + remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, + internalSegmentBytes : Integer = null): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) @@ -89,7 +90,7 @@ object LogTestUtils { logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, remoteLogStorageEnable: java.lang.Boolean) logProps.put(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, remoteLogCopyDisable: java.lang.Boolean) logProps.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, remoteLogDeleteOnDisable: java.lang.Boolean) - new LogConfig(logProps) + new LogConfig(logProps, internalSegmentBytes) } def createLog(dir: File, diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 015d2232fcffd..7ee2b8f887b31 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -682,7 +682,7 @@ class UnifiedLogTest { val msgPerSeg = 10 val segmentSize = msgPerSeg * setSize // each segment will be 10 messages // create a log - val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentSize) val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments, "There should be exactly 1 segment.") @@ -709,7 +709,7 @@ class UnifiedLogTest { def testInitializationOfProducerSnapshotsUpgradePath(): Unit = { // simulate the upgrade path by creating a new log with several segments, deleting the // snapshot files, and then reloading the log - val logConfig = LogTestUtils.createLogConfig(segmentBytes = 64 * 10) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = 64 * 10) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) @@ -1123,7 +1123,7 @@ class UnifiedLogTest { @Test def testProducerIdMapTruncateFullyAndStartAt(): Unit = { val records = TestUtils.singletonRecords("foo".getBytes) - val logConfig = LogTestUtils.createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) val log = createLog(logDir, logConfig) log.appendAsLeader(records, 0) log.takeProducerSnapshot() @@ -1146,7 +1146,7 @@ class UnifiedLogTest { def testProducerIdExpirationOnSegmentDeletion(): Unit = { val pid1 = 1L val records = TestUtils.records(Seq(new SimpleRecord("foo".getBytes)), producerId = pid1, producerEpoch = 0, sequence = 0) - val logConfig = LogTestUtils.createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) val log = createLog(logDir, logConfig) log.appendAsLeader(records, 0) log.takeProducerSnapshot() @@ -1199,7 +1199,7 @@ class UnifiedLogTest { @Test def testProducerSnapshotAfterSegmentRollOnAppend(): Unit = { val producerId = 1L - val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(Seq(new SimpleRecord(mockTime.milliseconds(), new Array[Byte](512))), @@ -1588,7 +1588,7 @@ class UnifiedLogTest { val msgPerSeg = 10 val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages // create a log - val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentSize) val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments, "There should be exactly 1 segment.") @@ -1889,7 +1889,7 @@ class UnifiedLogTest { val messageSet = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes)) // append messages to log val configSegmentSize = messageSet.sizeInBytes - 1 - val logConfig = LogTestUtils.createLogConfig(segmentBytes = configSegmentSize) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = configSegmentSize) val log = createLog(logDir, logConfig) assertThrows(classOf[RecordBatchTooLargeException], () => log.appendAsLeader(messageSet, 0)) @@ -2478,7 +2478,7 @@ class UnifiedLogTest { val setSize = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds).sizeInBytes val msgPerSeg = 10 val segmentSize = msgPerSeg * setSize // each segment will be 10 messages - val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = setSize - 1) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentSize, indexIntervalBytes = setSize - 1) val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments, "There should be exactly 1 segment.") @@ -2893,7 +2893,7 @@ class UnifiedLogTest { @Test def testLogDeletionAfterDeleteRecords(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5) val log = createLog(logDir, logConfig) for (_ <- 0 until 15) @@ -2923,7 +2923,7 @@ class UnifiedLogTest { @Test def shouldDeleteSizeBasedSegments(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2938,7 +2938,7 @@ class UnifiedLogTest { @Test def shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 15) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 15) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2968,7 +2968,7 @@ class UnifiedLogTest { @Test def shouldNotDeleteTimeBasedSegmentsWhenNoneReadyToBeDeleted(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, timestamp = mockTime.milliseconds) - val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000000) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000000) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -3018,7 +3018,7 @@ class UnifiedLogTest { def shouldDeleteStartOffsetBreachedSegmentsWhenPolicyDoesNotIncludeDelete(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes, timestamp = 10L) val recordsPerSegment = 5 - val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * recordsPerSegment, retentionMs = 10000, cleanupPolicy = "compact") + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * recordsPerSegment, retentionMs = 10000, cleanupPolicy = "compact") val log = createLog(logDir, logConfig, brokerTopicStats) // append some messages to create some segments @@ -3090,7 +3090,7 @@ class UnifiedLogTest { @Test def shouldTruncateLeaderEpochsWhenDeletingSegments(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) val cache = epochCache(log) @@ -3115,7 +3115,7 @@ class UnifiedLogTest { @Test def shouldUpdateOffsetForLeaderEpochsWhenDeletingSegments(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) val cache = epochCache(log) @@ -3144,7 +3144,7 @@ class UnifiedLogTest { baseOffset = startOffset, partitionLeaderEpoch = epoch) } - val logConfig = LogTestUtils.createLogConfig(segmentBytes = 10 * createRecords(0, 0).sizeInBytes) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = 10 * createRecords(0, 0).sizeInBytes) val log = createLog(logDir, logConfig) val cache = epochCache(log) @@ -3668,7 +3668,7 @@ class UnifiedLogTest { new SimpleRecord("b".getBytes), new SimpleRecord("c".getBytes)) - val logConfig = LogTestUtils.createLogConfig(segmentBytes = records.sizeInBytes) + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = records.sizeInBytes) val log = createLog(logDir, logConfig) val firstAppendInfo = log.appendAsLeader(records, 0) @@ -3709,7 +3709,7 @@ class UnifiedLogTest { @Test def testSegmentDeletionWithHighWatermarkInitialization(): Unit = { val logConfig = LogTestUtils.createLogConfig( - segmentBytes = 512, + internalSegmentBytes = 512, segmentIndexBytes = 1000, retentionMs = 999 ) @@ -3733,7 +3733,7 @@ class UnifiedLogTest { @Test def testCannotDeleteSegmentsAtOrAboveHighWatermark(): Unit = { val logConfig = LogTestUtils.createLogConfig( - segmentBytes = 512, + internalSegmentBytes = 512, segmentIndexBytes = 1000, retentionMs = 999 ) @@ -4404,7 +4404,7 @@ class UnifiedLogTest { def testRetentionOnLocalLogDeletionWhenRemoteLogCopyEnabledAndDefaultLocalRetentionBytes(): Unit = { def createRecords = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes))) val segmentBytes = createRecords.sizeInBytes() - val retentionBytesConfig = LogTestUtils.createLogConfig(segmentBytes = segmentBytes, retentionBytes = 1, + val retentionBytesConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentBytes, retentionBytes = 1, fileDeleteDelayMs = 0, remoteLogStorageEnable = true) val log = createLog(logDir, retentionBytesConfig, remoteStorageSystemEnable = true) @@ -4427,7 +4427,7 @@ class UnifiedLogTest { def testRetentionOnLocalLogDeletionWhenRemoteLogCopyEnabledAndDefaultLocalRetentionMs(): Unit = { def createRecords = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes))) val segmentBytes = createRecords.sizeInBytes() - val retentionBytesConfig = LogTestUtils.createLogConfig(segmentBytes = segmentBytes, retentionMs = 1000, + val retentionBytesConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentBytes, retentionMs = 1000, fileDeleteDelayMs = 0, remoteLogStorageEnable = true) val log = createLog(logDir, retentionBytesConfig, remoteStorageSystemEnable = true) @@ -4452,7 +4452,7 @@ class UnifiedLogTest { def testRetentionOnLocalLogDeletionWhenRemoteLogCopyDisabled(): Unit = { def createRecords = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes))) val segmentBytes = createRecords.sizeInBytes() - val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentBytes, localRetentionBytes = 1, retentionBytes = segmentBytes * 5, + val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentBytes, localRetentionBytes = 1, retentionBytes = segmentBytes * 5, fileDeleteDelayMs = 0, remoteLogStorageEnable = true) val log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) @@ -4479,7 +4479,7 @@ class UnifiedLogTest { assertEquals(2, log.localLogStartOffset()) // add remoteCopyDisabled = true - val copyDisabledLogConfig = LogTestUtils.createLogConfig(segmentBytes = segmentBytes, localRetentionBytes = 1, retentionBytes = segmentBytes * 5, + val copyDisabledLogConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentBytes, localRetentionBytes = 1, retentionBytes = segmentBytes * 5, fileDeleteDelayMs = 0, remoteLogStorageEnable = true, remoteLogCopyDisable = true) log.updateConfig(copyDisabledLogConfig) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index ea8bc400decea..16a4d3c4358f4 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -99,9 +99,9 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { @ValueSource(strings = Array("kraft")) def testDynamicTopicConfigChange(quorum: String): Unit = { val tp = new TopicPartition("test", 0) - val oldSegmentSize = 1000 + val oldSegmentSize = 2 * 1024 * 1024 val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) createTopic(tp.topic, 1, 1, logProps) TestUtils.retry(10000) { val logOpt = this.brokers.head.logManager.getLog(tp) @@ -109,11 +109,11 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { assertEquals(oldSegmentSize, logOpt.get.config.segmentSize) } - val newSegmentSize = 2000 + val newSegmentSize = 4 * 1024 * 1024 val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), + val op = new AlterConfigOp(new ConfigEntry(TopicConfig.SEGMENT_BYTES_CONFIG, newSegmentSize.toString), OpType.SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get } finally { diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 517741cf2d869..a29bcba13f8fb 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -815,6 +815,7 @@ class KafkaConfigTest { case ServerLogConfigs.LOG_DIRS_CONFIG => // ignore string case ServerLogConfigs.LOG_DIR_CONFIG => // ignore string case ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", Records.LOG_OVERHEAD - 1) + case ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG => // no op case ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 946ac7490e6bc..70b55e0f96b6d 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -403,7 +403,7 @@ public long maxSegmentMs() { public int initFileSize() { if (preallocate) - return segmentSize; + return segmentSize(); else return 0; } @@ -437,15 +437,19 @@ public String overriddenConfigsAsLoggableString() { return ConfigUtils.configMapToRedactedString(overriddenTopicProps, CONFIG); } - /** - * Create a log config instance using the given properties and defaults - */ - public static LogConfig fromProps(Map defaults, Properties overrides) { + public static LogConfig fromProps(Map defaults, Properties overrides, Integer internalLogSegmentSize) { Properties props = new Properties(); props.putAll(defaults); props.putAll(overrides); Set overriddenKeys = overrides.keySet().stream().map(k -> (String) k).collect(Collectors.toSet()); - return new LogConfig(props, overriddenKeys); + return new LogConfig(props, overriddenKeys, internalLogSegmentSize); + } + + /** + * Create a log config instance using the given properties and defaults + */ + public static LogConfig fromProps(Map defaults, Properties overrides) { + return fromProps(defaults, overrides, null); } // Visible for testing, return a copy since it's a mutable global variable @@ -643,7 +647,7 @@ public static void validate(Map existingConfigs, @Override public String toString() { return "LogConfig{" + - "segmentSize=" + segmentSize + + "segmentSize=" + segmentSize() + ", segmentMs=" + segmentMs + ", segmentJitterMs=" + segmentJitterMs + ", maxIndexSize=" + maxIndexSize + From 6417e0e00602f0ae29c42312edec2058a6305e79 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 29 Apr 2025 22:12:54 +0800 Subject: [PATCH 50/74] fix some error --- .../kafka/api/SaslSslAdminIntegrationTest.scala | 4 ++-- .../log/AbstractLogCleanerIntegrationTest.scala | 8 ++------ .../kafka/log/LogCleanerIntegrationTest.scala | 2 +- .../kafka/log/LogCleanerLagIntegrationTest.scala | 2 +- .../LogCleanerParameterizedIntegrationTest.scala | 6 +++--- .../scala/unit/kafka/log/LogCleanerTest.scala | 15 +++++---------- 6 files changed, 14 insertions(+), 23 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 9facc4f41a3b7..79ce331b87a19 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -567,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(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG -> "100000").asJava + val configsOverride = Map(TopicConfig.SEGMENT_BYTES_CONFIG -> "20000000").asJava 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)) @@ -580,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 == ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG).get + val segmentBytesConfig = topicConfigs.find(_.name == TopicConfig.SEGMENT_BYTES_DOC).get assertEquals(100000, segmentBytesConfig.value.toLong) assertEquals(ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, segmentBytesConfig.source) val compressionConfig = topicConfigs.find(_.name == TopicConfig.COMPRESSION_TYPE_CONFIG).get diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index 9c00a348f14ec..e6820b713a269 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -24,7 +24,6 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, RecordVersion} import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{CleanerConfig, LogCleaner, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, UnifiedLog} import org.apache.kafka.storage.log.metrics.BrokerTopicStats @@ -68,11 +67,9 @@ abstract class AbstractLogCleanerIntegrationTest { minCleanableDirtyRatio: Float = defaultMinCleanableDirtyRatio, minCompactionLagMs: Long = defaultMinCompactionLagMS, deleteDelay: Int = defaultDeleteDelay, - segmentSize: Int = defaultSegmentSize, maxCompactionLagMs: Long = defaultMaxCompactionLagMs): Properties = { val props = new Properties() props.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, maxMessageSize: java.lang.Integer) - props.put(ServerLogConfigs.INTERNAL_LOG_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) @@ -90,7 +87,7 @@ abstract class AbstractLogCleanerIntegrationTest { maxMessageSize: Int = defaultMaxMessageSize, minCompactionLagMs: Long = defaultMinCompactionLagMS, deleteDelay: Int = defaultDeleteDelay, - segmentSize: Int = defaultSegmentSize, + internalSegmentSize: Int = defaultSegmentSize, maxCompactionLagMs: Long = defaultMaxCompactionLagMs, cleanerIoBufferSize: Option[Int] = None, propertyOverrides: Properties = new Properties()): LogCleaner = { @@ -105,8 +102,7 @@ abstract class AbstractLogCleanerIntegrationTest { minCleanableDirtyRatio = minCleanableDirtyRatio, minCompactionLagMs = minCompactionLagMs, deleteDelay = deleteDelay, - segmentSize = segmentSize, - maxCompactionLagMs = maxCompactionLagMs)) + maxCompactionLagMs = maxCompactionLagMs), internalSegmentSize) val log = UnifiedLog.create( dir, logConfig, diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 38aa789729a9a..d519c4d11a3f6 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -137,7 +137,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest { cleaner = makeCleaner(partitions = topicPartitions, backoffMs = cleanerBackOffMs, minCompactionLagMs = minCompactionLagMs, - segmentSize = segmentSize, + internalSegmentSize = segmentSize, maxCompactionLagMs= maxCompactionLagMs, minCleanableDirtyRatio = minCleanableDirtyRatio) val log = cleaner.logs.get(topicPartitions(0)) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala index c632f2c0bf1dd..d6ed10d8e6bac 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala @@ -52,7 +52,7 @@ class LogCleanerLagIntegrationTest extends AbstractLogCleanerIntegrationTest wit cleaner = makeCleaner(partitions = topicPartitions, backoffMs = cleanerBackOffMs, minCompactionLagMs = minCompactionLag, - segmentSize = segmentSize) + internalSegmentSize = segmentSize) val log = cleaner.logs.get(topicPartitions(0)) // t = T0 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala index e0d3ac5601d23..5c8a48b50f391 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala @@ -156,7 +156,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati val log = cleaner.logs.get(topicPartitions(0)) val props = logConfigProperties(maxMessageSize = maxMessageSize) props.put(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.name) - val logConfig = new LogConfig(props) + val logConfig = new LogConfig(props, 2048) log.updateConfig(logConfig) val appends1 = writeDups(numKeys = 100, numDups = 3, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V0) @@ -198,10 +198,10 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati def testCleaningNestedMessagesWithV0V1(compressionType: CompressionType): Unit = { val compression = Compression.of(compressionType).build() val maxMessageSize = 192 - cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = maxMessageSize, segmentSize = 256) + cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = maxMessageSize, internalSegmentSize = 256) val log = cleaner.logs.get(topicPartitions(0)) - val logConfig = new LogConfig(logConfigProperties(maxMessageSize = maxMessageSize, segmentSize = 256)) + val logConfig = new LogConfig(logConfigProperties(maxMessageSize = maxMessageSize), 256) log.updateConfig(logConfig) // with compression enabled, these messages will be written as a single message containing all the individual messages diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 6c3688f497794..c36e6ee10c126 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -57,10 +57,9 @@ class LogCleanerTest extends Logging { val tmpdir = TestUtils.tempDir() val dir = TestUtils.randomPartitionLogDir(tmpdir) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_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 = new LogConfig(logProps) + val logConfig = new LogConfig(logProps, 1024) val time = new MockTime() val throttler = new Throttler(Double.MaxValue, Long.MaxValue, "throttler", "entries", time) val tombstoneRetentionMs = 86400000 @@ -149,9 +148,8 @@ class LogCleanerTest extends Logging { def testCleanSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) - val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps, 1024)) // append messages to the log until we have four segments while (log.numberOfSegments < 4) @@ -995,9 +993,8 @@ class LogCleanerTest extends Logging { def testCleaningWithDeletes(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) - val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps, 1024)) // append messages with the keys 0 through N while (log.numberOfSegments < 2) @@ -1244,8 +1241,7 @@ class LogCleanerTest extends Logging { def testLogToCleanWithUncleanableSection(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) - val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps, 100)) // create 6 segments with only one message in each segment def createRecords = TestUtils.singletonRecords(value = Array.fill[Byte](25)(0), key = 1.toString.getBytes) @@ -1667,11 +1663,10 @@ class LogCleanerTest extends Logging { def testRecoveryAfterCrash(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 10: java.lang.Integer) - val config = LogConfig.fromProps(logConfig.originals, logProps) + val config = LogConfig.fromProps(logConfig.originals, logProps, 300) // create a log and append some messages var log = makeLog(config = config) From 2ababd62100a988b6ff0ec68f39d920731af526f Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 29 Apr 2025 22:17:52 +0800 Subject: [PATCH 51/74] fix some error --- .../integration/kafka/api/SaslSslAdminIntegrationTest.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 79ce331b87a19..c4f2508d0ca24 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -567,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 -> "20000000").asJava + val configsOverride = Map(TopicConfig.SEGMENT_BYTES_CONFIG -> "1048576").asJava 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)) @@ -580,8 +580,8 @@ 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_DOC).get - assertEquals(100000, segmentBytesConfig.value.toLong) + val segmentBytesConfig = topicConfigs.find(_.name == TopicConfig.SEGMENT_BYTES_CONFIG).get + assertEquals(1048576, segmentBytesConfig.value.toLong) assertEquals(ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, segmentBytesConfig.source) val compressionConfig = topicConfigs.find(_.name == TopicConfig.COMPRESSION_TYPE_CONFIG).get assertEquals(ServerLogConfigs.COMPRESSION_TYPE_DEFAULT, compressionConfig.value) From 09b7b78fc3e12c8626784c3b2d764ca5907c0af1 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 29 Apr 2025 23:08:24 +0800 Subject: [PATCH 52/74] fix some error --- core/src/main/scala/kafka/log/LogManager.scala | 2 +- .../api/PlaintextAdminIntegrationTest.scala | 8 ++++++-- .../AlterReplicaLogDirsRequestTest.scala | 18 ++++++++++++++++-- 3 files changed, 23 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index d8a2501d407b4..80b17fb44143f 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -948,7 +948,7 @@ class LogManager(logDirs: Seq[File], wasRemoteLogEnabled: Boolean): Unit = { topicConfigUpdated(topic) val logs = logsByTopic(topic) - val newLogConfig = LogConfig.fromProps(currentDefaultConfig.originals, newTopicConfig) + val newLogConfig = LogConfig.fromProps(currentDefaultConfig.originals, newTopicConfig, currentDefaultConfig.segmentSize()) val isRemoteLogStorageEnabled = newLogConfig.remoteStorageEnable() // We would like to validate the configuration no matter whether the logs have materialised on disk or not. // Otherwise we risk someone creating a tiered-topic, disabling Tiered Storage cluster-wide and the check diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 27865e29b0989..ea0d7494a83b1 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -87,6 +87,10 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { + if (testInfo.getTestMethod.get.getName == "testDeleteRecordsAfterCorruptRecords") { + this.serverConfig.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1100") + this.controllerConfig.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1100") + } super.setUp(testInfo) Configurator.reconfigure(); brokerLoggerConfigResource = new ConfigResource( @@ -1568,7 +1572,6 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @MethodSource(Array("getTestGroupProtocolParametersAll")) def testDeleteRecordsAfterCorruptRecords(groupProtocol: String): Unit = { val config = new Properties() - config.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "200") createTopic(topic, numPartitions = 1, replicationFactor = 1, config) client = createAdminClient @@ -1576,7 +1579,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val producer = createProducer() def sendRecords(begin: Int, end: Int) = { val futures = (begin until end).map( i => { - val record = new ProducerRecord(topic, partition, s"$i".getBytes, s"$i".getBytes) + val data : Array[Byte] = Array.fill(25)(i.toByte) + val record = new ProducerRecord(topic, partition, data, data) producer.send(record) }) futures.foreach(_.get) diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index a3de06a393bbd..3e5461da4934e 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.requests.{AlterReplicaLogDirsRequest, AlterReplic import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.storage.internals.log.LogFileUtils import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -38,10 +39,24 @@ import scala.util.Random class AlterReplicaLogDirsRequestTest extends BaseRequestTest { override val logDirCount = 5 override val brokerCount = 1 - + var testInfo : TestInfo = _ val topic = "topic" + @BeforeEach + override def setUp(testInfo: TestInfo): Unit = { + this.testInfo = testInfo + if (testInfo.getTestMethod.get.getName == "testAlterReplicaLogDirsRequestWithRetention") { + this.serverConfig.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1024") + this.controllerConfig.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1024") + } + super.setUp(testInfo) + } + override def brokerPropertyOverrides(properties: Properties): Unit = { + if (testInfo != null && testInfo.getTestMethod.get.getName == "testAlterReplicaLogDirsRequestWithRetention") { + properties.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1024") + properties.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1024") + } properties.put(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, "0") properties.put(ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG, "1000") } @@ -148,7 +163,6 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { // We don't want files with `.deleted` suffix are removed too fast, // so we can validate there will be orphan files and orphan files will be removed eventually. topicProperties.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, "10000") - topicProperties.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1024") createTopic(topic, partitionNum, 1, topicProperties) assertEquals(logDir1, brokers.head.logManager.getLog(tp).get.dir.getParent) From 0f81aac38a2966988a1b7d2ea945a5720a153561 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 29 Apr 2025 23:54:00 +0800 Subject: [PATCH 53/74] fix some error --- core/src/main/scala/kafka/log/LogManager.scala | 4 ++-- .../org/apache/kafka/storage/internals/log/LogConfig.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 80b17fb44143f..e22a78a08dd1d 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -582,7 +582,7 @@ class LogManager(logDirs: Seq[File], val overrides = configRepository.topicConfig(topicName) // save memory by only including configs for topics with overrides if (!overrides.isEmpty) { - val logConfig = LogConfig.fromProps(defaultProps, overrides, defaultConfig.segmentSize()) + val logConfig = LogConfig.fromProps(defaultProps, overrides) topicConfigOverrides(topicName) = logConfig } } @@ -948,7 +948,7 @@ class LogManager(logDirs: Seq[File], wasRemoteLogEnabled: Boolean): Unit = { topicConfigUpdated(topic) val logs = logsByTopic(topic) - val newLogConfig = LogConfig.fromProps(currentDefaultConfig.originals, newTopicConfig, currentDefaultConfig.segmentSize()) + val newLogConfig = LogConfig.fromProps(currentDefaultConfig.originals, newTopicConfig) val isRemoteLogStorageEnabled = newLogConfig.remoteStorageEnable() // We would like to validate the configuration no matter whether the logs have materialised on disk or not. // Otherwise we risk someone creating a tiered-topic, disabling Tiered Storage cluster-wide and the check diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 70b55e0f96b6d..ebc4ea60e8a57 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -263,7 +263,7 @@ public Optional serverConfigName(String configName) { * should also be in `KafkaConfig#extractLogConfigMap`. */ private final int segmentSize; - private final Integer internalSegmentSize; + public final Integer internalSegmentSize; public final long segmentMs; public final long segmentJitterMs; public final int maxIndexSize; From e0fa8d0574bd4a0f74f519cb683bff456c7b0e6a Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 1 May 2025 22:10:28 +0800 Subject: [PATCH 54/74] reset to trunk --- .../clients/consumer/ShareConsumerTest.java | 120 +--- .../kafka/clients/CommonClientConfigs.java | 14 +- .../AbstractHeartbeatRequestManager.java | 5 +- .../server/builders/KafkaApisBuilder.java | 10 +- .../kafka/server/share/SharePartition.java | 62 +- .../server/share/SharePartitionManager.java | 16 +- .../src/main/scala/kafka/log/LogManager.scala | 2 +- .../scala/kafka/raft/KafkaMetadataLog.scala | 19 +- .../main/scala/kafka/raft/RaftManager.scala | 3 +- .../server/AutoTopicCreationManager.scala | 8 +- .../scala/kafka/server/BrokerServer.scala | 74 +- .../kafka/server/DynamicBrokerConfig.scala | 2 +- .../main/scala/kafka/server/KafkaApis.scala | 119 ++-- .../metadata/BrokerMetadataPublisher.scala | 40 +- .../share/SharePartitionManagerTest.java | 93 +-- .../server/share/SharePartitionTest.java | 130 ++-- .../AbstractAuthorizerIntegrationTest.scala | 2 - .../kafka/api/AuthorizerIntegrationTest.scala | 648 +----------------- .../kafka/api/IntegrationTestHarness.scala | 4 + .../api/PlaintextAdminIntegrationTest.scala | 8 +- .../api/SaslSslAdminIntegrationTest.scala | 4 +- .../kafka/raft/KafkaMetadataLogTest.scala | 35 +- .../kafka/cluster/AbstractPartitionTest.scala | 6 +- .../kafka/cluster/PartitionLockTest.scala | 4 +- .../AbstractLogCleanerIntegrationTest.scala | 7 +- .../kafka/log/LogCleanerIntegrationTest.scala | 2 +- .../log/LogCleanerLagIntegrationTest.scala | 2 +- .../kafka/log/LogCleanerManagerTest.scala | 19 +- ...gCleanerParameterizedIntegrationTest.scala | 6 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 78 ++- .../unit/kafka/log/LogConcurrencyTest.scala | 4 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 4 +- .../scala/unit/kafka/log/LogManagerTest.scala | 24 +- .../scala/unit/kafka/log/LogTestUtils.scala | 7 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 51 +- .../AlterReplicaLogDirsRequestTest.scala | 18 +- .../server/AutoTopicCreationManagerTest.scala | 22 +- .../server/DynamicBrokerConfigTest.scala | 12 +- .../server/DynamicConfigChangeTest.scala | 4 +- .../GroupCoordinatorBaseRequestTest.scala | 29 - .../unit/kafka/server/KafkaApisTest.scala | 552 +++++++++------ .../unit/kafka/server/KafkaConfigTest.scala | 3 +- .../ShareFetchAcknowledgeRequestTest.scala | 311 ++++----- .../ShareGroupDescribeRequestTest.scala | 9 +- .../ShareGroupHeartbeatRequestTest.scala | 91 +-- .../BrokerMetadataPublisherTest.scala | 2 +- .../kafka/tools/DumpLogSegmentsTest.scala | 3 +- .../group/modern/share/ShareGroupConfig.java | 14 +- .../group/GroupMetadataManagerTest.java | 46 +- .../modern/share/ShareGroupConfigTest.java | 1 + .../apache/kafka/raft/KafkaRaftClient.java | 3 +- .../apache/kafka/raft/MetadataLogConfig.java | 17 +- .../kafka/server/config/ServerLogConfigs.java | 3 - .../server/config/AbstractKafkaConfig.java | 4 - .../share/session/ShareSessionCache.java | 35 +- .../share/session/ShareSessionCacheTest.java | 46 +- .../kafka/storage/internals/log/Cleaner.java | 2 +- .../storage/internals/log/LogConfig.java | 45 +- .../storage/internals/log/UnifiedLog.java | 8 +- .../storage/internals/log/LocalLogTest.java | 4 +- .../PurgeRepartitionTopicIntegrationTest.java | 19 +- .../assignors/StickyTaskAssignor.java | 4 +- .../SessionWindowedDeserializerTest.java | 30 +- .../SessionWindowedSerializerTest.java | 30 +- .../kstream/TimeWindowedDeserializerTest.java | 57 +- .../kstream/TimeWindowedSerializerTest.java | 30 +- .../streams/tests/RelationalSmokeTest.java | 12 +- .../tests/RelationalSmokeTestTest.java | 1 - .../services/kafka/config_property.py | 2 +- tests/kafkatest/services/kafka/kafka.py | 7 +- .../tests/streams/base_streams_test.py | 1 - .../streams/streams_relational_smoke_test.py | 25 +- .../consumer/group/ShareGroupCommand.java | 84 +-- .../group/ShareGroupCommandOptions.java | 4 +- .../kafka/tools/GetOffsetShellTest.java | 2 +- .../apache/kafka/tools/GroupsCommandTest.java | 2 +- .../consumer/group/ShareGroupCommandTest.java | 205 ------ 77 files changed, 1145 insertions(+), 2286 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java index edb6d56215eee..d48f620880ef3 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java @@ -36,7 +36,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.config.ConfigResource; -import org.apache.kafka.common.errors.GroupMaxSizeReachedException; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.InvalidRecordStateException; @@ -56,6 +55,7 @@ import org.apache.kafka.common.test.api.ClusterConfigProperty; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; +import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.coordinator.group.GroupConfig; import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig; @@ -113,6 +113,8 @@ @ClusterTestDefaults( serverProperties = { @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), + @ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + @ClusterConfigProperty(key = "group.share.enable", value = "true"), @ClusterConfigProperty(key = "group.share.partition.max.record.locks", value = "10000"), @ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000"), @ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), @@ -121,7 +123,8 @@ @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), @ClusterConfigProperty(key = "transaction.state.log.min.isr", value = "1"), @ClusterConfigProperty(key = "transaction.state.log.replication.factor", value = "1") - } + }, + types = {Type.KRAFT} ) public class ShareConsumerTest { private final ClusterInstance cluster; @@ -1709,9 +1712,11 @@ public void testShareAutoOffsetResetEarliest() { public void testShareAutoOffsetResetEarliestAfterLsoMovement() { alterShareAutoOffsetReset("group1", "earliest"); try ( + ShareConsumer shareConsumer = createShareConsumer("group1"); Producer producer = createProducer(); Admin adminClient = createAdminClient() ) { + shareConsumer.subscribe(Set.of(tp.topic())); ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); // We write 10 records to the topic, so they would be written from offsets 0-9 on the topic. @@ -1846,6 +1851,8 @@ public void testShareAutoOffsetResetByDurationInvalidFormat() throws Exception { brokers = 3, serverProperties = { @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), + @ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + @ClusterConfigProperty(key = "group.share.enable", value = "true"), @ClusterConfigProperty(key = "group.share.partition.max.record.locks", value = "10000"), @ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000"), @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "3"), @@ -1991,66 +1998,12 @@ public void testShareConsumerAfterCoordinatorMovement() throws Exception { verifyShareGroupStateTopicRecordsProduced(); } - @ClusterTest - public void testDeliveryCountNotIncreaseAfterSessionClose() { - alterShareAutoOffsetReset("group1", "earliest"); - try (Producer producer = createProducer()) { - ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); - // We write 10 records to the topic, so they would be written from offsets 0-9 on the topic. - for (int i = 0; i < 10; i++) { - assertDoesNotThrow(() -> producer.send(record).get(), "Failed to send records"); - } - } - - // Perform the fetch, close in a loop. - for (int count = 0; count < ShareGroupConfig.SHARE_GROUP_DELIVERY_COUNT_LIMIT_DEFAULT; count++) { - consumeMessages(new AtomicInteger(0), 10, "group1", 1, 10, false); - } - - // If the delivery count is increased, consumer will get nothing. - int consumedMessageCount = consumeMessages(new AtomicInteger(0), 10, "group1", 1, 10, true); - // The records returned belong to offsets 0-9. - assertEquals(10, consumedMessageCount); - verifyShareGroupStateTopicRecordsProduced(); - } - - @ClusterTest - public void testDeliveryCountDifferentBehaviorWhenClosingSessionWithExplicitAcknowledgement() { - alterShareAutoOffsetReset("group1", "earliest"); - try (Producer producer = createProducer(); - ShareConsumer shareConsumer = createShareConsumer( - "group1", - Map.of(ConsumerConfig.SHARE_ACKNOWLEDGEMENT_MODE_CONFIG, EXPLICIT))) { - - ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, - "key".getBytes(), "value".getBytes()); - producer.send(record); - producer.send(record); - producer.flush(); - - shareConsumer.subscribe(Set.of(tp.topic())); - ConsumerRecords records = waitedPoll(shareConsumer, 2500L, 2); - assertEquals(2, records.count()); - // Acknowledge the first record with AcknowledgeType.RELEASE - shareConsumer.acknowledge(records.records(tp).get(0), AcknowledgeType.RELEASE); - Map> result = shareConsumer.commitSync(); - assertEquals(1, result.size()); - } - - // Test delivery count - try (ShareConsumer shareConsumer = createShareConsumer("group1", Map.of())) { - shareConsumer.subscribe(Set.of(tp.topic())); - ConsumerRecords records = waitedPoll(shareConsumer, 2500L, 2); - assertEquals(2, records.count()); - assertEquals((short) 2, records.records(tp).get(0).deliveryCount().get()); - assertEquals((short) 1, records.records(tp).get(1).deliveryCount().get()); - } - } - @ClusterTest( brokers = 3, serverProperties = { @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), + @ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + @ClusterConfigProperty(key = "group.share.enable", value = "true"), @ClusterConfigProperty(key = "group.share.partition.max.record.locks", value = "10000"), @ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000"), @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "3"), @@ -2112,57 +2065,6 @@ public void testComplexShareConsumer() throws Exception { verifyShareGroupStateTopicRecordsProduced(); } - @ClusterTest( - brokers = 1, - serverProperties = { - @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), - @ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - @ClusterConfigProperty(key = "group.share.enable", value = "true"), - @ClusterConfigProperty(key = "group.share.partition.max.record.locks", value = "10000"), - @ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000"), - @ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), - @ClusterConfigProperty(key = "share.coordinator.state.topic.min.isr", value = "1"), - @ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "3"), - @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - @ClusterConfigProperty(key = "transaction.state.log.min.isr", value = "1"), - @ClusterConfigProperty(key = "transaction.state.log.replication.factor", value = "1"), - @ClusterConfigProperty(key = "group.share.max.size", value = "3") // Setting max group size to 3 - } - ) - public void testShareGroupMaxSizeConfigExceeded() throws Exception { - // creating 3 consumers in the group1 - ShareConsumer shareConsumer1 = createShareConsumer("group1"); - ShareConsumer shareConsumer2 = createShareConsumer("group1"); - ShareConsumer shareConsumer3 = createShareConsumer("group1"); - - shareConsumer1.subscribe(Set.of(tp.topic())); - shareConsumer2.subscribe(Set.of(tp.topic())); - shareConsumer3.subscribe(Set.of(tp.topic())); - - shareConsumer1.poll(Duration.ofMillis(5000)); - shareConsumer2.poll(Duration.ofMillis(5000)); - shareConsumer3.poll(Duration.ofMillis(5000)); - - ShareConsumer shareConsumer4 = createShareConsumer("group1"); - shareConsumer4.subscribe(Set.of(tp.topic())); - - TestUtils.waitForCondition(() -> { - try { - shareConsumer4.poll(Duration.ofMillis(5000)); - } catch (GroupMaxSizeReachedException e) { - return true; - } catch (Exception e) { - return false; - } - return false; - }, 30000, 200L, () -> "The 4th consumer was not kicked out of the group"); - - shareConsumer1.close(); - shareConsumer2.close(); - shareConsumer3.close(); - shareConsumer4.close(); - } - @ClusterTest public void testReadCommittedIsolationLevel() { alterShareAutoOffsetReset("group1", "earliest"); diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index e2022e0f4d02f..aa3b5c9d628c9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -192,8 +192,7 @@ public class CommonClientConfigs { + "is considered failed and the group will rebalance in order to reassign the partitions to another member. " + "For consumers using a non-null group.instance.id which reach this timeout, partitions will not be immediately reassigned. " + "Instead, the consumer will stop sending heartbeats and partitions will be reassigned " - + "after expiration of the session timeout (defined by the client config session.timeout.ms if using the Classic rebalance protocol, or by the broker config group.consumer.session.timeout.ms if using the Consumer protocol). " - + "This mirrors the behavior of a static consumer which has shutdown."; + + "after expiration of session.timeout.ms. This mirrors the behavior of a static consumer which has shutdown."; public static final String REBALANCE_TIMEOUT_MS_CONFIG = "rebalance.timeout.ms"; public static final String REBALANCE_TIMEOUT_MS_DOC = "The maximum allowed time for each worker to join the group " @@ -207,18 +206,15 @@ public class CommonClientConfigs { + "to the broker. If no heartbeats are received by the broker before the expiration of this session timeout, " + "then the broker will remove this client from the group and initiate a rebalance. Note that the value " + "must be in the allowable range as configured in the broker configuration by group.min.session.timeout.ms " - + "and group.max.session.timeout.ms. Note that this client configuration is not supported when group.protocol " - + "is set to \"consumer\". In that case, session timeout is controlled by the broker config group.consumer.session.timeout.ms."; + + "and group.max.session.timeout.ms. Note that this configuration is not supported when group.protocol " + + "is set to \"consumer\"."; public static final String HEARTBEAT_INTERVAL_MS_CONFIG = "heartbeat.interval.ms"; public static final String HEARTBEAT_INTERVAL_MS_DOC = "The expected time between heartbeats to the consumer " + "coordinator when using Kafka's group management facilities. Heartbeats are used to ensure that the " + "consumer's session stays active and to facilitate rebalancing when new consumers join or leave the group. " - + "This config is only supported if group.protocol is set to \"classic\". In that case, " - + "the value must be set lower than session.timeout.ms, but typically should be set no higher " - + "than 1/3 of that value. It can be adjusted even lower to control the expected time for normal rebalances." - + "If group.protocol is set to \"consumer\", this config is not supported, as " - + "the heartbeat interval is controlled by the broker with group.consumer.heartbeat.interval.ms."; + + "The value must be set lower than session.timeout.ms, but typically should be set no higher " + + "than 1/3 of that value. It can be adjusted even lower to control the expected time for normal rebalances."; public static final String DEFAULT_API_TIMEOUT_MS_CONFIG = "default.api.timeout.ms"; public static final String DEFAULT_API_TIMEOUT_MS_DOC = "Specifies the timeout (in milliseconds) for client APIs. " + diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java index 3998d672006a3..9d219907926d4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java @@ -63,9 +63,8 @@ public abstract class AbstractHeartbeatRequestManager shareCoordinator = Optional.empty(); private GroupConfigManager groupConfigManager = null; public KafkaApisBuilder setRequestChannel(RequestChannel requestChannel) { @@ -97,7 +97,7 @@ public KafkaApisBuilder setTxnCoordinator(TransactionCoordinator txnCoordinator) return this; } - public KafkaApisBuilder setShareCoordinator(ShareCoordinator shareCoordinator) { + public KafkaApisBuilder setShareCoordinator(Optional shareCoordinator) { this.shareCoordinator = shareCoordinator; return this; } @@ -194,8 +194,8 @@ public KafkaApis build() { if (replicaManager == null) throw new RuntimeException("You must set replicaManager"); if (groupCoordinator == null) throw new RuntimeException("You must set groupCoordinator"); if (txnCoordinator == null) throw new RuntimeException("You must set txnCoordinator"); - if (shareCoordinator == null) throw new RuntimeException("You must set shareCoordinator"); - if (autoTopicCreationManager == null) throw new RuntimeException("You must set autoTopicCreationManager"); + if (autoTopicCreationManager == null) + throw new RuntimeException("You must set autoTopicCreationManager"); if (config == null) config = new KafkaConfig(Map.of()); if (configRepository == null) throw new RuntimeException("You must set configRepository"); if (metadataCache == null) throw new RuntimeException("You must set metadataCache"); @@ -213,7 +213,7 @@ public KafkaApis build() { replicaManager, groupCoordinator, txnCoordinator, - shareCoordinator, + OptionConverters.toScala(shareCoordinator), autoTopicCreationManager, brokerId, config, diff --git a/core/src/main/java/kafka/server/share/SharePartition.java b/core/src/main/java/kafka/server/share/SharePartition.java index fa80c230a5813..db2ff4fc8cfeb 100644 --- a/core/src/main/java/kafka/server/share/SharePartition.java +++ b/core/src/main/java/kafka/server/share/SharePartition.java @@ -197,16 +197,6 @@ public byte id() { } } - /** - * The DeliveryCountOps is used to specify the behavior on the delivery count: increase, decrease, - * or do nothing. - */ - private enum DeliveryCountOps { - INCREASE, - DECREASE, - NO_OP - } - /** * The group id of the share partition belongs to. */ @@ -842,7 +832,7 @@ public ShareAcquiredRecords acquire( continue; } - InFlightState updateResult = inFlightBatch.tryUpdateBatchState(RecordState.ACQUIRED, DeliveryCountOps.INCREASE, maxDeliveryCount, memberId); + InFlightState updateResult = inFlightBatch.tryUpdateBatchState(RecordState.ACQUIRED, true, maxDeliveryCount, memberId); if (updateResult == null) { log.info("Unable to acquire records for the batch: {} in share partition: {}-{}", inFlightBatch, groupId, topicIdPartition); @@ -1034,10 +1024,9 @@ private Optional releaseAcquiredRecordsForPerOffsetBatch(String membe return Optional.empty(); } if (offsetState.getValue().state == RecordState.ACQUIRED) { - // These records were fetched but they were not actually delivered to the client. InFlightState updateResult = offsetState.getValue().startStateTransition( offsetState.getKey() < startOffset ? RecordState.ARCHIVED : recordState, - DeliveryCountOps.DECREASE, + false, this.maxDeliveryCount, EMPTY_MEMBER_ID ); @@ -1083,7 +1072,7 @@ private Optional releaseAcquiredRecordsForCompleteBatch(String member if (inFlightBatch.batchState() == RecordState.ACQUIRED) { InFlightState updateResult = inFlightBatch.startBatchStateTransition( inFlightBatch.lastOffset() < startOffset ? RecordState.ARCHIVED : recordState, - DeliveryCountOps.DECREASE, + false, this.maxDeliveryCount, EMPTY_MEMBER_ID ); @@ -1635,8 +1624,8 @@ private int acquireSubsetBatchRecords( continue; } - InFlightState updateResult = offsetState.getValue().tryUpdateState(RecordState.ACQUIRED, DeliveryCountOps.INCREASE, - maxDeliveryCount, memberId); + InFlightState updateResult = offsetState.getValue().tryUpdateState(RecordState.ACQUIRED, true, maxDeliveryCount, + memberId); if (updateResult == null) { log.trace("Unable to acquire records for the offset: {} in batch: {}" + " for the share partition: {}-{}", offsetState.getKey(), inFlightBatch, @@ -1908,7 +1897,7 @@ private Optional acknowledgePerOffsetBatchRecords( recordStateDefault; InFlightState updateResult = offsetState.getValue().startStateTransition( recordState, - DeliveryCountOps.NO_OP, + false, this.maxDeliveryCount, EMPTY_MEMBER_ID ); @@ -1961,7 +1950,7 @@ private Optional acknowledgeCompleteBatch( // is only important when the batch is acquired. InFlightState updateResult = inFlightBatch.startBatchStateTransition( recordState, - DeliveryCountOps.NO_OP, + false, this.maxDeliveryCount, EMPTY_MEMBER_ID ); @@ -2420,7 +2409,7 @@ private void releaseAcquisitionLockOnTimeoutForCompleteBatch(InFlightBatch inFli if (inFlightBatch.batchState() == RecordState.ACQUIRED) { InFlightState updateResult = inFlightBatch.tryUpdateBatchState( inFlightBatch.lastOffset() < startOffset ? RecordState.ARCHIVED : RecordState.AVAILABLE, - DeliveryCountOps.NO_OP, + false, maxDeliveryCount, EMPTY_MEMBER_ID); if (updateResult == null) { @@ -2466,7 +2455,7 @@ private void releaseAcquisitionLockOnTimeoutForPerOffsetBatch(InFlightBatch inFl } InFlightState updateResult = offsetState.getValue().tryUpdateState( offsetState.getKey() < startOffset ? RecordState.ARCHIVED : RecordState.AVAILABLE, - DeliveryCountOps.NO_OP, + false, maxDeliveryCount, EMPTY_MEMBER_ID); if (updateResult == null) { @@ -2886,19 +2875,19 @@ private void archiveBatch(String newMemberId) { inFlightState().archive(newMemberId); } - private InFlightState tryUpdateBatchState(RecordState newState, DeliveryCountOps ops, int maxDeliveryCount, String newMemberId) { + private InFlightState tryUpdateBatchState(RecordState newState, boolean incrementDeliveryCount, int maxDeliveryCount, String newMemberId) { if (batchState == null) { throw new IllegalStateException("The batch state update is not available as the offset state is maintained"); } - return batchState.tryUpdateState(newState, ops, maxDeliveryCount, newMemberId); + return batchState.tryUpdateState(newState, incrementDeliveryCount, maxDeliveryCount, newMemberId); } - private InFlightState startBatchStateTransition(RecordState newState, DeliveryCountOps ops, int maxDeliveryCount, + private InFlightState startBatchStateTransition(RecordState newState, boolean incrementDeliveryCount, int maxDeliveryCount, String newMemberId) { if (batchState == null) { throw new IllegalStateException("The batch state update is not available as the offset state is maintained"); } - return batchState.startStateTransition(newState, ops, maxDeliveryCount, newMemberId); + return batchState.startStateTransition(newState, incrementDeliveryCount, maxDeliveryCount, newMemberId); } private void maybeInitializeOffsetStateUpdate() { @@ -3010,23 +2999,23 @@ private boolean hasOngoingStateTransition() { /** * Try to update the state of the records. The state of the records can only be updated if the - * new state is allowed to be transitioned from old state. The delivery count is not changed + * new state is allowed to be transitioned from old state. The delivery count is not incremented * if the state update is unsuccessful. * * @param newState The new state of the records. - * @param ops The behavior on the delivery count. + * @param incrementDeliveryCount Whether to increment the delivery count. * * @return {@code InFlightState} if update succeeds, null otherwise. Returning state * helps update chaining. */ - private InFlightState tryUpdateState(RecordState newState, DeliveryCountOps ops, int maxDeliveryCount, String newMemberId) { + private InFlightState tryUpdateState(RecordState newState, boolean incrementDeliveryCount, int maxDeliveryCount, String newMemberId) { try { - if (newState == RecordState.AVAILABLE && ops != DeliveryCountOps.DECREASE && deliveryCount >= maxDeliveryCount) { + if (newState == RecordState.AVAILABLE && deliveryCount >= maxDeliveryCount) { newState = RecordState.ARCHIVED; } state = state.validateTransition(newState); - if (newState != RecordState.ARCHIVED) { - deliveryCount = updatedDeliveryCount(ops); + if (incrementDeliveryCount && newState != RecordState.ARCHIVED) { + deliveryCount++; } memberId = newMemberId; return this; @@ -3036,23 +3025,14 @@ private InFlightState tryUpdateState(RecordState newState, DeliveryCountOps ops, } } - private int updatedDeliveryCount(DeliveryCountOps ops) { - return switch (ops) { - case INCREASE -> deliveryCount + 1; - case DECREASE -> deliveryCount - 1; - // do nothing - case NO_OP -> deliveryCount; - }; - } - private void archive(String newMemberId) { state = RecordState.ARCHIVED; memberId = newMemberId; } - private InFlightState startStateTransition(RecordState newState, DeliveryCountOps ops, int maxDeliveryCount, String newMemberId) { + private InFlightState startStateTransition(RecordState newState, boolean incrementDeliveryCount, int maxDeliveryCount, String newMemberId) { rollbackState = new InFlightState(state, deliveryCount, memberId, acquisitionLockTimeoutTask); - return tryUpdateState(newState, ops, maxDeliveryCount, newMemberId); + return tryUpdateState(newState, incrementDeliveryCount, maxDeliveryCount, newMemberId); } private void completeStateTransition(boolean commit) { diff --git a/core/src/main/java/kafka/server/share/SharePartitionManager.java b/core/src/main/java/kafka/server/share/SharePartitionManager.java index 44af40ec8f8d6..a53f846a01c24 100644 --- a/core/src/main/java/kafka/server/share/SharePartitionManager.java +++ b/core/src/main/java/kafka/server/share/SharePartitionManager.java @@ -420,18 +420,11 @@ private CompletableFuture shareFetchData, - List toForget, - ShareRequestMetadata reqMetadata, - Boolean isAcknowledgeDataPresent, - String clientConnectionId - ) { + public ShareFetchContext newContext(String groupId, List shareFetchData, + List toForget, ShareRequestMetadata reqMetadata, Boolean isAcknowledgeDataPresent) { ShareFetchContext context; // If the request's epoch is FINAL_EPOCH or INITIAL_EPOCH, we should remove the existing sessions. Also, start a // new session in case it is INITIAL_EPOCH. Hence, we need to treat them as special cases. @@ -455,8 +448,7 @@ public ShareFetchContext newContext( ImplicitLinkedHashCollection<>(shareFetchData.size()); shareFetchData.forEach(topicIdPartition -> cachedSharePartitions.mustAdd(new CachedSharePartition(topicIdPartition, false))); - ShareSessionKey responseShareSessionKey = cache.maybeCreateSession(groupId, reqMetadata.memberId(), - cachedSharePartitions, clientConnectionId); + ShareSessionKey responseShareSessionKey = cache.maybeCreateSession(groupId, reqMetadata.memberId(), cachedSharePartitions); if (responseShareSessionKey == null) { log.error("Could not create a share session for group {} member {}", groupId, reqMetadata.memberId()); throw Errors.SHARE_SESSION_NOT_FOUND.exception(); diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index e22a78a08dd1d..c116bc398cfa6 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -1528,7 +1528,7 @@ object LogManager { val defaultProps = config.extractLogConfigMap LogConfig.validateBrokerLogConfigValues(defaultProps, config.remoteLogManagerConfig.isRemoteStorageSystemEnabled) - val defaultLogConfig = new LogConfig(defaultProps, config.internalSegmentSize()) + val defaultLogConfig = new LogConfig(defaultProps) val cleanerConfig = new CleanerConfig(config) val transactionLogConfig = new TransactionLogConfig(config) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index afd2101608f80..01f69b374bc3d 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record.{MemoryRecords, Records} import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} -import org.apache.kafka.raft.{Isolation, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} +import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.Scheduler @@ -583,8 +583,7 @@ object KafkaMetadataLog extends Logging { time: Time, scheduler: Scheduler, config: MetadataLogConfig, - nodeId: Int, - internalSegmentSize: Integer = null + nodeId: Int ): KafkaMetadataLog = { val props = new Properties() props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString) @@ -596,9 +595,13 @@ object KafkaMetadataLog extends Logging { props.setProperty(TopicConfig.RETENTION_MS_CONFIG, "-1") props.setProperty(TopicConfig.RETENTION_BYTES_CONFIG, "-1") LogConfig.validate(props) - val defaultLogConfig = new LogConfig(props, internalSegmentSize) + val defaultLogConfig = new LogConfig(props) - if (defaultLogConfig.retentionMs >= 0) { + if (config.logSegmentBytes < config.logSegmentMinBytes) { + throw new InvalidConfigurationException( + s"Cannot set ${MetadataLogConfig.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}." ) @@ -634,6 +637,12 @@ object KafkaMetadataLog extends Logging { nodeId ) + // Print a warning if users have overridden the internal config + if (config.logSegmentMinBytes != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) { + metadataLog.error(s"Overriding ${MetadataLogConfig.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.") + } + // 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() diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index 266fe1317ce20..0727c660fe406 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -230,8 +230,7 @@ class KafkaRaftManager[T]( time, scheduler, config = new MetadataLogConfig(config), - config.nodeId, - config.internalSegmentSize() + config.nodeId ) } diff --git a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala index 0cd8e230c9daa..a2c2bd4d80b35 100644 --- a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala +++ b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala @@ -59,7 +59,7 @@ class DefaultAutoTopicCreationManager( channelManager: NodeToControllerChannelManager, groupCoordinator: GroupCoordinator, txnCoordinator: TransactionCoordinator, - shareCoordinator: ShareCoordinator + shareCoordinator: Option[ShareCoordinator] ) extends AutoTopicCreationManager with Logging { private val inflightTopics = Collections.newSetFromMap(new ConcurrentHashMap[String, java.lang.Boolean]()) @@ -198,11 +198,15 @@ class DefaultAutoTopicCreationManager( .setConfigs(convertToTopicConfigCollections( txnCoordinator.transactionTopicConfigs)) case SHARE_GROUP_STATE_TOPIC_NAME => + val props = shareCoordinator match { + case Some(coordinator) => coordinator.shareGroupStateTopicConfigs() + case None => new Properties() + } new CreatableTopic() .setName(topic) .setNumPartitions(config.shareCoordinatorConfig.shareCoordinatorStateTopicNumPartitions()) .setReplicationFactor(config.shareCoordinatorConfig.shareCoordinatorStateTopicReplicationFactor()) - .setConfigs(convertToTopicConfigCollections(shareCoordinator.shareGroupStateTopicConfigs())) + .setConfigs(convertToTopicConfigCollections(props)) case topicName => new CreatableTopic() .setName(topicName) diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index d219e6461cd71..172c02ae924bd 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -126,7 +126,7 @@ class BrokerServer( var transactionCoordinator: TransactionCoordinator = _ - var shareCoordinator: ShareCoordinator = _ + var shareCoordinator: Option[ShareCoordinator] = None var clientToControllerChannelManager: NodeToControllerChannelManager = _ @@ -259,13 +259,7 @@ class BrokerServer( Optional.of(clientMetricsManager) ) - val shareFetchSessionCache : ShareSessionCache = new ShareSessionCache(config.shareGroupConfig.shareGroupMaxShareSessions()) - - val connectionDisconnectListeners = Seq( - clientMetricsManager.connectionDisconnectListener(), - shareFetchSessionCache.connectionDisconnectListener() - ) - + val connectionDisconnectListeners = Seq(clientMetricsManager.connectionDisconnectListener()) // Create and start the socket server acceptor threads so that the bound port is known. // Delay starting processors until the end of the initialization sequence to ensure // that credentials have been loaded before processing authentications. @@ -432,6 +426,8 @@ class BrokerServer( )) val fetchManager = new FetchManager(Time.SYSTEM, new FetchSessionCache(fetchSessionCacheShards)) + val shareFetchSessionCache : ShareSessionCache = new ShareSessionCache(config.shareGroupConfig.shareGroupMaxShareSessions()) + sharePartitionManager = new SharePartitionManager( replicaManager, time, @@ -633,35 +629,41 @@ class BrokerServer( .build() } - private def createShareCoordinator(): ShareCoordinator = { - val time = Time.SYSTEM - val timer = new SystemTimerReaper( - "share-coordinator-reaper", - new SystemTimer("share-coordinator") - ) + private def createShareCoordinator(): Option[ShareCoordinator] = { + if (config.shareGroupConfig.isShareGroupEnabled && + config.shareGroupConfig.shareGroupPersisterClassName().nonEmpty) { + val time = Time.SYSTEM + val timer = new SystemTimerReaper( + "share-coordinator-reaper", + new SystemTimer("share-coordinator") + ) - val serde = new ShareCoordinatorRecordSerde - val loader = new CoordinatorLoaderImpl[CoordinatorRecord]( - time, - replicaManager, - serde, - config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize() - ) - val writer = new CoordinatorPartitionWriter( - replicaManager - ) - new ShareCoordinatorService.Builder(config.brokerId, config.shareCoordinatorConfig) - .withTimer(timer) - .withTime(time) - .withLoader(loader) - .withWriter(writer) - .withCoordinatorRuntimeMetrics(new ShareCoordinatorRuntimeMetrics(metrics)) - .withCoordinatorMetrics(new ShareCoordinatorMetrics(metrics)) - .build() + val serde = new ShareCoordinatorRecordSerde + val loader = new CoordinatorLoaderImpl[CoordinatorRecord]( + time, + replicaManager, + serde, + config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize() + ) + val writer = new CoordinatorPartitionWriter( + replicaManager + ) + Some(new ShareCoordinatorService.Builder(config.brokerId, config.shareCoordinatorConfig) + .withTimer(timer) + .withTime(time) + .withLoader(loader) + .withWriter(writer) + .withCoordinatorRuntimeMetrics(new ShareCoordinatorRuntimeMetrics(metrics)) + .withCoordinatorMetrics(new ShareCoordinatorMetrics(metrics)) + .build()) + } else { + None + } } private def createShareStatePersister(): Persister = { - if (config.shareGroupConfig.shareGroupPersisterClassName.nonEmpty) { + if (config.shareGroupConfig.isShareGroupEnabled && + config.shareGroupConfig.shareGroupPersisterClassName.nonEmpty) { val klass = Utils.loadClass(config.shareGroupConfig.shareGroupPersisterClassName, classOf[Object]).asInstanceOf[Class[Persister]] if (klass.getName.equals(classOf[DefaultStatePersister].getName)) { @@ -669,7 +671,7 @@ class BrokerServer( .newInstance( new PersisterStateManager( NetworkUtils.buildNetworkClient("Persister", config, metrics, Time.SYSTEM, new LogContext(s"[Persister broker=${config.brokerId}]")), - new ShareCoordinatorMetadataCacheHelperImpl(metadataCache, key => shareCoordinator.partitionFor(key), config.interBrokerListenerName), + new ShareCoordinatorMetadataCacheHelperImpl(metadataCache, key => shareCoordinator.get.partitionFor(key), config.interBrokerListenerName), Time.SYSTEM, new SystemTimerReaper( "persister-state-manager-reaper", @@ -780,8 +782,8 @@ class BrokerServer( CoreUtils.swallow(groupConfigManager.close(), this) if (groupCoordinator != null) CoreUtils.swallow(groupCoordinator.shutdown(), this) - if (shareCoordinator != null) - CoreUtils.swallow(shareCoordinator.shutdown(), this) + if (shareCoordinator.isDefined) + CoreUtils.swallow(shareCoordinator.get.shutdown(), this) if (assignmentsManager != null) CoreUtils.swallow(assignmentsManager.close(), this) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index c936da2a7a990..dc4a09e0d23da 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -705,7 +705,7 @@ class DynamicLogConfig(logManager: LogManager) extends BrokerReconfigurable with override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { val newBrokerDefaults = new util.HashMap[String, Object](newConfig.extractLogConfigMap) - logManager.reconfigureDefaultLogConfig(new LogConfig(newBrokerDefaults, newConfig.internalSegmentSize())) + logManager.reconfigureDefaultLogConfig(new LogConfig(newBrokerDefaults)) updateLogsConfig(newBrokerDefaults.asScala) } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 1e5180ae8dbb8..2841dda409ef0 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -62,7 +62,7 @@ import org.apache.kafka.coordinator.share.ShareCoordinator import org.apache.kafka.metadata.{ConfigRepository, MetadataCache} import org.apache.kafka.server.{ApiVersionManager, ClientMetricsManager, DelegationTokenManager, ProcessRole} import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.common.{GroupVersion, RequestLocal, ShareVersion, StreamsVersion, TransactionVersion} +import org.apache.kafka.server.common.{GroupVersion, RequestLocal, StreamsVersion, TransactionVersion} import org.apache.kafka.server.config.DelegationTokenManagerConfigs import org.apache.kafka.server.share.context.ShareFetchContext import org.apache.kafka.server.share.{ErroneousAndValidPartitionData, SharePartitionKey} @@ -91,7 +91,7 @@ class KafkaApis(val requestChannel: RequestChannel, val replicaManager: ReplicaManager, val groupCoordinator: GroupCoordinator, val txnCoordinator: TransactionCoordinator, - val shareCoordinator: ShareCoordinator, + val shareCoordinator: Option[ShareCoordinator], val autoTopicCreationManager: AutoTopicCreationManager, val brokerId: Int, val config: KafkaConfig, @@ -1185,6 +1185,9 @@ class KafkaApis(val requestChannel: RequestChannel, else { if (keyType == CoordinatorType.SHARE.id) { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) + if (shareCoordinator.isEmpty) { + return (Errors.INVALID_REQUEST, Node.noNode) + } try { SharePartitionKey.validate(key) } catch { @@ -1202,7 +1205,7 @@ class KafkaApis(val requestChannel: RequestChannel, case CoordinatorType.SHARE => // We know that shareCoordinator is defined at this stage. - (shareCoordinator.partitionFor(SharePartitionKey.getInstance(key)), SHARE_GROUP_STATE_TOPIC_NAME) + (shareCoordinator.get.partitionFor(SharePartitionKey.getInstance(key)), SHARE_GROUP_STATE_TOPIC_NAME) } val topicMetadata = metadataCache.getTopicMetadata(Set(internalTopicName).asJava, request.context.listenerName, false, false).asScala @@ -3041,7 +3044,7 @@ class KafkaApis(val requestChannel: RequestChannel, try { // Creating the shareFetchContext for Share Session Handling. if context creation fails, the request is failed directly here. - shareFetchContext = sharePartitionManager.newContext(groupId, shareFetchData, forgottenTopics, newReqMetadata, isAcknowledgeDataPresent, request.context.connectionId) + shareFetchContext = sharePartitionManager.newContext(groupId, shareFetchData, forgottenTopics, newReqMetadata, isAcknowledgeDataPresent) } catch { case e: Exception => requestHelper.sendMaybeThrottle(request, shareFetchRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, e)) @@ -3401,14 +3404,21 @@ class KafkaApis(val requestChannel: RequestChannel, return CompletableFuture.completedFuture[Unit](()) } - shareCoordinator.initializeState(request.context, initializeShareGroupStateRequest.data) - .handle[Unit] { (response, exception) => - if (exception != null) { - requestHelper.sendMaybeThrottle(request, initializeShareGroupStateRequest.getErrorResponse(exception)) - } else { - requestHelper.sendMaybeThrottle(request, new InitializeShareGroupStateResponse(response)) + shareCoordinator match { + case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + initializeShareGroupStateRequest.getErrorResponse(requestThrottleMs, + new ApiException("Share coordinator is not enabled."))) + CompletableFuture.completedFuture[Unit](()) + + case Some(coordinator) => coordinator.initializeState(request.context, initializeShareGroupStateRequest.data) + .handle[Unit] { (response, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, initializeShareGroupStateRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, new InitializeShareGroupStateResponse(response)) + } } - } + } } def handleReadShareGroupStateRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { @@ -3423,14 +3433,20 @@ class KafkaApis(val requestChannel: RequestChannel, return CompletableFuture.completedFuture[Unit](()) } - shareCoordinator.readState(request.context, readShareGroupStateRequest.data) - .handle[Unit] { (response, exception) => - if (exception != null) { - requestHelper.sendMaybeThrottle(request, readShareGroupStateRequest.getErrorResponse(exception)) - } else { - requestHelper.sendMaybeThrottle(request, new ReadShareGroupStateResponse(response)) + shareCoordinator match { + case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + readShareGroupStateRequest.getErrorResponse(requestThrottleMs, + new ApiException("Share coordinator is not enabled."))) + CompletableFuture.completedFuture[Unit](()) + case Some(coordinator) => coordinator.readState(request.context, readShareGroupStateRequest.data) + .handle[Unit] { (response, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, readShareGroupStateRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, new ReadShareGroupStateResponse(response)) + } } - } + } } def handleWriteShareGroupStateRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { @@ -3445,14 +3461,20 @@ class KafkaApis(val requestChannel: RequestChannel, return CompletableFuture.completedFuture[Unit](()) } - shareCoordinator.writeState(request.context, writeShareGroupStateRequest.data) - .handle[Unit] { (response, exception) => - if (exception != null) { - requestHelper.sendMaybeThrottle(request, writeShareGroupStateRequest.getErrorResponse(exception)) - } else { - requestHelper.sendMaybeThrottle(request, new WriteShareGroupStateResponse(response)) + shareCoordinator match { + case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + writeShareGroupStateRequest.getErrorResponse(requestThrottleMs, + new ApiException("Share coordinator is not enabled."))) + CompletableFuture.completedFuture[Unit](()) + case Some(coordinator) => coordinator.writeState(request.context, writeShareGroupStateRequest.data) + .handle[Unit] { (response, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, writeShareGroupStateRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, new WriteShareGroupStateResponse(response)) + } } - } + } } def handleDeleteShareGroupStateRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { @@ -3467,14 +3489,21 @@ class KafkaApis(val requestChannel: RequestChannel, return CompletableFuture.completedFuture[Unit](()) } - shareCoordinator.deleteState(request.context, deleteShareGroupStateRequest.data) - .handle[Unit] { (response, exception) => - if (exception != null) { - requestHelper.sendMaybeThrottle(request, deleteShareGroupStateRequest.getErrorResponse(exception)) - } else { - requestHelper.sendMaybeThrottle(request, new DeleteShareGroupStateResponse(response)) + shareCoordinator match { + case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + deleteShareGroupStateRequest.getErrorResponse(requestThrottleMs, + new ApiException("Share coordinator is not enabled."))) + CompletableFuture.completedFuture[Unit](()) + + case Some(coordinator) => coordinator.deleteState(request.context, deleteShareGroupStateRequest.data) + .handle[Unit] { (response, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, deleteShareGroupStateRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, new DeleteShareGroupStateResponse(response)) + } } - } + } } def handleReadShareGroupStateSummaryRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { @@ -3489,14 +3518,20 @@ class KafkaApis(val requestChannel: RequestChannel, return CompletableFuture.completedFuture[Unit](()) } - shareCoordinator.readStateSummary(request.context, readShareGroupStateSummaryRequest.data) - .handle[Unit] { (response, exception) => - if (exception != null) { - requestHelper.sendMaybeThrottle(request, readShareGroupStateSummaryRequest.getErrorResponse(exception)) - } else { - requestHelper.sendMaybeThrottle(request, new ReadShareGroupStateSummaryResponse(response)) + shareCoordinator match { + case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + readShareGroupStateSummaryRequest.getErrorResponse(requestThrottleMs, + new ApiException("Share coordinator is not enabled."))) + CompletableFuture.completedFuture[Unit](()) + case Some(coordinator) => coordinator.readStateSummary(request.context, readShareGroupStateSummaryRequest.data) + .handle[Unit] { (response, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, readShareGroupStateSummaryRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, new ReadShareGroupStateSummaryResponse(response)) + } } - } + } } def handleDescribeShareGroupOffsetsRequest(request: RequestChannel.Request): Unit = { @@ -3929,12 +3964,8 @@ class KafkaApis(val requestChannel: RequestChannel, .setCurrentLeader(partitionData.currentLeader) } - private def shareVersion(): ShareVersion = { - ShareVersion.fromFeatureLevel(metadataCache.features.finalizedFeatures.getOrDefault(ShareVersion.FEATURE_NAME, 0.toShort)) - } - private def isShareGroupProtocolEnabled: Boolean = { - config.shareGroupConfig.isShareGroupEnabled || shareVersion().supportsShareGroups + config.shareGroupConfig.isShareGroupEnabled } private def updateRecordConversionStats(request: RequestChannel.Request, diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 7fbbe1f19597d..d95be34ff51de 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -71,7 +71,7 @@ class BrokerMetadataPublisher( replicaManager: ReplicaManager, groupCoordinator: GroupCoordinator, txnCoordinator: TransactionCoordinator, - shareCoordinator: ShareCoordinator, + shareCoordinator: Option[ShareCoordinator], var dynamicConfigPublisher: DynamicConfigPublisher, dynamicClientQuotaPublisher: DynamicClientQuotaPublisher, dynamicTopicClusterQuotaPublisher: DynamicTopicClusterQuotaPublisher, @@ -166,16 +166,18 @@ class BrokerMetadataPublisher( case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating txn " + s"coordinator with local changes in $deltaName", t) } - try { - updateCoordinator(newImage, - delta, - Topic.SHARE_GROUP_STATE_TOPIC_NAME, - shareCoordinator.onElection, - (partitionIndex, leaderEpochOpt) => shareCoordinator.onResignation(partitionIndex, toOptionalInt(leaderEpochOpt)) - ) - } catch { - case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating share " + - s"coordinator with local changes in $deltaName", t) + if (shareCoordinator.isDefined) { + try { + updateCoordinator(newImage, + delta, + Topic.SHARE_GROUP_STATE_TOPIC_NAME, + shareCoordinator.get.onElection, + (partitionIndex, leaderEpochOpt) => shareCoordinator.get.onResignation(partitionIndex, toOptionalInt(leaderEpochOpt)) + ) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating share " + + s"coordinator with local changes in $deltaName", t) + } } try { // Notify the group coordinator about deleted topics. @@ -223,7 +225,7 @@ class BrokerMetadataPublisher( try { // Propagate the new image to the share coordinator. - shareCoordinator.onNewMetadataImage(newImage, delta) + shareCoordinator.foreach(coordinator => coordinator.onNewMetadataImage(newImage, delta)) } catch { case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating share " + s"coordinator with local changes in $deltaName", t) @@ -338,12 +340,14 @@ class BrokerMetadataPublisher( } catch { case t: Throwable => fatalFaultHandler.handleFault("Error starting TransactionCoordinator", t) } - try { - // Start the share coordinator. - shareCoordinator.startup(() => metadataCache.numPartitions(Topic.SHARE_GROUP_STATE_TOPIC_NAME) - .orElse(config.shareCoordinatorConfig.shareCoordinatorStateTopicNumPartitions())) - } catch { - case t: Throwable => fatalFaultHandler.handleFault("Error starting Share coordinator", t) + if (config.shareGroupConfig.isShareGroupEnabled && shareCoordinator.isDefined) { + try { + // Start the share coordinator. + shareCoordinator.get.startup(() => metadataCache.numPartitions( + Topic.SHARE_GROUP_STATE_TOPIC_NAME).orElse(config.shareCoordinatorConfig.shareCoordinatorStateTopicNumPartitions())) + } catch { + case t: Throwable => fatalFaultHandler.handleFault("Error starting Share coordinator", t) + } } } diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index 0b3f8e4828dc5..a45724dd1e767 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -153,7 +153,6 @@ public class SharePartitionManagerTest { FetchRequest.ORDINARY_CONSUMER_ID, -1, DELAYED_SHARE_FETCH_MAX_WAIT_MS, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty(), true); private static final String TIMER_NAME_PREFIX = "share-partition-manager"; - private static final String CONNECTION_ID = "id-1"; static final int DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL = 1000; @@ -201,12 +200,12 @@ public void testNewContextReturnsFinalContextWithoutRequestData() { List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); assertInstanceOf(ShareSessionContext.class, context1); assertFalse(((ShareSessionContext) context1).isSubsequent()); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(memberId, ShareRequestMetadata.FINAL_EPOCH); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), List.of(), reqMetadata2, true, CONNECTION_ID); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), List.of(), reqMetadata2, true); assertEquals(FinalContext.class, context2.getClass()); } @@ -218,6 +217,7 @@ public void testNewContextReturnsFinalContextWithRequestData() { .build(); Uuid tpId0 = Uuid.randomUuid(); + Uuid tpId1 = Uuid.randomUuid(); TopicIdPartition tp0 = new TopicIdPartition(tpId0, new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(tpId0, new TopicPartition("foo", 1)); @@ -228,15 +228,16 @@ public void testNewContextReturnsFinalContextWithRequestData() { List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); assertInstanceOf(ShareSessionContext.class, context1); assertFalse(((ShareSessionContext) context1).isSubsequent()); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(memberId, ShareRequestMetadata.FINAL_EPOCH); - // Sending a Request with FINAL_EPOCH. This should return a FinalContext. - List reqData2 = List.of(tp0, tp1); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, true, CONNECTION_ID); + // shareFetch is not empty, but the maxBytes of topic partition is 0, which means this is added only for acknowledgements. + // New context should be created successfully + List reqData3 = List.of(new TopicIdPartition(tpId1, new TopicPartition("foo", 0))); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData3, List.of(), reqMetadata2, true); assertEquals(FinalContext.class, context2.getClass()); } @@ -259,16 +260,16 @@ public void testNewContextReturnsFinalContextWhenTopicPartitionsArePresentInRequ List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); assertInstanceOf(ShareSessionContext.class, context1); assertFalse(((ShareSessionContext) context1).isSubsequent()); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(memberId, ShareRequestMetadata.FINAL_EPOCH); // shareFetch is not empty, and it contains tpId1, which should return FinalContext instance since it is FINAL_EPOCH - List reqData2 = List.of(new TopicIdPartition(tpId1, new TopicPartition("foo", 0))); + List reqData3 = List.of(new TopicIdPartition(tpId1, new TopicPartition("foo", 0))); assertInstanceOf(FinalContext.class, - sharePartitionManager.newContext(groupId, reqData2, List.of(), reqMetadata2, true, CONNECTION_ID)); + sharePartitionManager.newContext(groupId, reqData3, List.of(), reqMetadata2, true)); } @Test @@ -294,7 +295,7 @@ public void testNewContext() { List reqData2 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false, CONNECTION_ID); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false); assertInstanceOf(ShareSessionContext.class, context2); assertFalse(((ShareSessionContext) context2).isSubsequent()); @@ -313,16 +314,16 @@ public void testNewContext() { // Test trying to create a new session with an invalid epoch assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, "id-2")); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); // Test trying to create a new session with a non-existent session key Uuid memberId4 = Uuid.randomUuid(); assertThrows(ShareSessionNotFoundException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(memberId4, 1), true, "id-3")); + new ShareRequestMetadata(memberId4, 1), true)); // Continue the first share session we created. ShareFetchContext context5 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true, CONNECTION_ID); + new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context5); assertTrue(((ShareSessionContext) context5).isSubsequent()); @@ -340,18 +341,18 @@ public void testNewContext() { // Test setting an invalid share session epoch. assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, CONNECTION_ID)); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); // Test generating a throttled response for a subsequent share session ShareFetchContext context7 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true, CONNECTION_ID); + new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); ShareFetchResponse resp7 = context7.throttleResponse(100); assertEquals(Errors.NONE, resp7.error()); assertEquals(100, resp7.throttleTimeMs()); // Get the final share session. ShareFetchContext context8 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true, CONNECTION_ID); + new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); assertEquals(FinalContext.class, context8.getClass()); assertEquals(1, cache.size()); @@ -388,7 +389,7 @@ public void testSubsequentShareSession() { String groupId = "grp"; ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); assertInstanceOf(ShareSessionContext.class, context1); LinkedHashMap respData1 = new LinkedHashMap<>(); @@ -404,7 +405,7 @@ public void testSubsequentShareSession() { List removed2 = new ArrayList<>(); removed2.add(tp0); ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, removed2, - new ShareRequestMetadata(reqMetadata1.memberId(), 1), true, CONNECTION_ID); + new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context2); Set expectedTopicIdPartitions2 = new HashSet<>(); @@ -451,7 +452,7 @@ public void testZeroSizeShareSession() { String groupId = "grp"; ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); assertInstanceOf(ShareSessionContext.class, context1); LinkedHashMap respData1 = new LinkedHashMap<>(); @@ -468,7 +469,7 @@ public void testZeroSizeShareSession() { removed2.add(foo0); removed2.add(foo1); ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), removed2, - new ShareRequestMetadata(reqMetadata1.memberId(), 1), true, CONNECTION_ID); + new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context2); LinkedHashMap respData2 = new LinkedHashMap<>(); @@ -494,14 +495,14 @@ public void testToForgetPartitions() { List reqData1 = List.of(foo, bar); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); assertInstanceOf(ShareSessionContext.class, context1); assertPartitionsPresent((ShareSessionContext) context1, List.of(foo, bar)); mockUpdateAndGenerateResponseData(context1, groupId, reqMetadata1.memberId()); ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), List.of(foo), - new ShareRequestMetadata(reqMetadata1.memberId(), 1), true, CONNECTION_ID); + new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); // So foo is removed but not the others. assertPartitionsPresent((ShareSessionContext) context2, List.of(bar)); @@ -509,7 +510,7 @@ public void testToForgetPartitions() { mockUpdateAndGenerateResponseData(context2, groupId, reqMetadata1.memberId()); ShareFetchContext context3 = sharePartitionManager.newContext(groupId, List.of(), List.of(bar), - new ShareRequestMetadata(reqMetadata1.memberId(), 2), true, CONNECTION_ID); + new ShareRequestMetadata(reqMetadata1.memberId(), 2), true); assertPartitionsPresent((ShareSessionContext) context3, List.of()); } @@ -536,7 +537,7 @@ public void testShareSessionUpdateTopicIdsBrokerSide() { List reqData1 = List.of(foo, bar); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); assertInstanceOf(ShareSessionContext.class, context1); assertFalse(((ShareSessionContext) context1).isSubsequent()); @@ -552,7 +553,7 @@ public void testShareSessionUpdateTopicIdsBrokerSide() { // Create a subsequent share fetch request as though no topics changed. ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata1.memberId(), 1), true, CONNECTION_ID); + new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context2); assertTrue(((ShareSessionContext) context2).isSubsequent()); @@ -586,7 +587,7 @@ public void testGetErroneousAndValidTopicIdPartitions() { List reqData2 = List.of(tp0, tp1, tpNull1); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false, CONNECTION_ID); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false); assertInstanceOf(ShareSessionContext.class, context2); assertFalse(((ShareSessionContext) context2).isSubsequent()); assertErroneousAndValidTopicIdPartitions(context2.getErroneousAndValidTopicIdPartitions(), List.of(tpNull1), List.of(tp0, tp1)); @@ -608,15 +609,15 @@ public void testGetErroneousAndValidTopicIdPartitions() { // Test trying to create a new session with an invalid epoch assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, CONNECTION_ID)); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); // Test trying to create a new session with a non-existent session key assertThrows(ShareSessionNotFoundException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(Uuid.randomUuid(), 1), true, CONNECTION_ID)); + new ShareRequestMetadata(Uuid.randomUuid(), 1), true)); // Continue the first share session we created. ShareFetchContext context5 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true, CONNECTION_ID); + new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context5); assertTrue(((ShareSessionContext) context5).isSubsequent()); @@ -627,12 +628,12 @@ public void testGetErroneousAndValidTopicIdPartitions() { // Test setting an invalid share session epoch. assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, CONNECTION_ID)); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); // Test generating a throttled response for a subsequent share session List reqData7 = List.of(tpNull2); ShareFetchContext context7 = sharePartitionManager.newContext(groupId, reqData7, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true, CONNECTION_ID); + new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); // Check for throttled response ShareFetchResponse resp7 = context7.throttleResponse(100); assertEquals(Errors.NONE, resp7.error()); @@ -642,7 +643,7 @@ public void testGetErroneousAndValidTopicIdPartitions() { // Get the final share session. ShareFetchContext context8 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true, CONNECTION_ID); + new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); assertEquals(FinalContext.class, context8.getClass()); assertEquals(1, cache.size()); @@ -687,7 +688,7 @@ public void testShareFetchContextResponseSize() { short version = ApiKeys.SHARE_FETCH.latestVersion(); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false, CONNECTION_ID); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false); assertInstanceOf(ShareSessionContext.class, context2); assertFalse(((ShareSessionContext) context2).isSubsequent()); @@ -707,17 +708,17 @@ public void testShareFetchContextResponseSize() { // Test trying to create a new session with an invalid epoch assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, CONNECTION_ID)); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); // Test trying to create a new session with a non-existent session key Uuid memberId4 = Uuid.randomUuid(); assertThrows(ShareSessionNotFoundException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(memberId4, 1), true, CONNECTION_ID)); + new ShareRequestMetadata(memberId4, 1), true)); // Continue the first share session we created. List reqData5 = List.of(tp2); ShareFetchContext context5 = sharePartitionManager.newContext(groupId, reqData5, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true, CONNECTION_ID); + new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context5); assertTrue(((ShareSessionContext) context5).isSubsequent()); @@ -732,11 +733,11 @@ public void testShareFetchContextResponseSize() { // Test setting an invalid share session epoch. assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, CONNECTION_ID)); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); // Test generating a throttled response for a subsequent share session ShareFetchContext context7 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true, CONNECTION_ID); + new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); int respSize7 = context7.responseSize(respData2, version); ShareFetchResponse resp7 = context7.throttleResponse(100); @@ -747,7 +748,7 @@ public void testShareFetchContextResponseSize() { // Get the final share session. ShareFetchContext context8 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true, CONNECTION_ID); + new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); assertEquals(FinalContext.class, context8.getClass()); assertEquals(1, cache.size()); @@ -793,7 +794,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId1, ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); assertInstanceOf(ShareSessionContext.class, context1); assertFalse(((ShareSessionContext) context1).isSubsequent()); @@ -814,7 +815,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { List reqData2 = List.of(tp2); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(memberId2, ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false, CONNECTION_ID); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false); assertInstanceOf(ShareSessionContext.class, context2); assertFalse(((ShareSessionContext) context2).isSubsequent()); @@ -832,7 +833,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { // Continue the first share session we created. List reqData3 = List.of(tp2); ShareFetchContext context3 = sharePartitionManager.newContext(groupId, reqData3, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey1.memberId(), 1), true, CONNECTION_ID); + new ShareRequestMetadata(shareSessionKey1.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context3); assertTrue(((ShareSessionContext) context3).isSubsequent()); @@ -847,7 +848,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { // Continue the second session we created. List reqData4 = List.of(tp3); ShareFetchContext context4 = sharePartitionManager.newContext(groupId, reqData4, List.of(tp2), - new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true, CONNECTION_ID); + new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); assertInstanceOf(ShareSessionContext.class, context4); assertTrue(((ShareSessionContext) context4).isSubsequent()); @@ -860,7 +861,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { // Get the final share session. ShareFetchContext context5 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata1.memberId(), ShareRequestMetadata.FINAL_EPOCH), true, CONNECTION_ID); + new ShareRequestMetadata(reqMetadata1.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); assertEquals(FinalContext.class, context5.getClass()); LinkedHashMap respData5 = new LinkedHashMap<>(); @@ -875,7 +876,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { // Continue the second share session . ShareFetchContext context6 = sharePartitionManager.newContext(groupId, List.of(), List.of(tp3), - new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true, CONNECTION_ID); + new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); assertInstanceOf(ShareSessionContext.class, context6); assertTrue(((ShareSessionContext) context6).isSubsequent()); diff --git a/core/src/test/java/kafka/server/share/SharePartitionTest.java b/core/src/test/java/kafka/server/share/SharePartitionTest.java index 465bce6de6a70..64781648774d6 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionTest.java @@ -3652,8 +3652,7 @@ public void testReleaseSingleRecordBatch() { assertEquals(0, sharePartition.nextFetchOffset()); assertEquals(1, sharePartition.cachedState().size()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(0L).batchState()); - // Release delivery count. - assertEquals(0, sharePartition.cachedState().get(0L).batchDeliveryCount()); + assertEquals(1, sharePartition.cachedState().get(0L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(0L).offsetState()); } @@ -3670,7 +3669,7 @@ public void testReleaseMultipleRecordBatch() { assertEquals(5, sharePartition.nextFetchOffset()); assertEquals(1, sharePartition.cachedState().size()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(5L).batchState()); - assertEquals(0, sharePartition.cachedState().get(5L).batchDeliveryCount()); + assertEquals(1, sharePartition.cachedState().get(5L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(5L).offsetState()); } @@ -3732,7 +3731,7 @@ public void testReleaseAcknowledgedMultipleSubsetRecordBatch() { assertEquals(5, sharePartition.nextFetchOffset()); // Check cached state. Map expectedOffsetStateMap = new HashMap<>(); - expectedOffsetStateMap.put(5L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(5L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(6L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(5L).offsetState()); @@ -3746,8 +3745,8 @@ public void testReleaseAcknowledgedMultipleSubsetRecordBatch() { expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); } @@ -3793,8 +3792,8 @@ public void testReleaseAcquiredRecordsWithAnotherMember() { expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); // Release acquired records for "member-2". @@ -3816,8 +3815,8 @@ public void testReleaseAcquiredRecordsWithAnotherMember() { expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); } @@ -3863,8 +3862,8 @@ public void testReleaseAcquiredRecordsWithAnotherMemberAndSubsetAcknowledged() { expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); // Ack subset of records by "member-2". @@ -3880,7 +3879,7 @@ public void testReleaseAcquiredRecordsWithAnotherMemberAndSubsetAcknowledged() { // Check cached state. expectedOffsetStateMap.clear(); expectedOffsetStateMap.put(5L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(6L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(6L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(5L).offsetState()); expectedOffsetStateMap.clear(); expectedOffsetStateMap.put(10L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); @@ -3892,8 +3891,8 @@ public void testReleaseAcquiredRecordsWithAnotherMemberAndSubsetAcknowledged() { expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); } @@ -3927,14 +3926,14 @@ public void testReleaseAcquiredRecordsAfterDifferentAcknowledges() { Map expectedOffsetStateMap = new HashMap<>(); expectedOffsetStateMap.put(5L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(6L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(7L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(7L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(8L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(9L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(5L).offsetState()); } @Test - public void testMaxDeliveryCountLimitNotExceededForRecordsSubsetAfterReleaseAcquiredRecords() { + public void testMaxDeliveryCountLimitExceededForRecordsSubsetAfterReleaseAcquiredRecords() { SharePartition sharePartition = SharePartitionBuilder.builder() .withMaxDeliveryCount(2) .withState(SharePartitionState.ACTIVE) @@ -3956,12 +3955,12 @@ public void testMaxDeliveryCountLimitNotExceededForRecordsSubsetAfterReleaseAcqu assertEquals(0, sharePartition.nextFetchOffset()); assertEquals(2, sharePartition.cachedState().size()); - assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(10L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(10L).batchState()); assertNull(sharePartition.cachedState().get(10L).offsetState()); } @Test - public void testMaxDeliveryCountLimitNotExceededForRecordsSubsetAfterReleaseAcquiredRecordsSubset() { + public void testMaxDeliveryCountLimitExceededForRecordsSubsetAfterReleaseAcquiredRecordsSubset() { SharePartition sharePartition = SharePartitionBuilder.builder() .withMaxDeliveryCount(2) .withState(SharePartitionState.ACTIVE) @@ -4000,21 +3999,21 @@ public void testMaxDeliveryCountLimitNotExceededForRecordsSubsetAfterReleaseAcqu assertNotNull(sharePartition.cachedState().get(10L).offsetState()); assertThrows(IllegalStateException.class, () -> sharePartition.cachedState().get(15L).batchState()); assertNotNull(sharePartition.cachedState().get(10L).offsetState()); - assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(20L).batchMemberId()); assertNull(sharePartition.cachedState().get(20L).offsetState()); Map expectedOffsetStateMap = new HashMap<>(); - expectedOffsetStateMap.put(10L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(11L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(12L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(13L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(14L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(10L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(11L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(12L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(13L, new InFlightState(RecordState.ARCHIVED, (short) 2, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(14L, new InFlightState(RecordState.ARCHIVED, (short) 2, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); expectedOffsetStateMap.clear(); - expectedOffsetStateMap.put(15L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(16L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(15L, new InFlightState(RecordState.ARCHIVED, (short) 2, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ARCHIVED, (short) 2, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(19L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); @@ -4051,10 +4050,9 @@ public void testMaxDeliveryCountLimitExceededForRecordsSubsetCacheCleared() { fetchAcquiredRecords(sharePartition, records2, 2); fetchAcquiredRecords(sharePartition, records3, 5); - sharePartition.acknowledge(MEMBER_ID, new ArrayList<>(List.of( - new ShareAcknowledgementBatch(13, 16, List.of((byte) 2)), - new ShareAcknowledgementBatch(20, 24, List.of((byte) 2)) - ))); + CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); + assertNull(releaseResult.join()); + assertFalse(releaseResult.isCompletedExceptionally()); assertEquals(25, sharePartition.nextFetchOffset()); assertEquals(0, sharePartition.cachedState().size()); @@ -4174,7 +4172,7 @@ public void testAcquisitionLockOnReleasingMultipleRecordBatch() { assertEquals(5, sharePartition.nextFetchOffset()); assertEquals(1, sharePartition.cachedState().size()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(5L).batchState()); - assertEquals(0, sharePartition.cachedState().get(5L).batchDeliveryCount()); + assertEquals(1, sharePartition.cachedState().get(5L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(5L).offsetState()); // Acquisition lock timer task would be cancelled by the release acquired records operation. assertNull(sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask()); @@ -4217,7 +4215,7 @@ public void testAcquisitionLockOnReleasingAcknowledgedMultipleSubsetRecordBatchW assertEquals(5, sharePartition.nextFetchOffset()); // Check cached state. Map expectedOffsetStateMap = new HashMap<>(); - expectedOffsetStateMap.put(5L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(5L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(6L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(5L).offsetState()); @@ -4231,8 +4229,8 @@ public void testAcquisitionLockOnReleasingAcknowledgedMultipleSubsetRecordBatchW expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); // Acquisition lock timer task would be cancelled by the release acquired records operation. @@ -4818,7 +4816,7 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovement() { expectedOffsetStateMap.put(21L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(22L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(23L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(24L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(24L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(20L).offsetState()); @@ -4832,8 +4830,8 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovement() { expectedOffsetStateMap.put(35L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(36L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(37L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(38L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(39L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(38L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(39L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(35L).offsetState()); } @@ -4889,57 +4887,11 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovementToMiddleOfBa Map expectedOffsetStateMap = new HashMap<>(); expectedOffsetStateMap.put(10L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(11L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(12L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(13L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(14L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - - assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); - } - - @Test - public void testReleaseAcquiredRecordsDecreaseDeliveryCount() { - SharePartition sharePartition = SharePartitionBuilder.builder().withState(SharePartitionState.ACTIVE).build(); - - fetchAcquiredRecords(sharePartition, memoryRecords(5, 5), 5); - fetchAcquiredRecords(sharePartition, memoryRecords(5, 10), 5); - - sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(12, 13, List.of((byte) 1)))); - - // LSO is at 11. - sharePartition.updateCacheAndOffsets(11); - - assertEquals(15, sharePartition.nextFetchOffset()); - assertEquals(11, sharePartition.startOffset()); - assertEquals(14, sharePartition.endOffset()); - assertEquals(2, sharePartition.cachedState().size()); - - // Before release, the delivery count was incremented. - Map expectedOffsetStateMap = new HashMap<>(); - expectedOffsetStateMap.put(10L, new InFlightState(RecordState.ACQUIRED, (short) 1, MEMBER_ID)); - expectedOffsetStateMap.put(11L, new InFlightState(RecordState.ACQUIRED, (short) 1, MEMBER_ID)); - expectedOffsetStateMap.put(12L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(13L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(14L, new InFlightState(RecordState.ACQUIRED, (short) 1, MEMBER_ID)); - assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); - - // Release acquired records. - CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); - assertNull(releaseResult.join()); - assertFalse(releaseResult.isCompletedExceptionally()); - - // Check delivery count. - assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(5L).batchMemberId()); - assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(5L).batchState()); - assertEquals(1, sharePartition.cachedState().get(5L).batchDeliveryCount()); + expectedOffsetStateMap.put(11L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(12L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(13L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(14L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - // After release, the delivery count was decremented. - expectedOffsetStateMap = new HashMap<>(); - expectedOffsetStateMap.put(10L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(11L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(12L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(13L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(14L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); } @@ -6716,7 +6668,7 @@ public void testAcquireWhenBatchesRemovedForFetchOffsetForSameCachedBatch() { }); }); } - + private String assertionFailedMessage(SharePartition sharePartition, Map> offsets) { StringBuilder errorMessage = new StringBuilder(ACQUISITION_LOCK_NEVER_GOT_RELEASED + String.format( " timer size: %d, next fetch offset: %d\n", diff --git a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala index 54f6d71a27853..dc836352787b2 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala @@ -80,13 +80,11 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { val tp = new TopicPartition(topic, part) val logDir = "logDir" val group = "my-group" - val shareGroup = "share-group" val protocolType = "consumer" val protocolName = "consumer-range" val clusterResource = new ResourcePattern(CLUSTER, Resource.CLUSTER_NAME, LITERAL) val topicResource = new ResourcePattern(TOPIC, topic, LITERAL) val groupResource = new ResourcePattern(GROUP, group, LITERAL) - val shareGroupResource = new ResourcePattern(GROUP, shareGroup, LITERAL) val transactionalIdResource = new ResourcePattern(TRANSACTIONAL_ID, transactionalId, LITERAL) producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "1") diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index d42ba245f6ebd..d75bdc9df6de4 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -37,7 +37,7 @@ import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProt import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderPartition, OffsetForLeaderTopic, OffsetForLeaderTopicCollection} -import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteShareGroupOffsetsRequestData, DeleteShareGroupStateRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeShareGroupOffsetsRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, InitializeShareGroupStateRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, OffsetFetchRequestData, ProduceRequestData, ReadShareGroupStateRequestData, ReadShareGroupStateSummaryRequestData, ShareAcknowledgeRequestData, ShareFetchRequestData, ShareGroupDescribeRequestData, ShareGroupHeartbeatRequestData, SyncGroupRequestData, WriteShareGroupStateRequestData, WriteTxnMarkersRequestData} +import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, OffsetFetchRequestData, ProduceRequestData, SyncGroupRequestData, WriteTxnMarkersRequestData} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord} @@ -48,7 +48,7 @@ import org.apache.kafka.common.resource.ResourceType._ import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.Utils -import org.apache.kafka.common.{ElectionType, IsolationLevel, KafkaException, TopicIdPartition, TopicPartition, Uuid, requests} +import org.apache.kafka.common.{ElectionType, IsolationLevel, KafkaException, TopicPartition, Uuid, requests} import org.apache.kafka.test.{TestUtils => JTestUtils} import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST @@ -73,11 +73,6 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val groupDeleteAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW))) val groupDescribeConfigsAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE_CONFIGS, ALLOW))) val groupAlterConfigsAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER_CONFIGS, ALLOW))) - val shareGroupReadAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW))) - val shareGroupDescribeAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW))) - val shareGroupDeleteAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW))) - val shareGroupDescribeConfigsAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE_CONFIGS, ALLOW))) - val shareGroupAlterConfigsAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER_CONFIGS, ALLOW))) val clusterAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CLUSTER_ACTION, ALLOW))) val clusterCreateAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CREATE, ALLOW))) val clusterAlterAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER, ALLOW))) @@ -204,26 +199,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }), ApiKeys.CONSUMER_GROUP_HEARTBEAT -> ((resp: ConsumerGroupHeartbeatResponse) => Errors.forCode(resp.data.errorCode)), ApiKeys.CONSUMER_GROUP_DESCRIBE -> ((resp: ConsumerGroupDescribeResponse) => - Errors.forCode(resp.data.groups.asScala.find(g => group == g.groupId).head.errorCode)), - ApiKeys.SHARE_GROUP_HEARTBEAT -> ((resp: ShareGroupHeartbeatResponse) => Errors.forCode(resp.data.errorCode)), - ApiKeys.SHARE_GROUP_DESCRIBE -> ((resp: ShareGroupDescribeResponse) => - Errors.forCode(resp.data.groups.asScala.find(g => shareGroup == g.groupId).head.errorCode)), - ApiKeys.SHARE_FETCH -> ((resp: ShareFetchResponse) => Errors.forCode(resp.data.errorCode)), - ApiKeys.SHARE_ACKNOWLEDGE -> ((resp: ShareAcknowledgeResponse) => Errors.forCode(resp.data.errorCode)), - ApiKeys.INITIALIZE_SHARE_GROUP_STATE -> ((resp: InitializeShareGroupStateResponse) => Errors.forCode( - resp.data.results.get(0).partitions.get(0).errorCode)), - ApiKeys.READ_SHARE_GROUP_STATE -> ((resp: ReadShareGroupStateResponse) => Errors.forCode( - resp.data.results.get(0).partitions.get(0).errorCode)), - ApiKeys.WRITE_SHARE_GROUP_STATE -> ((resp: WriteShareGroupStateResponse) => Errors.forCode( - resp.data.results.get(0).partitions.get(0).errorCode)), - ApiKeys.DELETE_SHARE_GROUP_STATE -> ((resp: DeleteShareGroupStateResponse) => Errors.forCode( - resp.data.results.get(0).partitions.get(0).errorCode)), - ApiKeys.READ_SHARE_GROUP_STATE_SUMMARY -> ((resp: ReadShareGroupStateSummaryResponse) => Errors.forCode( - resp.data.results.get(0).partitions.get(0).errorCode)), - ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> ((resp: DescribeShareGroupOffsetsResponse) => Errors.forCode( - resp.data.groups.asScala.find(g => shareGroup == g.groupId).head.errorCode)), - ApiKeys.DELETE_SHARE_GROUP_OFFSETS -> ((resp: DeleteShareGroupOffsetsResponse) => Errors.forCode( - resp.data.errorCode)) + Errors.forCode(resp.data.groups.asScala.find(g => group == g.groupId).head.errorCode)) ) def findErrorForTopicId(id: Uuid, response: AbstractResponse): Errors = { @@ -279,18 +255,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ApiKeys.DESCRIBE_PRODUCERS -> topicReadAcl, ApiKeys.DESCRIBE_TRANSACTIONS -> transactionalIdDescribeAcl, ApiKeys.CONSUMER_GROUP_HEARTBEAT -> groupReadAcl, - ApiKeys.CONSUMER_GROUP_DESCRIBE -> groupDescribeAcl, - ApiKeys.SHARE_GROUP_HEARTBEAT -> (shareGroupReadAcl ++ topicDescribeAcl), - ApiKeys.SHARE_GROUP_DESCRIBE -> (shareGroupDescribeAcl ++ topicDescribeAcl), - ApiKeys.SHARE_FETCH -> (shareGroupReadAcl ++ topicReadAcl), - ApiKeys.SHARE_ACKNOWLEDGE -> (shareGroupReadAcl ++ topicReadAcl), - ApiKeys.INITIALIZE_SHARE_GROUP_STATE -> clusterAcl, - ApiKeys.READ_SHARE_GROUP_STATE -> clusterAcl, - ApiKeys.WRITE_SHARE_GROUP_STATE -> clusterAcl, - ApiKeys.DELETE_SHARE_GROUP_STATE -> clusterAcl, - ApiKeys.READ_SHARE_GROUP_STATE_SUMMARY -> clusterAcl, - ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> (shareGroupDescribeAcl ++ topicDescribeAcl), - ApiKeys.DELETE_SHARE_GROUP_OFFSETS -> (shareGroupDeleteAcl ++ topicReadAcl) + ApiKeys.CONSUMER_GROUP_DESCRIBE -> groupDescribeAcl ) private def createMetadataRequest(allowAutoTopicCreation: Boolean) = { @@ -690,120 +655,6 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { .setGroupIds(List(group).asJava) .setIncludeAuthorizedOperations(false)).build() - private def shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( - new ShareGroupHeartbeatRequestData() - .setGroupId(shareGroup) - .setMemberEpoch(0) - .setSubscribedTopicNames(List(topic).asJava)).build(ApiKeys.SHARE_GROUP_HEARTBEAT.latestVersion) - - - private def shareGroupDescribeRequest = new ShareGroupDescribeRequest.Builder( - new ShareGroupDescribeRequestData() - .setGroupIds(List(shareGroup).asJava) - .setIncludeAuthorizedOperations(false)).build(ApiKeys.SHARE_GROUP_DESCRIBE.latestVersion) - - - private def createShareFetchRequest = { - val metadata: ShareRequestMetadata = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH) - val send: Seq[TopicIdPartition] = Seq( - new TopicIdPartition(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID), new TopicPartition(topic, part))) - val ackMap = new util.HashMap[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] - requests.ShareFetchRequest.Builder.forConsumer(shareGroup, metadata, 100, 0, Int.MaxValue, 500, 500, - send.asJava, Seq.empty.asJava, ackMap).build() - } - - private def shareAcknowledgeRequest = { - val shareAcknowledgeRequestData = new ShareAcknowledgeRequestData() - .setGroupId(shareGroup) - .setMemberId(Uuid.randomUuid().toString) - .setShareSessionEpoch(1) - .setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic() - .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) - .setPartitions(List( - new ShareAcknowledgeRequestData.AcknowledgePartition() - .setPartitionIndex(part) - .setAcknowledgementBatches(List( - new ShareAcknowledgeRequestData.AcknowledgementBatch() - .setFirstOffset(0) - .setLastOffset(1) - .setAcknowledgeTypes(Collections.singletonList(1.toByte)) - ).asJava) - ).asJava) - ).asJava) - - new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData).build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) - } - - private def initializeShareGroupStateRequest = new InitializeShareGroupStateRequest.Builder( - new InitializeShareGroupStateRequestData() - .setGroupId(shareGroup) - .setTopics(List(new InitializeShareGroupStateRequestData.InitializeStateData() - .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) - .setPartitions(List(new InitializeShareGroupStateRequestData.PartitionData() - .setPartition(part) - ).asJava) - ).asJava)).build() - - private def readShareGroupStateRequest = new ReadShareGroupStateRequest.Builder( - new ReadShareGroupStateRequestData() - .setGroupId(shareGroup) - .setTopics(List(new ReadShareGroupStateRequestData.ReadStateData() - .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) - .setPartitions(List(new ReadShareGroupStateRequestData.PartitionData() - .setPartition(part) - .setLeaderEpoch(0) - ).asJava) - ).asJava)).build() - - private def writeShareGroupStateRequest = new WriteShareGroupStateRequest.Builder( - new WriteShareGroupStateRequestData() - .setGroupId(shareGroup) - .setTopics(List(new WriteShareGroupStateRequestData.WriteStateData() - .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) - .setPartitions(List(new WriteShareGroupStateRequestData.PartitionData() - .setPartition(part) - ).asJava) - ).asJava)).build() - - private def deleteShareGroupStateRequest = new DeleteShareGroupStateRequest.Builder( - new DeleteShareGroupStateRequestData() - .setGroupId(shareGroup) - .setTopics(List(new DeleteShareGroupStateRequestData.DeleteStateData() - .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) - .setPartitions(List(new DeleteShareGroupStateRequestData.PartitionData() - .setPartition(part) - ).asJava) - ).asJava)).build() - - private def readShareGroupStateSummaryRequest = new ReadShareGroupStateSummaryRequest.Builder( - new ReadShareGroupStateSummaryRequestData() - .setGroupId(shareGroup) - .setTopics(List(new ReadShareGroupStateSummaryRequestData.ReadStateSummaryData() - .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) - .setPartitions(List(new ReadShareGroupStateSummaryRequestData.PartitionData() - .setPartition(part) - .setLeaderEpoch(0) - ).asJava) - ).asJava)).build(ApiKeys.READ_SHARE_GROUP_STATE_SUMMARY.latestVersion) - - private def describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequest.Builder( - new DescribeShareGroupOffsetsRequestData() - .setGroups(List(new DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup() - .setGroupId(shareGroup) - .setTopics(List(new DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestTopic() - .setTopicName(topic) - .setPartitions(List(Integer.valueOf(part) - ).asJava) - ).asJava) - ).asJava)).build(ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS.latestVersion) - - private def deleteShareGroupOffsetsRequest = new DeleteShareGroupOffsetsRequest.Builder( - new DeleteShareGroupOffsetsRequestData() - .setGroupId(shareGroup) - .setTopics(List(new DeleteShareGroupOffsetsRequestData.DeleteShareGroupOffsetsRequestTopic() - .setTopicName(topic) - ).asJava)).build(ApiKeys.DELETE_SHARE_GROUP_OFFSETS.latestVersion) - private def sendRequests(requestKeyToRequest: mutable.Map[ApiKeys, AbstractRequest], topicExists: Boolean = true, topicNames: Map[Uuid, String] = getTopicNames()) = { for ((key, request) <- requestKeyToRequest) { @@ -818,8 +669,6 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { // In KRaft mode, trying to delete a topic that doesn't exist but that you do have // describe permission for will give UNKNOWN_TOPIC_OR_PARTITION. true - } else if (resourceToAcls.size > 1) { - false } else { describeAcls == acls } @@ -835,7 +684,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } @ParameterizedTest - @ValueSource(strings = Array("kip932")) + @ValueSource(strings = Array("kraft")) def testAuthorizationWithTopicExisting(quorum: String): Unit = { //First create the topic so we have a valid topic ID sendRequests(mutable.Map(ApiKeys.CREATE_TOPICS -> createTopicsRequest)) @@ -874,18 +723,6 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ApiKeys.WRITE_TXN_MARKERS -> writeTxnMarkersRequest, ApiKeys.CONSUMER_GROUP_HEARTBEAT -> consumerGroupHeartbeatRequest, ApiKeys.CONSUMER_GROUP_DESCRIBE -> consumerGroupDescribeRequest, - ApiKeys.SHARE_GROUP_HEARTBEAT -> shareGroupHeartbeatRequest, - ApiKeys.SHARE_GROUP_DESCRIBE -> shareGroupDescribeRequest, - ApiKeys.SHARE_FETCH -> createShareFetchRequest, - ApiKeys.SHARE_ACKNOWLEDGE -> shareAcknowledgeRequest, - ApiKeys.INITIALIZE_SHARE_GROUP_STATE -> initializeShareGroupStateRequest, - ApiKeys.READ_SHARE_GROUP_STATE -> readShareGroupStateRequest, - ApiKeys.WRITE_SHARE_GROUP_STATE -> writeShareGroupStateRequest, - ApiKeys.DELETE_SHARE_GROUP_STATE -> deleteShareGroupStateRequest, - ApiKeys.READ_SHARE_GROUP_STATE_SUMMARY -> readShareGroupStateSummaryRequest, - ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> describeShareGroupOffsetsRequest, - ApiKeys.DELETE_SHARE_GROUP_OFFSETS -> deleteShareGroupOffsetsRequest, - // Delete the topic last ApiKeys.DELETE_TOPICS -> deleteTopicsRequest ) @@ -915,10 +752,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ApiKeys.CREATE_PARTITIONS -> createPartitionsRequest, ApiKeys.DELETE_GROUPS -> deleteGroupsRequest, ApiKeys.OFFSET_FOR_LEADER_EPOCH -> offsetsForLeaderEpochRequest, - ApiKeys.ELECT_LEADERS -> electLeadersRequest, - ApiKeys.SHARE_FETCH -> createShareFetchRequest, - ApiKeys.SHARE_ACKNOWLEDGE -> shareAcknowledgeRequest, - ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> describeShareGroupOffsetsRequest + ApiKeys.ELECT_LEADERS -> electLeadersRequest ) sendRequests(requestKeyToRequest, false, topicNames) @@ -2819,476 +2653,6 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendAndReceiveRegexHeartbeat(member1Response, interBrokerListenerName, Some(0), fullRequest = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupHeartbeatWithGroupReadAndTopicDescribeAcl(quorum: String): Unit = { - addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) - addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) - - val request = shareGroupHeartbeatRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupHeartbeatWithOperationAll(quorum: String): Unit = { - val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) - addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) - addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) - - val request = shareGroupHeartbeatRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupHeartbeatWithoutGroupReadOrTopicDescribeAcl(quorum: String): Unit = { - removeAllClientAcls() - - val request = shareGroupHeartbeatRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupHeartbeatWithoutGroupReadAcl(quorum: String): Unit = { - addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) - - val request = shareGroupHeartbeatRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupHeartbeatWithoutTopicDescribeAcl(quorum: String): Unit = { - addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) - - val request = shareGroupHeartbeatRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - private def createShareGroupToDescribe(): Unit = { - createTopicWithBrokerPrincipal(topic) - addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), shareGroupResource) - addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) - shareConsumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, shareGroup) - val consumer = createShareConsumer() - consumer.subscribe(Collections.singleton(topic)) - consumer.poll(Duration.ofMillis(500L)) - removeAllClientAcls() - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupDescribeWithGroupDescribeAndTopicDescribeAcl(quorum: String): Unit = { - createShareGroupToDescribe() - addAndVerifyAcls(shareGroupDescribeAcl(shareGroupResource), shareGroupResource) - addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) - - val request = shareGroupDescribeRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupDescribeWithOperationAll(quorum: String): Unit = { - createShareGroupToDescribe() - - val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) - addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) - addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) - - val request = shareGroupDescribeRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupDescribeWithoutGroupDescribeAcl(quorum: String): Unit = { - createShareGroupToDescribe() - addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) - - val request = shareGroupDescribeRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupDescribeWithoutGroupDescribeOrTopicDescribeAcl(quorum: String): Unit = { - createShareGroupToDescribe() - - val request = shareGroupDescribeRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareFetchWithGroupReadAndTopicReadAcl(quorum: String): Unit = { - addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) - addAndVerifyAcls(topicReadAcl(topicResource), topicResource) - - val request = createShareFetchRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareFetchWithOperationAll(quorum: String): Unit = { - val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) - addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) - addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) - - val request = createShareFetchRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareFetchWithoutGroupReadOrTopicReadAcl(quorum: String): Unit = { - removeAllClientAcls() - - val request = createShareFetchRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareFetchWithoutGroupReadAcl(quorum: String): Unit = { - addAndVerifyAcls(topicReadAcl(topicResource), topicResource) - - val request = createShareFetchRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareFetchWithoutTopicReadAcl(quorum: String): Unit = { - createTopicWithBrokerPrincipal(topic) - addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) - - val request = createShareFetchRequest - val response = connectAndReceive[ShareFetchResponse](request, listenerName = listenerName) - assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED, Errors.forCode(response.data.responses.get(0).partitions.get(0).errorCode)) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareAcknowledgeWithGroupReadAndTopicReadAcl(quorum: String): Unit = { - addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) - addAndVerifyAcls(topicReadAcl(topicResource), topicResource) - - val request = shareAcknowledgeRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareAcknowledgeWithOperationAll(quorum: String): Unit = { - val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) - addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) - addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) - - val request = shareAcknowledgeRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareAcknowledgeWithoutGroupReadOrTopicReadAcl(quorum: String): Unit = { - removeAllClientAcls() - - val request = shareAcknowledgeRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareAcknowledgeFetchWithoutGroupReadAcl(quorum: String): Unit = { - addAndVerifyAcls(topicReadAcl(topicResource), topicResource) - - val request = shareAcknowledgeRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testInitializeShareGroupStateWithClusterAcl(quorum: String): Unit = { - addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) - - val request = initializeShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testInitializeShareGroupStateWithOperationAll(quorum: String): Unit = { - val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) - addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) - - val request = initializeShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testInitializeShareGroupStateWithoutClusterAcl(quorum: String): Unit = { - removeAllClientAcls() - - val request = initializeShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateWithClusterAcl(quorum: String): Unit = { - addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) - - val request = readShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateWithOperationAll(quorum: String): Unit = { - val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) - addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) - - val request = readShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateWithoutClusterAcl(quorum: String): Unit = { - removeAllClientAcls() - - val request = readShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testWriteShareGroupStateWithClusterAcl(quorum: String): Unit = { - addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) - - val request = writeShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testWriteShareGroupStateWithOperationAll(quorum: String): Unit = { - val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) - addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) - - val request = writeShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testWriteShareGroupStateWithoutClusterAcl(quorum: String): Unit = { - removeAllClientAcls() - - val request = writeShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupStateWithClusterAcl(quorum: String): Unit = { - addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) - - val request = deleteShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupStateWithOperationAll(quorum: String): Unit = { - val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) - addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) - - val request = deleteShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupStateWithoutClusterAcl(quorum: String): Unit = { - removeAllClientAcls() - - val request = deleteShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateSummaryWithClusterAcl(quorum: String): Unit = { - addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) - - val request = readShareGroupStateSummaryRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateSummaryWithOperationAll(quorum: String): Unit = { - val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) - addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) - - val request = readShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateSummaryWithoutClusterAcl(quorum: String): Unit = { - removeAllClientAcls() - - val request = readShareGroupStateRequest - val resource = Set[ResourceType](CLUSTER) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDescribeShareGroupOffsetsWithGroupDescribeAndTopicDescribeAcl(quorum: String): Unit = { - addAndVerifyAcls(shareGroupDescribeAcl(shareGroupResource), shareGroupResource) - addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) - - val request = describeShareGroupOffsetsRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDescribeShareGroupOffsetsWithOperationAll(quorum: String): Unit = { - val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) - addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) - addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) - - val request = describeShareGroupOffsetsRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDescribeShareGroupOffsetsWithoutGroupDescribeOrTopicDescribeAcl(quorum: String): Unit = { - removeAllClientAcls() - - val request = describeShareGroupOffsetsRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDescribeShareGroupOffsetsWithoutGroupDescribeAcl(quorum: String): Unit = { - addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) - - val request = describeShareGroupOffsetsRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDescribeShareGroupOffsetsWithoutTopicDescribeAcl(quorum: String): Unit = { - addAndVerifyAcls(shareGroupDescribeAcl(shareGroupResource), shareGroupResource) - - val request = describeShareGroupOffsetsRequest - val response = connectAndReceive[DescribeShareGroupOffsetsResponse](request, listenerName = listenerName) - assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED, Errors.forCode(response.data.groups.get(0).topics.get(0).partitions.get(0).errorCode)) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupOffsetsWithGroupDeleteAndTopicReadAcl(quorum: String): Unit = { - addAndVerifyAcls(shareGroupDeleteAcl(shareGroupResource), shareGroupResource) - addAndVerifyAcls(topicReadAcl(topicResource), topicResource) - - val request = deleteShareGroupOffsetsRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupOffsetsWithOperationAll(quorum: String): Unit = { - val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) - addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) - addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) - - val request = deleteShareGroupOffsetsRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupOffsetsWithoutGroupDeleteOrTopicReadAcl(quorum: String): Unit = { - removeAllClientAcls() - - val request = deleteShareGroupOffsetsRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupOffsetsWithoutGroupDeleteAcl(quorum: String): Unit = { - addAndVerifyAcls(topicReadAcl(topicResource), topicResource) - - val request = deleteShareGroupOffsetsRequest - val resource = Set[ResourceType](GROUP, TOPIC) - sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) - } - - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupOffsetsWithoutTopicReadAcl(quorum: String): Unit = { - addAndVerifyAcls(shareGroupDeleteAcl(shareGroupResource), shareGroupResource) - - val request = deleteShareGroupOffsetsRequest - val response = connectAndReceive[DeleteShareGroupOffsetsResponse](request, listenerName = listenerName) - assertEquals(1, response.data.responses.size) - assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, response.data.responses.get(0).errorCode, s"Unexpected response $response") - } - private def sendAndReceiveFirstRegexHeartbeat(memberId: String, listenerName: ListenerName): ConsumerGroupHeartbeatResponseData = { val request = new ConsumerGroupHeartbeatRequest.Builder( diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index e062dcc09fa08..e35db63e81270 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -30,6 +30,7 @@ import kafka.security.JaasTestUtils import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} import org.apache.kafka.common.network.{ConnectionMode, ListenerName} import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, Serializer} +import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.MetadataLogConfig import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs} @@ -70,6 +71,9 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties, logDirCount = logDirCount) configureListeners(cfgs) modifyConfigs(cfgs) + if (isShareGroupTest()) { + cfgs.foreach(_.setProperty(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer,share")) + } cfgs.foreach(_.setProperty(MetadataLogConfig.METADATA_LOG_DIR_CONFIG, TestUtils.tempDir().getAbsolutePath)) insertControllerListenersIfNeeded(cfgs) cfgs.map(KafkaConfig.fromProps) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index ea0d7494a83b1..0ab2328c53f79 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -87,10 +87,6 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - if (testInfo.getTestMethod.get.getName == "testDeleteRecordsAfterCorruptRecords") { - this.serverConfig.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1100") - this.controllerConfig.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1100") - } super.setUp(testInfo) Configurator.reconfigure(); brokerLoggerConfigResource = new ConfigResource( @@ -1572,6 +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") createTopic(topic, numPartitions = 1, replicationFactor = 1, config) client = createAdminClient @@ -1579,8 +1576,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val producer = createProducer() def sendRecords(begin: Int, end: Int) = { val futures = (begin until end).map( i => { - val data : Array[Byte] = Array.fill(25)(i.toByte) - val record = new ProducerRecord(topic, partition, data, data) + val record = new ProducerRecord(topic, partition, s"$i".getBytes, s"$i".getBytes) producer.send(record) }) futures.foreach(_.get) diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index c4f2508d0ca24..66fca0db9badc 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -567,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 -> "1048576").asJava + val configsOverride = Map(TopicConfig.SEGMENT_BYTES_CONFIG -> "100000").asJava 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)) @@ -581,7 +581,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu val topicConfigs = result.config(topic1).get().entries.asScala assertTrue(topicConfigs.nonEmpty) val segmentBytesConfig = topicConfigs.find(_.name == TopicConfig.SEGMENT_BYTES_CONFIG).get - assertEquals(1048576, segmentBytesConfig.value.toLong) + assertEquals(100000, segmentBytesConfig.value.toLong) assertEquals(ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, segmentBytesConfig.source) val compressionConfig = topicConfigs.find(_.name == TopicConfig.COMPRESSION_TYPE_CONFIG).get assertEquals(ServerLogConfigs.COMPRESSION_TYPE_DEFAULT, compressionConfig.value) diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 35cef2cfed40b..1d3b3493cce8e 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -20,7 +20,7 @@ import kafka.server.{KafkaConfig, KafkaRaftServer} import kafka.utils.TestUtils import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.errors.CorruptRecordException -import org.apache.kafka.common.errors.RecordTooLargeException +import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException} import org.apache.kafka.common.protocol import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable} import org.apache.kafka.common.record.ArbitraryMemoryRecords @@ -40,10 +40,10 @@ 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.ArgumentsSource + import net.jqwik.api.AfterFailureMode import net.jqwik.api.ForAll import net.jqwik.api.Property -import org.apache.kafka.common.config.ConfigException import java.io.File import java.nio.ByteBuffer @@ -78,13 +78,13 @@ final class KafkaMetadataLogTest { props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240)) props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024)) - assertThrows(classOf[ConfigException], () => { + assertThrows(classOf[InvalidConfigurationException], () => { val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) }) - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10 * 1024 * 1024)) + props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) @@ -689,6 +689,7 @@ final class KafkaMetadataLogTest { val recordSize = 64 val config = new MetadataLogConfig( DefaultMetadataLogConfig.logSegmentBytes, + DefaultMetadataLogConfig.logSegmentMinBytes, DefaultMetadataLogConfig.logSegmentMillis, DefaultMetadataLogConfig.retentionMaxBytes, DefaultMetadataLogConfig.retentionMillis, @@ -907,7 +908,8 @@ final class KafkaMetadataLogTest { @Test def testAdvanceLogStartOffsetAfterCleaning(): Unit = { val config = new MetadataLogConfig( - 1024 * 1024, + 512, + 512, 10 * 1000, 256, 60 * 1000, @@ -943,7 +945,8 @@ final class KafkaMetadataLogTest { def testDeleteSnapshots(): Unit = { // Generate some logs and a few snapshots, set retention low and verify that cleaning occurs val config = new MetadataLogConfig( - 1024 * 1024, + 1024, + 1024, 10 * 1000, 1024, 60 * 1000, @@ -976,7 +979,8 @@ final class KafkaMetadataLogTest { def testSoftRetentionLimit(): Unit = { // Set retention equal to the segment size and generate slightly more than one segment of logs val config = new MetadataLogConfig( - 1024 * 1024, + 10240, + 10240, 10 * 1000, 10240, 60 * 1000, @@ -1019,7 +1023,8 @@ final class KafkaMetadataLogTest { @Test def testSegmentsLessThanLatestSnapshot(): Unit = { val config = new MetadataLogConfig( - 1024 * 1024, + 10240, + 10240, 10 * 1000, 10240, 60 * 1000, @@ -1027,7 +1032,7 @@ final class KafkaMetadataLogTest { DefaultMetadataLogConfig.maxFetchSizeInBytes, DefaultMetadataLogConfig.deleteDelayMillis ) - val log = buildMetadataLog(tempDir, mockTime, config, 10240) + val log = buildMetadataLog(tempDir, mockTime, config) // Generate enough data to cause a segment roll for (_ <- 0 to 2000) { @@ -1077,7 +1082,8 @@ object KafkaMetadataLogTest { } val DefaultMetadataLogConfig = new MetadataLogConfig( - 1024 * 1024, + 100 * 1024, + 100 * 1024, 10 * 1000, 100 * 1024, 60 * 1000, @@ -1089,8 +1095,7 @@ object KafkaMetadataLogTest { def buildMetadataLogAndDir( tempDir: File, time: MockTime, - metadataLogConfig: MetadataLogConfig = DefaultMetadataLogConfig, - internalSegmentSize: Integer = null, + metadataLogConfig: MetadataLogConfig = DefaultMetadataLogConfig ): (Path, KafkaMetadataLog, MetadataLogConfig) = { val logDir = createLogDirectory( @@ -1105,8 +1110,7 @@ object KafkaMetadataLogTest { time, time.scheduler, metadataLogConfig, - 1, - internalSegmentSize + 1 ) (logDir.toPath, metadataLog, metadataLogConfig) @@ -1116,9 +1120,8 @@ object KafkaMetadataLogTest { tempDir: File, time: MockTime, metadataLogConfig: MetadataLogConfig = DefaultMetadataLogConfig, - internalSegmentSize: Integer = null, ): KafkaMetadataLog = { - val (_, log, _) = buildMetadataLogAndDir(tempDir, time, metadataLogConfig, internalSegmentSize) + val (_, log, _) = buildMetadataLogAndDir(tempDir, time, metadataLogConfig) log } diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index 65d50434e0efa..d9eaa7b2aacc0 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.requests.LeaderAndIsrRequest import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.{MetadataCache, MockConfigRepository} import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpoints import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} @@ -69,7 +69,7 @@ class AbstractPartitionTest { TestUtils.clearYammerMetrics() val logProps = createLogProperties(Map.empty) - logConfig = new LogConfig(logProps, 1024) + logConfig = new LogConfig(logProps) configRepository = MockConfigRepository.forTopic(topicPartition.topic, logProps) tmpDir = TestUtils.tempDir() @@ -100,7 +100,7 @@ class AbstractPartitionTest { def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 512: java.lang.Integer) + logProps.put(TopicConfig.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) } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 390d457ab4583..4a3051ddc9567 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -32,7 +32,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.{LeaderAndIsr, MetadataCache, MockConfigRepository} import org.apache.kafka.server.common.{RequestLocal, TopicIdPartition} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpoints @@ -362,7 +362,7 @@ class PartitionLockTest extends Logging { private def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 512: java.lang.Integer) + logProps.put(TopicConfig.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) } diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index e6820b713a269..acf21e69ec377 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -67,9 +67,11 @@ abstract class AbstractLogCleanerIntegrationTest { minCleanableDirtyRatio: Float = defaultMinCleanableDirtyRatio, minCompactionLagMs: Long = defaultMinCompactionLagMS, deleteDelay: Int = defaultDeleteDelay, + segmentSize: Int = defaultSegmentSize, 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(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) @@ -87,7 +89,7 @@ abstract class AbstractLogCleanerIntegrationTest { maxMessageSize: Int = defaultMaxMessageSize, minCompactionLagMs: Long = defaultMinCompactionLagMS, deleteDelay: Int = defaultDeleteDelay, - internalSegmentSize: Int = defaultSegmentSize, + segmentSize: Int = defaultSegmentSize, maxCompactionLagMs: Long = defaultMaxCompactionLagMs, cleanerIoBufferSize: Option[Int] = None, propertyOverrides: Properties = new Properties()): LogCleaner = { @@ -102,7 +104,8 @@ abstract class AbstractLogCleanerIntegrationTest { minCleanableDirtyRatio = minCleanableDirtyRatio, minCompactionLagMs = minCompactionLagMs, deleteDelay = deleteDelay, - maxCompactionLagMs = maxCompactionLagMs), internalSegmentSize) + segmentSize = segmentSize, + maxCompactionLagMs = maxCompactionLagMs)) val log = UnifiedLog.create( dir, logConfig, diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index d519c4d11a3f6..38aa789729a9a 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -137,7 +137,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest { cleaner = makeCleaner(partitions = topicPartitions, backoffMs = cleanerBackOffMs, minCompactionLagMs = minCompactionLagMs, - internalSegmentSize = segmentSize, + segmentSize = segmentSize, maxCompactionLagMs= maxCompactionLagMs, minCleanableDirtyRatio = minCleanableDirtyRatio) val log = cleaner.logs.get(topicPartitions(0)) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala index d6ed10d8e6bac..c632f2c0bf1dd 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala @@ -52,7 +52,7 @@ class LogCleanerLagIntegrationTest extends AbstractLogCleanerIntegrationTest wit cleaner = makeCleaner(partitions = topicPartitions, backoffMs = cleanerBackOffMs, minCompactionLagMs = minCompactionLag, - internalSegmentSize = segmentSize) + segmentSize = segmentSize) val log = cleaner.logs.get(topicPartitions(0)) // t = T0 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 01ad7f34abf1d..f93d703f07777 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -27,7 +27,6 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.LogCleaningState.{LOG_CLEANING_ABORTED, LOG_CLEANING_IN_PROGRESS} import org.apache.kafka.storage.internals.log.{AppendOrigin, LocalLog, LogCleanerManager, LogCleaningException, LogCleaningState, LogConfig, LogDirFailureChannel, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, PreCleanStats, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} @@ -54,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(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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) @@ -371,7 +370,7 @@ class LogCleanerManagerTest extends Logging { // change cleanup policy from delete to compact val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, log.config.segmentSize: Integer) + logProps.put(TopicConfig.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) @@ -549,7 +548,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsForNone(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -571,7 +570,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsActiveSegment(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -593,7 +592,7 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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)) @@ -626,7 +625,7 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForShortTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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)) @@ -668,7 +667,7 @@ class LogCleanerManagerTest extends Logging { def testUndecidedTransactionalDataNotCleanable(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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)) @@ -712,7 +711,7 @@ class LogCleanerManagerTest extends Logging { @Test def testDoneCleaning(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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) @@ -831,7 +830,7 @@ class LogCleanerManagerTest extends Logging { private def createLowRetentionLogConfig(segmentSize: Int, cleanupPolicy: String): LogConfig = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, segmentSize: Integer) + logProps.put(TopicConfig.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 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala index 5c8a48b50f391..e0d3ac5601d23 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala @@ -156,7 +156,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati val log = cleaner.logs.get(topicPartitions(0)) val props = logConfigProperties(maxMessageSize = maxMessageSize) props.put(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.name) - val logConfig = new LogConfig(props, 2048) + val logConfig = new LogConfig(props) log.updateConfig(logConfig) val appends1 = writeDups(numKeys = 100, numDups = 3, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V0) @@ -198,10 +198,10 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati def testCleaningNestedMessagesWithV0V1(compressionType: CompressionType): Unit = { val compression = Compression.of(compressionType).build() val maxMessageSize = 192 - cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = maxMessageSize, internalSegmentSize = 256) + cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = maxMessageSize, segmentSize = 256) val log = cleaner.logs.get(topicPartitions(0)) - val logConfig = new LogConfig(logConfigProperties(maxMessageSize = maxMessageSize), 256) + val logConfig = new LogConfig(logConfigProperties(maxMessageSize = maxMessageSize, segmentSize = 256)) log.updateConfig(logConfig) // with compression enabled, these messages will be written as a single message containing all the individual messages diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index c36e6ee10c126..09a9d1c40f140 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -26,7 +26,6 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanedTransactionMetadata, Cleaner, CleanerConfig, CleanerStats, LocalLog, LogAppendInfo, LogCleaner, LogCleanerManager, LogCleaningAbortedException, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetMap, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} @@ -57,9 +56,10 @@ class LogCleanerTest extends Logging { val tmpdir = TestUtils.tempDir() val dir = TestUtils.randomPartitionLogDir(tmpdir) val logProps = new Properties() + logProps.put(TopicConfig.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 = new LogConfig(logProps, 1024) + val logConfig = new LogConfig(logProps) val time = new MockTime() val throttler = new Throttler(Double.MaxValue, Long.MaxValue, "throttler", "entries", time) val tombstoneRetentionMs = 86400000 @@ -148,8 +148,9 @@ class LogCleanerTest extends Logging { def testCleanSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) - val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps, 1024)) + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append messages to the log until we have four segments while (log.numberOfSegments < 4) @@ -180,7 +181,7 @@ class LogCleanerTest extends Logging { // Construct a log instance. The replaceSegments() method of the log instance is overridden so that // it waits for another thread to execute deleteOldSegments() val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) @@ -270,7 +271,7 @@ class LogCleanerTest extends Logging { val originalMaxFileSize = 1024 val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact": java.lang.String) logProps.put(TopicConfig.PREALLOCATE_CONFIG, "true": java.lang.String) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -294,7 +295,7 @@ class LogCleanerTest extends Logging { def testDuplicateCheckAfterCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) var log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -463,7 +464,7 @@ class LogCleanerTest extends Logging { def testBasicTransactionAwareCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -496,7 +497,7 @@ class LogCleanerTest extends Logging { def testCleanWithTransactionsSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -542,7 +543,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -590,7 +591,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(capacity = Int.MaxValue, maxMessageSize = 100) val logProps = new Properties() logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, 100: java.lang.Integer) - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -618,7 +619,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -681,7 +682,7 @@ class LogCleanerTest extends Logging { def testCleanEmptyControlBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -713,7 +714,7 @@ class LogCleanerTest extends Logging { def testCommittedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -735,7 +736,7 @@ class LogCleanerTest extends Logging { def testAbortedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -764,7 +765,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -798,7 +799,7 @@ class LogCleanerTest extends Logging { val producerId = 1L val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, 0, AppendOrigin.REPLICATION) @@ -831,7 +832,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -895,7 +896,7 @@ class LogCleanerTest extends Logging { // Create cleaner with very small default max message size val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -967,7 +968,7 @@ class LogCleanerTest extends Logging { def createLogWithMessagesLargerThanMaxSize(largeMessageSize: Int): (UnifiedLog, FakeOffsetMap) = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -993,8 +994,9 @@ class LogCleanerTest extends Logging { def testCleaningWithDeletes(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) - val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps, 1024)) + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // append messages with the keys 0 through N while (log.numberOfSegments < 2) @@ -1019,7 +1021,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 3 messages in the map val cleaner = makeCleaner(4) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1048,7 +1050,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsProducerLastSequence(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) log.appendAsLeader(record(0, 0), 0) // offset 0 @@ -1071,7 +1073,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsLastSequenceEvenIfTransactionAborted(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -1105,7 +1107,7 @@ class LogCleanerTest extends Logging { def testCleaningWithKeysConflictingWithTxnMarkerKeys(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val leaderEpoch = 5 val producerEpoch = 0.toShort @@ -1149,7 +1151,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 1 message in the map val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1180,7 +1182,7 @@ class LogCleanerTest extends Logging { def testCleaningWithUncleanableSection(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1223,7 +1225,7 @@ class LogCleanerTest extends Logging { def testLogToClean(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1241,7 +1243,8 @@ class LogCleanerTest extends Logging { def testLogToCleanWithUncleanableSection(): Unit = { // create a log with small segment size val logProps = new Properties() - val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps, 100)) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment def createRecords = TestUtils.singletonRecords(value = Array.fill[Byte](25)(0), key = 1.toString.getBytes) @@ -1273,7 +1276,7 @@ class LogCleanerTest extends Logging { // create a log with compaction turned off so we can append unkeyed messages val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1331,7 +1334,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsWithAbort(): Unit = { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1353,7 +1356,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsRetainingLastEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1385,7 +1388,7 @@ class LogCleanerTest extends Logging { def testSegmentGrouping(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1486,7 +1489,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1538,7 +1541,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) //mimic the effect of loading an empty index file logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 400: java.lang.Integer) @@ -1663,10 +1666,11 @@ class LogCleanerTest extends Logging { def testRecoveryAfterCrash(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 10: java.lang.Integer) - val config = LogConfig.fromProps(logConfig.originals, logProps, 300) + val config = LogConfig.fromProps(logConfig.originals, logProps) // create a log and append some messages var log = makeLog(config = config) @@ -1793,7 +1797,7 @@ class LogCleanerTest extends Logging { def testBuildOffsetMapFakeLarge(): Unit = { val map = new FakeOffsetMap(1000) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) @@ -1941,7 +1945,7 @@ class LogCleanerTest extends Logging { @Test def testCleaningBeyondMissingOffsets(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) val cleaner = makeCleaner(Int.MaxValue) diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 16a02a274dd2e..0da8366f443ee 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -20,10 +20,10 @@ package kafka.log import java.util.{Optional, Properties} import java.util.concurrent.{Callable, Executors} import kafka.utils.TestUtils +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.KafkaScheduler import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, UnifiedLog} @@ -60,7 +60,7 @@ class LogConcurrencyTest { @Test def testUncommittedDataNotConsumedFrequentSegmentRolls(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 237: Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 237: Integer) val logConfig = new LogConfig(logProps) testUncommittedDataNotConsumed(createLog(logConfig)) } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 673d651848c6f..8e417a695ee36 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -21,6 +21,7 @@ import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.compress.Compression +import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, SimpleRecord, TimestampType} import org.apache.kafka.common.utils.{Time, Utils} @@ -244,7 +245,8 @@ class LogLoaderTest { @Test def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = { val logProps = new Properties() - val logConfig = new LogConfig(logProps, 640) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "640") + val logConfig = new LogConfig(logProps) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 9be056bc37896..67880e0ced54c 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -43,7 +43,7 @@ import java.util.{Collections, Optional, OptionalLong, Properties} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.{FileLock, KafkaScheduler, MockTime, Scheduler} -import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog, LogManager => JLogManager} +import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogManager => JLogManager, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} import org.apache.kafka.storage.internals.checkpoint.{CleanShutdownFileHandler, OffsetCheckpointFile} import org.apache.kafka.storage.log.metrics.BrokerTopicStats import org.junit.jupiter.api.function.Executable @@ -60,9 +60,10 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 60 * 1000 val logProps = new Properties() + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) - val logConfig = new LogConfig(logProps, 1024) + val logConfig = new LogConfig(logProps) var logDir: File = _ var logManager: LogManager = _ val name = "kafka" @@ -390,17 +391,11 @@ class LogManagerTest { logManager.shutdown() val segmentBytes = 10 * setSize val properties = new Properties() + properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes.toString) properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString) val configRepository = MockConfigRepository.forTopic(name, properties) - logManager = TestUtils.createLogManager( - defaultConfig = new LogConfig(logProps, segmentBytes), - configRepository = configRepository, - logDirs = Seq(this.logDir), - time = this.time, - recoveryThreadsPerDataDir = 1, - initialTaskDelayMs = initialTaskDelayMs - ) + logManager = createLogManager(configRepository = configRepository) logManager.startup(Set.empty) // create a log @@ -454,14 +449,7 @@ class LogManagerTest { logManager.shutdown() val configRepository = MockConfigRepository.forTopic(name, TopicConfig.CLEANUP_POLICY_CONFIG, policy) - logManager = TestUtils.createLogManager( - defaultConfig = new LogConfig(logProps, 1024), - configRepository = configRepository, - logDirs = Seq(this.logDir), - time = this.time, - recoveryThreadsPerDataDir = 1, - initialTaskDelayMs = initialTaskDelayMs - ) + logManager = createLogManager(configRepository = configRepository) val log = logManager.getOrCreateLog(new TopicPartition(name, 0), topicId = Optional.empty) var offset = 0L for (_ <- 0 until 200) { diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index 77d5d8bd17cb3..04c91741037bb 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -72,11 +72,10 @@ object LogTestUtils { fileDeleteDelayMs: Long = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, remoteLogStorageEnable: Boolean = LogConfig.DEFAULT_REMOTE_STORAGE_ENABLE, remoteLogCopyDisable: Boolean = DEFAULT_REMOTE_LOG_COPY_DISABLE_CONFIG, - remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, - internalSegmentBytes : Integer = null): LogConfig = { + remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs: java.lang.Long) logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) @@ -90,7 +89,7 @@ object LogTestUtils { logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, remoteLogStorageEnable: java.lang.Boolean) logProps.put(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, remoteLogCopyDisable: java.lang.Boolean) logProps.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, remoteLogDeleteOnDisable: java.lang.Boolean) - new LogConfig(logProps, internalSegmentBytes) + new LogConfig(logProps) } def createLog(dir: File, diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 7ee2b8f887b31..3f73f8f731a71 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -52,10 +52,11 @@ import org.junit.jupiter.params.provider.{EnumSource, ValueSource} import org.mockito.ArgumentMatchers import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{doAnswer, doThrow, spy} + import net.jqwik.api.AfterFailureMode import net.jqwik.api.ForAll import net.jqwik.api.Property -import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.KRaftConfigs import java.io._ import java.nio.ByteBuffer @@ -682,7 +683,7 @@ class UnifiedLogTest { val msgPerSeg = 10 val segmentSize = msgPerSeg * setSize // each segment will be 10 messages // create a log - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentSize) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize) val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments, "There should be exactly 1 segment.") @@ -709,7 +710,7 @@ class UnifiedLogTest { def testInitializationOfProducerSnapshotsUpgradePath(): Unit = { // simulate the upgrade path by creating a new log with several segments, deleting the // snapshot files, and then reloading the log - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = 64 * 10) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 64 * 10) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) @@ -1123,7 +1124,7 @@ class UnifiedLogTest { @Test def testProducerIdMapTruncateFullyAndStartAt(): Unit = { val records = TestUtils.singletonRecords("foo".getBytes) - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) val log = createLog(logDir, logConfig) log.appendAsLeader(records, 0) log.takeProducerSnapshot() @@ -1146,7 +1147,7 @@ class UnifiedLogTest { def testProducerIdExpirationOnSegmentDeletion(): Unit = { val pid1 = 1L val records = TestUtils.records(Seq(new SimpleRecord("foo".getBytes)), producerId = pid1, producerEpoch = 0, sequence = 0) - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = records.sizeInBytes, retentionBytes = records.sizeInBytes * 2) val log = createLog(logDir, logConfig) log.appendAsLeader(records, 0) log.takeProducerSnapshot() @@ -1199,7 +1200,7 @@ class UnifiedLogTest { @Test def testProducerSnapshotAfterSegmentRollOnAppend(): Unit = { val producerId = 1L - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = 1024) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024) val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(Seq(new SimpleRecord(mockTime.milliseconds(), new Array[Byte](512))), @@ -1588,7 +1589,7 @@ class UnifiedLogTest { val msgPerSeg = 10 val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages // create a log - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentSize) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize) val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments, "There should be exactly 1 segment.") @@ -1889,7 +1890,7 @@ class UnifiedLogTest { val messageSet = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes)) // append messages to log val configSegmentSize = messageSet.sizeInBytes - 1 - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = configSegmentSize) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = configSegmentSize) val log = createLog(logDir, logConfig) assertThrows(classOf[RecordBatchTooLargeException], () => log.appendAsLeader(messageSet, 0)) @@ -2478,7 +2479,7 @@ class UnifiedLogTest { val setSize = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds).sizeInBytes val msgPerSeg = 10 val segmentSize = msgPerSeg * setSize // each segment will be 10 messages - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentSize, indexIntervalBytes = setSize - 1) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = setSize - 1) val log = createLog(logDir, logConfig) assertEquals(1, log.numberOfSegments, "There should be exactly 1 segment.") @@ -2733,7 +2734,7 @@ class UnifiedLogTest { @Test def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1000") + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000") logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") val logConfig = new LogConfig(logProps) @@ -2893,7 +2894,7 @@ class UnifiedLogTest { @Test def testLogDeletionAfterDeleteRecords(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5) val log = createLog(logDir, logConfig) for (_ <- 0 until 15) @@ -2923,7 +2924,7 @@ class UnifiedLogTest { @Test def shouldDeleteSizeBasedSegments(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2938,7 +2939,7 @@ class UnifiedLogTest { @Test def shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 15) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 15) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -2968,7 +2969,7 @@ class UnifiedLogTest { @Test def shouldNotDeleteTimeBasedSegmentsWhenNoneReadyToBeDeleted(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, timestamp = mockTime.milliseconds) - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000000) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionMs = 10000000) val log = createLog(logDir, logConfig) // append some messages to create some segments @@ -3018,7 +3019,7 @@ class UnifiedLogTest { def shouldDeleteStartOffsetBreachedSegmentsWhenPolicyDoesNotIncludeDelete(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes, timestamp = 10L) val recordsPerSegment = 5 - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * recordsPerSegment, retentionMs = 10000, cleanupPolicy = "compact") + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * recordsPerSegment, retentionMs = 10000, cleanupPolicy = "compact") val log = createLog(logDir, logConfig, brokerTopicStats) // append some messages to create some segments @@ -3090,7 +3091,7 @@ class UnifiedLogTest { @Test def shouldTruncateLeaderEpochsWhenDeletingSegments(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) val cache = epochCache(log) @@ -3115,7 +3116,7 @@ class UnifiedLogTest { @Test def shouldUpdateOffsetForLeaderEpochsWhenDeletingSegments(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes) - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, retentionBytes = createRecords.sizeInBytes * 10) val log = createLog(logDir, logConfig) val cache = epochCache(log) @@ -3144,7 +3145,7 @@ class UnifiedLogTest { baseOffset = startOffset, partitionLeaderEpoch = epoch) } - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = 10 * createRecords(0, 0).sizeInBytes) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 10 * createRecords(0, 0).sizeInBytes) val log = createLog(logDir, logConfig) val cache = epochCache(log) @@ -3668,7 +3669,7 @@ class UnifiedLogTest { new SimpleRecord("b".getBytes), new SimpleRecord("c".getBytes)) - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = records.sizeInBytes) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = records.sizeInBytes) val log = createLog(logDir, logConfig) val firstAppendInfo = log.appendAsLeader(records, 0) @@ -3709,7 +3710,7 @@ class UnifiedLogTest { @Test def testSegmentDeletionWithHighWatermarkInitialization(): Unit = { val logConfig = LogTestUtils.createLogConfig( - internalSegmentBytes = 512, + segmentBytes = 512, segmentIndexBytes = 1000, retentionMs = 999 ) @@ -3733,7 +3734,7 @@ class UnifiedLogTest { @Test def testCannotDeleteSegmentsAtOrAboveHighWatermark(): Unit = { val logConfig = LogTestUtils.createLogConfig( - internalSegmentBytes = 512, + segmentBytes = 512, segmentIndexBytes = 1000, retentionMs = 999 ) @@ -4404,7 +4405,7 @@ class UnifiedLogTest { def testRetentionOnLocalLogDeletionWhenRemoteLogCopyEnabledAndDefaultLocalRetentionBytes(): Unit = { def createRecords = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes))) val segmentBytes = createRecords.sizeInBytes() - val retentionBytesConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentBytes, retentionBytes = 1, + val retentionBytesConfig = LogTestUtils.createLogConfig(segmentBytes = segmentBytes, retentionBytes = 1, fileDeleteDelayMs = 0, remoteLogStorageEnable = true) val log = createLog(logDir, retentionBytesConfig, remoteStorageSystemEnable = true) @@ -4427,7 +4428,7 @@ class UnifiedLogTest { def testRetentionOnLocalLogDeletionWhenRemoteLogCopyEnabledAndDefaultLocalRetentionMs(): Unit = { def createRecords = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes))) val segmentBytes = createRecords.sizeInBytes() - val retentionBytesConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentBytes, retentionMs = 1000, + val retentionBytesConfig = LogTestUtils.createLogConfig(segmentBytes = segmentBytes, retentionMs = 1000, fileDeleteDelayMs = 0, remoteLogStorageEnable = true) val log = createLog(logDir, retentionBytesConfig, remoteStorageSystemEnable = true) @@ -4452,7 +4453,7 @@ class UnifiedLogTest { def testRetentionOnLocalLogDeletionWhenRemoteLogCopyDisabled(): Unit = { def createRecords = TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes))) val segmentBytes = createRecords.sizeInBytes() - val logConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentBytes, localRetentionBytes = 1, retentionBytes = segmentBytes * 5, + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentBytes, localRetentionBytes = 1, retentionBytes = segmentBytes * 5, fileDeleteDelayMs = 0, remoteLogStorageEnable = true) val log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) @@ -4479,7 +4480,7 @@ class UnifiedLogTest { assertEquals(2, log.localLogStartOffset()) // add remoteCopyDisabled = true - val copyDisabledLogConfig = LogTestUtils.createLogConfig(internalSegmentBytes = segmentBytes, localRetentionBytes = 1, retentionBytes = segmentBytes * 5, + val copyDisabledLogConfig = LogTestUtils.createLogConfig(segmentBytes = segmentBytes, localRetentionBytes = 1, retentionBytes = segmentBytes * 5, fileDeleteDelayMs = 0, remoteLogStorageEnable = true, remoteLogCopyDisable = true) log.updateConfig(copyDisabledLogConfig) diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index 3e5461da4934e..8e2698b0842cf 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -27,7 +27,6 @@ import org.apache.kafka.common.requests.{AlterReplicaLogDirsRequest, AlterReplic import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.storage.internals.log.LogFileUtils import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -39,24 +38,10 @@ import scala.util.Random class AlterReplicaLogDirsRequestTest extends BaseRequestTest { override val logDirCount = 5 override val brokerCount = 1 - var testInfo : TestInfo = _ - val topic = "topic" - @BeforeEach - override def setUp(testInfo: TestInfo): Unit = { - this.testInfo = testInfo - if (testInfo.getTestMethod.get.getName == "testAlterReplicaLogDirsRequestWithRetention") { - this.serverConfig.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1024") - this.controllerConfig.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1024") - } - super.setUp(testInfo) - } + val topic = "topic" override def brokerPropertyOverrides(properties: Properties): Unit = { - if (testInfo != null && testInfo.getTestMethod.get.getName == "testAlterReplicaLogDirsRequestWithRetention") { - properties.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1024") - properties.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "1024") - } properties.put(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, "0") properties.put(ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG, "1000") } @@ -163,6 +148,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { // We don't want files with `.deleted` suffix are removed too fast, // so we can validate there will be orphan files and orphan files will be removed eventually. topicProperties.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, "10000") + topicProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1024") createTopic(topic, partitionNum, 1, topicProperties) assertEquals(logDir1, brokers.head.logManager.getLog(tp).get.dir.getParent) diff --git a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala index 2d4c4455ad63d..45b470f402cf8 100644 --- a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala @@ -114,7 +114,7 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - shareCoordinator) + Some(shareCoordinator)) val topicsCollection = new CreateTopicsRequestData.CreatableTopicCollection topicsCollection.add(getNewTopic(topicName, numPartitions, replicationFactor)) @@ -217,7 +217,7 @@ class AutoTopicCreationManagerTest { @Test def testCreateStreamsInternalTopics(): Unit = { val topicConfig = new CreatableTopicConfigCollection() - topicConfig.add(new CreatableTopicConfig().setName("cleanup.policy").setValue("compact")) + topicConfig.add(new CreatableTopicConfig().setName("cleanup.policy").setValue("compact")); val topics = Map( "stream-topic-1" -> new CreatableTopic().setName("stream-topic-1").setNumPartitions(3).setReplicationFactor(2).setConfigs(topicConfig), @@ -230,7 +230,7 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - shareCoordinator) + Some(shareCoordinator)) autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) @@ -251,7 +251,7 @@ class AutoTopicCreationManagerTest { .build(ApiKeys.CREATE_TOPICS.latestVersion()) val forwardedRequestBuffer = capturedRequest.requestData().duplicate() - assertEquals(requestHeader, RequestHeader.parse(forwardedRequestBuffer)) + assertEquals(requestHeader, RequestHeader.parse(forwardedRequestBuffer)); assertEquals(requestBody.data(), CreateTopicsRequest.parse(new ByteBufferAccessor(forwardedRequestBuffer), ApiKeys.CREATE_TOPICS.latestVersion()).data()) } @@ -266,7 +266,7 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - shareCoordinator) + Some(shareCoordinator)) autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) @@ -287,9 +287,9 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - shareCoordinator) + Some(shareCoordinator)) - autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext); val argumentCaptor = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]) Mockito.verify(brokerToController).sendRequest( @@ -307,7 +307,7 @@ class AutoTopicCreationManagerTest { .setTimeoutMs(requestTimeout)) .build(ApiKeys.CREATE_TOPICS.latestVersion()) val forwardedRequestBuffer = capturedRequest.requestData().duplicate() - assertEquals(requestHeader, RequestHeader.parse(forwardedRequestBuffer)) + assertEquals(requestHeader, RequestHeader.parse(forwardedRequestBuffer)); assertEquals(requestBody.data(), CreateTopicsRequest.parse(new ByteBufferAccessor(forwardedRequestBuffer), ApiKeys.CREATE_TOPICS.latestVersion()).data()) } @@ -324,9 +324,9 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - shareCoordinator) + Some(shareCoordinator)) - autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext); val argumentCaptor = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]) Mockito.verify(brokerToController).sendRequest( @@ -355,7 +355,7 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - shareCoordinator) + Some(shareCoordinator)) val createTopicApiVersion = new ApiVersionsResponseData.ApiVersion() .setApiKey(ApiKeys.CREATE_TOPICS.id) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index c20cd042fc5fb..5f87b20d1f579 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.internals.Plugin import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.raft.QuorumConfig +import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.server.DynamicThreadPool import org.apache.kafka.server.authorizer._ @@ -670,6 +670,16 @@ class DynamicBrokerConfigTest { assertTrue(m.currentReporters.isEmpty) } + @Test + def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = { + val props = TestUtils.createBrokerConfig(0, port = 8181) + props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, "1024") + val config = new KafkaConfig(props) + assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) + config.updateCurrentConfig(new KafkaConfig(props)) + assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) + } + @Test def testDynamicLogLocalRetentionMsConfig(): Unit = { val props = TestUtils.createBrokerConfig(0, port = 8181) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 16a4d3c4358f4..28c8d694f9d51 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -99,7 +99,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { @ValueSource(strings = Array("kraft")) def testDynamicTopicConfigChange(quorum: String): Unit = { val tp = new TopicPartition("test", 0) - val oldSegmentSize = 2 * 1024 * 1024 + val oldSegmentSize = 1000 val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) createTopic(tp.topic, 1, 1, logProps) @@ -109,7 +109,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { assertEquals(oldSegmentSize, logOpt.get.config.segmentSize) } - val newSegmentSize = 4 * 1024 * 1024 + val newSegmentSize = 2000 val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) diff --git a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala index e7926df3e36cf..df939c29ffb45 100644 --- a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala @@ -33,11 +33,9 @@ import org.apache.kafka.common.utils.ProducerIdAndEpoch import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT import org.junit.jupiter.api.Assertions.{assertEquals, fail} -import java.net.Socket import java.util.{Comparator, Properties} import java.util.stream.Collectors import scala.collection.Seq -import scala.collection.mutable.ListBuffer import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag @@ -48,8 +46,6 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { protected var producer: KafkaProducer[String, String] = _ - protected var openSockets: ListBuffer[Socket] = ListBuffer[Socket]() - protected def createOffsetsTopic(): Unit = { val admin = cluster.admin() try { @@ -144,14 +140,6 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { keySerializer = new StringSerializer, valueSerializer = new StringSerializer) } - protected def closeSockets(): Unit = { - while (openSockets.nonEmpty) { - val socket = openSockets.head - socket.close() - openSockets.remove(0) - } - } - protected def closeProducer(): Unit = { if(producer != null) producer.close() @@ -934,23 +922,6 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { ) } - protected def connectAndReceiveWithoutClosingSocket[T <: AbstractResponse]( - request: AbstractRequest, - destination: Int - )(implicit classTag: ClassTag[T]): T = { - val socket = IntegrationTestUtils.connect(brokerSocketServer(destination), cluster.clientListener()) - openSockets += socket - IntegrationTestUtils.sendAndReceive[T](request, socket) - } - - protected def connectAndReceiveWithoutClosingSocket[T <: AbstractResponse]( - request: AbstractRequest - )(implicit classTag: ClassTag[T]): T = { - val socket = IntegrationTestUtils.connect(cluster.anyBrokerSocketServer(), cluster.clientListener()) - openSockets += socket - IntegrationTestUtils.sendAndReceive[T](request, socket) - } - private def brokerSocketServer(brokerId: Int): SocketServer = { getBrokers.find { broker => broker.config.brokerId == brokerId diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 18c1d7b3b6cbc..0d311435a2d82 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -88,7 +88,7 @@ import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.{ClientMetricsManager, SimpleApiVersionManager} import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authorizer} -import org.apache.kafka.server.common.{FeatureVersion, FinalizedFeatures, GroupVersion, KRaftVersion, MetadataVersion, RequestLocal, ShareVersion, StreamsVersion, TransactionVersion} +import org.apache.kafka.server.common.{FeatureVersion, FinalizedFeatures, GroupVersion, KRaftVersion, MetadataVersion, RequestLocal, StreamsVersion, TransactionVersion} import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs} import org.apache.kafka.server.metrics.ClientMetricsTestUtils import org.apache.kafka.server.share.{CachedSharePartition, ErroneousAndValidPartitionData, SharePartitionKey} @@ -192,7 +192,7 @@ class KafkaApisTest extends Logging { replicaManager = replicaManager, groupCoordinator = groupCoordinator, txnCoordinator = txnCoordinator, - shareCoordinator = shareCoordinator, + shareCoordinator = Some(shareCoordinator), autoTopicCreationManager = autoTopicCreationManager, brokerId = brokerId, config = config, @@ -225,28 +225,6 @@ class KafkaApisTest extends Logging { } } - def initializeMetadataCacheWithShareGroupsEnabled(enableShareGroups: Boolean = true): MetadataCache = { - val cache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) - val delta = new MetadataDelta(MetadataImage.EMPTY); - delta.replay(new FeatureLevelRecord() - .setName(MetadataVersion.FEATURE_NAME) - .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) - ) - if (enableShareGroups) { - delta.replay(new FeatureLevelRecord() - .setName(ShareVersion.FEATURE_NAME) - .setFeatureLevel(ShareVersion.SV_1.featureLevel()) - ) - } else { - delta.replay(new FeatureLevelRecord() - .setName(ShareVersion.FEATURE_NAME) - .setFeatureLevel(ShareVersion.SV_0.featureLevel()) - ) - } - cache.setImage(delta.apply(MetadataProvenance.EMPTY)) - cache - } - @Test def testDescribeConfigsWithAuthorizer(): Unit = { val authorizer: Authorizer = mock(classOf[Authorizer]) @@ -4164,7 +4142,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4188,7 +4166,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, shareSessionEpoch), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName))) ) @@ -4208,7 +4186,10 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -4228,7 +4209,7 @@ class KafkaApisTest extends Logging { def testHandleShareFetchRequestInvalidRequestOnInitialEpoch(): Unit = { val topicName = "foo" val topicId = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4257,11 +4238,10 @@ class KafkaApisTest extends Logging { cachedSharePartitions.mustAdd(new CachedSharePartition( new TopicIdPartition(topicId, partitionIndex, topicName), false)) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenThrow( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenThrow( Errors.INVALID_REQUEST.exception() - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2 - ))) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4286,7 +4266,10 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -4328,7 +4311,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4352,7 +4335,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) @@ -4382,7 +4365,10 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -4429,7 +4415,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4437,7 +4423,7 @@ class KafkaApisTest extends Logging { FutureUtils.failedFuture[util.Map[TopicIdPartition, ShareFetchResponseData.PartitionData]](Errors.UNKNOWN_SERVER_ERROR.exception()) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) @@ -4458,7 +4444,10 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -4471,7 +4460,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4502,10 +4491,9 @@ class KafkaApisTest extends Logging { cachedSharePartitions.mustAdd(new CachedSharePartition( new TopicIdPartition(topicId, partitionIndex, topicName), false)) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())) - .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) - ) + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())) + .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4530,7 +4518,10 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -4543,7 +4534,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4561,10 +4552,9 @@ class KafkaApisTest extends Logging { cachedSharePartitions.mustAdd(new CachedSharePartition( new TopicIdPartition(topicId, partitionIndex, topicName), false)) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())) - .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) - ) + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())) + .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4589,7 +4579,10 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -4602,7 +4595,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4618,7 +4611,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) @@ -4639,7 +4632,10 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -4660,7 +4656,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4682,7 +4678,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) @@ -4704,7 +4700,10 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -4746,7 +4745,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4768,7 +4767,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) @@ -4790,7 +4789,10 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -4829,7 +4831,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -4903,14 +4905,14 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId, partitionIndex, topicName), false) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 3)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 3)) ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( @@ -4930,7 +4932,10 @@ class KafkaApisTest extends Logging { var request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -5039,7 +5044,7 @@ class KafkaApisTest extends Logging { val topicName4 = "foo4" val topicId4 = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 2, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) addTopicToMetadataCache(topicName3, 1, topicId = topicId3) @@ -5165,17 +5170,17 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId4, 0, topicName4), false )) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId1, new TopicPartition(topicName1, 0)), new TopicIdPartition(topicId1, new TopicPartition(topicName1, 1)), new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)), new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions1, 2)) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions2, 3)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions1, 2)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions2, 3)) ).thenReturn(new FinalContext()) when(sharePartitionManager.releaseSession(any(), any())).thenReturn( @@ -5249,7 +5254,10 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -5499,7 +5507,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -5587,7 +5595,10 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = kafkaApis.handleFetchFromShareFetchRequest( request, @@ -5647,7 +5658,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -5723,7 +5734,10 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = kafkaApis.handleFetchFromShareFetchRequest( request, @@ -5777,7 +5791,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -5848,7 +5862,10 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = kafkaApis.handleFetchFromShareFetchRequest( request, @@ -5910,7 +5927,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid() val topicId3 = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) // topicName3 is not in the metadataCache. @@ -5994,7 +6011,10 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = kafkaApis.handleFetchFromShareFetchRequest( request, @@ -6086,7 +6106,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -6130,13 +6150,12 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId, 0, topicName), false )) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) - ) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -6162,7 +6181,10 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -6219,20 +6241,7 @@ class KafkaApisTest extends Logging { val memberId: Uuid = Uuid.randomUuid() val groupId = "group" - metadataCache = { - val cache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) - val delta = new MetadataDelta(MetadataImage.EMPTY); - delta.replay(new FeatureLevelRecord() - .setName(MetadataVersion.FEATURE_NAME) - .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) - ) - delta.replay(new FeatureLevelRecord() - .setName(ShareVersion.FEATURE_NAME) - .setFeatureLevel(ShareVersion.SV_0.featureLevel()) - ) - cache.setImage(delta.apply(MetadataProvenance.EMPTY)) - cache - } + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) val shareFetchRequestData = new ShareFetchRequestData(). setGroupId(groupId). @@ -6255,7 +6264,10 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "false"), + ) kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) @@ -6269,7 +6281,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6303,6 +6315,8 @@ class KafkaApisTest extends Logging { .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Option(authorizer), ) kafkaApis.handleShareFetchRequest(request) @@ -6318,7 +6332,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6333,7 +6347,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( new FinalContext() ) @@ -6363,7 +6377,10 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -6385,7 +6402,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -6426,7 +6443,10 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) val responseData = response.data() @@ -6446,20 +6466,7 @@ class KafkaApisTest extends Logging { val memberId: Uuid = Uuid.randomUuid() val groupId = "group" - metadataCache = { - val cache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) - val delta = new MetadataDelta(MetadataImage.EMPTY); - delta.replay(new FeatureLevelRecord() - .setName(MetadataVersion.FEATURE_NAME) - .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) - ) - delta.replay(new FeatureLevelRecord() - .setName(ShareVersion.FEATURE_NAME) - .setFeatureLevel(ShareVersion.SV_0.featureLevel()) - ) - cache.setImage(delta.apply(MetadataProvenance.EMPTY)) - cache - } + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) val shareAcknowledgeRequestData = new ShareAcknowledgeRequestData() .setGroupId(groupId) @@ -6482,7 +6489,10 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData).build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "false"), + ) kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) @@ -6496,7 +6506,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6530,6 +6540,8 @@ class KafkaApisTest extends Logging { .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Option(authorizer), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6545,7 +6557,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6578,7 +6590,10 @@ class KafkaApisTest extends Logging { .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) @@ -6592,7 +6607,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6625,7 +6640,10 @@ class KafkaApisTest extends Logging { .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) @@ -6639,7 +6657,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val groupId: String = "group" val memberId: Uuid = Uuid.ZERO_UUID @@ -6670,7 +6688,10 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) @@ -6692,7 +6713,7 @@ class KafkaApisTest extends Logging { val partitionIndex = 0 val topicIdPartition = new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) val topicPartition = topicIdPartition.topicPartition - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicPartition.topic, numPartitions = 1, numBrokers = 3, topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6741,7 +6762,10 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) @@ -6764,7 +6788,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -6800,7 +6824,10 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) val responseData = response.data() @@ -6813,7 +6840,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -6863,7 +6890,10 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) val responseData = response.data() @@ -6882,7 +6912,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -6927,7 +6957,10 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) val responseData = response.data() @@ -6954,7 +6987,7 @@ class KafkaApisTest extends Logging { def testGetAcknowledgeBatchesFromShareFetchRequest(): Unit = { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) val shareFetchRequestData = new ShareFetchRequestData(). setGroupId("group"). setMemberId(Uuid.randomUuid().toString). @@ -7005,7 +7038,10 @@ class KafkaApisTest extends Logging { topicNames.put(topicId2, "foo2") val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareFetchRequest(shareFetchRequest, topicNames, erroneous) assertEquals(4, acknowledgeBatches.size) @@ -7023,7 +7059,7 @@ class KafkaApisTest extends Logging { def testGetAcknowledgeBatchesFromShareFetchRequestError(): Unit = { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) val shareFetchRequestData = new ShareFetchRequestData(). setGroupId("group"). setMemberId(Uuid.randomUuid().toString). @@ -7067,7 +7103,10 @@ class KafkaApisTest extends Logging { topicIdNames.put(topicId1, "foo1") // topicId2 is not present in topicIdNames val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareFetchRequest(shareFetchRequest, topicIdNames, erroneous) val erroneousTopicIdPartitions = kafkaApis.validateAcknowledgementBatches(acknowledgeBatches, erroneous) @@ -7086,7 +7125,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) val shareAcknowledgeRequestData = new ShareAcknowledgeRequestData(). setGroupId("group"). setMemberId(Uuid.randomUuid().toString). @@ -7136,7 +7175,10 @@ class KafkaApisTest extends Logging { topicNames.put(topicId2, "foo2") val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareAcknowledgeRequest(shareAcknowledgeRequest, topicNames, erroneous) assertEquals(3, acknowledgeBatches.size) @@ -7154,7 +7196,7 @@ class KafkaApisTest extends Logging { def testGetAcknowledgeBatchesFromShareAcknowledgeRequestError(): Unit = { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) val shareAcknowledgeRequestData = new ShareAcknowledgeRequestData(). setGroupId("group"). setMemberId(Uuid.randomUuid().toString). @@ -7199,7 +7241,10 @@ class KafkaApisTest extends Logging { topicIdNames.put(topicId1, "foo1") // topicId2 not present in topicIdNames val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareAcknowledgeRequest(shareAcknowledgeRequest, topicIdNames, erroneous) val erroneousTopicIdPartitions = kafkaApis.validateAcknowledgementBatches(acknowledgeBatches, erroneous) @@ -7226,7 +7271,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid() val memberId = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -7267,7 +7312,10 @@ class KafkaApisTest extends Logging { val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val ackResult = kafkaApis.handleAcknowledgements( acknowledgementData, erroneous, @@ -7301,7 +7349,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid() val memberId = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -7342,7 +7390,10 @@ class KafkaApisTest extends Logging { val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val ackResult = kafkaApis.handleAcknowledgements( acknowledgementData, erroneous, @@ -7376,7 +7427,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid() val memberId = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) // Topic with id topicId1 is not present in Metadata Cache addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -7418,7 +7469,10 @@ class KafkaApisTest extends Logging { val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val ackResult = kafkaApis.handleAcknowledgements( acknowledgementData, erroneous, @@ -7452,7 +7506,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid() val memberId = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -7488,7 +7542,10 @@ class KafkaApisTest extends Logging { erroneous += (tp3 -> ShareAcknowledgeResponse.partitionResponse(tp3, Errors.UNKNOWN_TOPIC_ID)) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val ackResult = kafkaApis.handleAcknowledgements( acknowledgementData, erroneous, @@ -7520,7 +7577,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) val responseAcknowledgeData: mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData] = mutable.Map() responseAcknowledgeData += (new TopicIdPartition(topicId1, new TopicPartition("foo", 0)) -> new ShareAcknowledgeResponseData.PartitionData().setPartitionIndex(0).setErrorCode(Errors.NONE.code)) @@ -7582,7 +7639,9 @@ class KafkaApisTest extends Logging { .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( - ) + overrideProperties = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) val response = kafkaApis.processShareAcknowledgeResponse(responseAcknowledgeData, request) val responseData = response.data() val topicResponses = responseData.responses() @@ -10978,10 +11037,8 @@ class KafkaApisTest extends Logging { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequestData().setGroupId("group") val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest).build()) - metadataCache = mock(classOf[KRaftMetadataCache]) - kafkaApis = createKafkaApis( - featureVersions = Seq(ShareVersion.SV_0) - ) + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val expectedHeartbeatResponse = new ShareGroupHeartbeatResponseData() @@ -11001,8 +11058,10 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, shareGroupHeartbeatRequest )).thenReturn(future) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() - kafkaApis = createKafkaApis() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val shareGroupHeartbeatResponse = new ShareGroupHeartbeatResponseData() @@ -11022,8 +11081,9 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11034,7 +11094,7 @@ class KafkaApisTest extends Logging { @Test def testShareGroupHeartbeatRequestTopicAuthorizationFailed(): Unit = { - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) val groupId = "group" val fooTopicName = "foo" val barTopicName = "bar" @@ -11063,6 +11123,7 @@ class KafkaApisTest extends Logging { } kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11082,8 +11143,10 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, shareGroupHeartbeatRequest )).thenReturn(future) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() - kafkaApis = createKafkaApis() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) future.completeExceptionally(Errors.FENCED_MEMBER_EPOCH.exception) @@ -11122,7 +11185,8 @@ class KafkaApisTest extends Logging { new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(1)).setMembers(List(member1).asJava), new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(2)).setMembers(List(member2).asJava) ).asJava - getShareGroupDescribeResponse(groupIds, enableShareGroups = true, verifyNoErr = true, null, describedGroups) + getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") + , true, null, describedGroups) } @Test @@ -11132,7 +11196,7 @@ class KafkaApisTest extends Logging { new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(0)), new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(1)) ).asJava - val response = getShareGroupDescribeResponse(groupIds, enableShareGroups = false, verifyNoErr = false, null, describedGroups) + val response = getShareGroupDescribeResponse(groupIds, Map.empty, false, null, describedGroups) assertNotNull(response.data) assertEquals(2, response.data.groups.size) response.data.groups.forEach(group => assertEquals(Errors.UNSUPPORTED_VERSION.code(), group.errorCode())) @@ -11145,7 +11209,8 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava) - val response = getShareGroupDescribeResponse(groupIds, enableShareGroups = true, verifyNoErr = false, authorizer, describedGroups) + val response = getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") + , false, authorizer, describedGroups) assertNotNull(response.data) assertEquals(2, response.data.groups.size) response.data.groups.forEach(group => assertEquals(Errors.GROUP_AUTHORIZATION_FAILED.code(), group.errorCode())) @@ -11162,7 +11227,8 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) - val response = getShareGroupDescribeResponse(groupIds, enableShareGroups = true, verifyNoErr = false, authorizer, describedGroups) + val response = getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") + , false, authorizer, describedGroups) assertNotNull(response.data) assertEquals(2, response.data.groups.size) @@ -11225,7 +11291,8 @@ class KafkaApisTest extends Logging { .setGroupId(groupIds.get(2)) .setMembers(List(member2).asJava)).asJava - val response = getShareGroupDescribeResponse(groupIds, enableShareGroups = true, verifyNoErr = false, authorizer, describedGroups) + val response = getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") + , false, authorizer, describedGroups) assertNotNull(response.data) assertEquals(3, response.data.groups.size) @@ -11271,9 +11338,13 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava + val config = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", + ) + val response = getReadShareGroupStateResponse( readRequestData, - ShareCoordinatorTestConfig.testConfigMap().asScala, + config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, null, readStateResultData @@ -11322,9 +11393,13 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) + val config = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", + ) + val response = getReadShareGroupStateResponse( readRequestData, - ShareCoordinatorTestConfig.testConfigMap().asScala, + config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, authorizer, readStateResultData @@ -11366,9 +11441,13 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava + val config = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", + ) + val response = getReadShareGroupStateSummaryResponse( readSummaryRequestData, - ShareCoordinatorTestConfig.testConfigMap().asScala, + config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, null, readStateSummaryResultData @@ -11410,9 +11489,13 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) + val config = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", + ) + val response = getReadShareGroupStateSummaryResponse( readSummaryRequestData, - ShareCoordinatorTestConfig.testConfigMap().asScala, + config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, authorizer, readStateSummaryResultData @@ -11456,8 +11539,9 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(util.List.of(AuthorizationResult.DENIED)) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11483,8 +11567,9 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(util.List.of(AuthorizationResult.DENIED)) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11507,7 +11592,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid val topicName3 = "topic-3" val topicId3 = Uuid.randomUuid - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 1, topicId = topicId2) addTopicToMetadataCache(topicName3, 1, topicId = topicId3) @@ -11540,7 +11625,9 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, describeShareGroupOffsetsRequestGroup2 )).thenReturn(futureGroup2) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val describeShareGroupOffsetsResponseGroup1 = new DescribeShareGroupOffsetsResponseGroup() @@ -11621,7 +11708,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid val topicName3 = "topic-3" val topicId3 = Uuid.randomUuid - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 1, topicId = topicId2) addTopicToMetadataCache(topicName3, 1, topicId = topicId3) @@ -11682,6 +11769,7 @@ class KafkaApisTest extends Logging { }.asJava } kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer) ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11798,7 +11886,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid val topicName3 = "topic-3" val topicId3 = Uuid.randomUuid - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 1, topicId = topicId2) addTopicToMetadataCache(topicName3, 1, topicId = topicId3) @@ -11843,6 +11931,7 @@ class KafkaApisTest extends Logging { }.asJava } kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer) ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11959,7 +12048,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid val topicName3 = "topic-3" val topicId3 = Uuid.randomUuid - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 1, topicId = topicId2) addTopicToMetadataCache(topicName3, 1, topicId = topicId3) @@ -11983,7 +12072,9 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, describeShareGroupOffsetsRequestGroup2 )).thenReturn(futureGroup2) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val describeShareGroupOffsetsResponseGroup1 = new DescribeShareGroupOffsetsResponseGroup() @@ -12058,14 +12149,16 @@ class KafkaApisTest extends Logging { @Test def testDescribeShareGroupOffsetsRequestEmptyGroupsSuccess(): Unit = { - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val future = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val describeShareGroupOffsetsResponseGroup = new DescribeShareGroupOffsetsResponseGroup() @@ -12079,7 +12172,7 @@ class KafkaApisTest extends Logging { @Test def testDescribeShareGroupOffsetsRequestEmptyTopicsSuccess(): Unit = { - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) val describeShareGroupOffsetsRequestGroup = new DescribeShareGroupOffsetsRequestGroup().setGroupId("group") @@ -12092,7 +12185,9 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, describeShareGroupOffsetsRequestGroup )).thenReturn(future) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val describeShareGroupOffsetsResponseGroup = new DescribeShareGroupOffsetsResponseGroup() @@ -12132,8 +12227,9 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(util.List.of(AuthorizationResult.DENIED)) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -12163,7 +12259,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid val topicName2 = "topic-2" val topicId2 = Uuid.randomUuid - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 2, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -12198,6 +12294,7 @@ class KafkaApisTest extends Logging { .thenReturn(util.List.of(AuthorizationResult.ALLOWED)) kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer) ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -12245,7 +12342,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid val topicName3 = "topic-3" val topicId3 = Uuid.randomUuid - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) addTopicToMetadataCache(topicName3, 3, topicId = topicId3) @@ -12271,7 +12368,9 @@ class KafkaApisTest extends Logging { deleteShareGroupOffsetsRequestData )).thenReturn(resultFuture) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val deleteShareGroupOffsetsResponseData = new DeleteShareGroupOffsetsResponseData() @@ -12306,7 +12405,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid val topicName2 = "topic-2" val topicId2 = Uuid.randomUuid - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -12327,7 +12426,9 @@ class KafkaApisTest extends Logging { deleteShareGroupOffsetsRequestData )).thenReturn(CompletableFuture.failedFuture(Errors.UNKNOWN_SERVER_ERROR.exception)) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val deleteShareGroupOffsetsResponseData = new DeleteShareGroupOffsetsResponseData() @@ -12344,7 +12445,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid val topicName2 = "topic-2" val topicId2 = Uuid.randomUuid - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -12369,7 +12470,9 @@ class KafkaApisTest extends Logging { deleteShareGroupOffsetsRequestData )).thenReturn(CompletableFuture.completedFuture(groupCoordinatorResponse)) - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val deleteShareGroupOffsetsResponseData = new DeleteShareGroupOffsetsResponseData() @@ -12382,7 +12485,7 @@ class KafkaApisTest extends Logging { @Test def testDeleteShareGroupOffsetsRequestEmptyTopicsSuccess(): Unit = { - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) val deleteShareGroupOffsetsRequest = new DeleteShareGroupOffsetsRequestData() .setGroupId("group") @@ -12390,7 +12493,9 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] - kafkaApis = createKafkaApis() + kafkaApis = createKafkaApis( + overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), + ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val deleteShareGroupOffsetsResponse = new DeleteShareGroupOffsetsResponseData() @@ -12435,9 +12540,13 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava + val config = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", + ) + val response = getWriteShareGroupStateResponse( writeRequestData, - ShareCoordinatorTestConfig.testConfigMap().asScala, + config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, null, writeStateResultData @@ -12482,13 +12591,17 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava + val config = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", + ) + val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) val response = getWriteShareGroupStateResponse( writeRequestData, - ShareCoordinatorTestConfig.testConfigMap().asScala, + config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, authorizer, writeStateResultData @@ -12527,9 +12640,13 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava + val config = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", + ) + val response = getDeleteShareGroupStateResponse( deleteRequestData, - ShareCoordinatorTestConfig.testConfigMap().asScala, + config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, null, deleteStateResultData @@ -12568,9 +12685,13 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) + val config = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", + ) + val response = getDeleteShareGroupStateResponse( deleteRequestData, - ShareCoordinatorTestConfig.testConfigMap().asScala, + config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, authorizer, deleteStateResultData @@ -12610,9 +12731,13 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava + val config = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", + ) + val response = getInitializeShareGroupStateResponse( initRequestData, - ShareCoordinatorTestConfig.testConfigMap().asScala, + config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, null, initStateResultData @@ -12652,9 +12777,13 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) + val config = Map( + ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", + ) + val response = getInitializeShareGroupStateResponse( initRequestData, - ShareCoordinatorTestConfig.testConfigMap().asScala, + config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, authorizer, initStateResultData @@ -12668,7 +12797,7 @@ class KafkaApisTest extends Logging { }) } - def getShareGroupDescribeResponse(groupIds: util.List[String], enableShareGroups: Boolean = true, + def getShareGroupDescribeResponse(groupIds: util.List[String], configOverrides: Map[String, String] = Map.empty, verifyNoErr: Boolean = true, authorizer: Authorizer = null, describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup]): ShareGroupDescribeResponse = { val shareGroupDescribeRequestData = new ShareGroupDescribeRequestData() @@ -12680,8 +12809,9 @@ class KafkaApisTest extends Logging { any[RequestContext], any[util.List[String]] )).thenReturn(future) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled(enableShareGroups) + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( + overrideProperties = configOverrides, authorizer = Option(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -12707,7 +12837,7 @@ class KafkaApisTest extends Logging { any[RequestContext], any[ReadShareGroupStateRequestData] )).thenReturn(future) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = configOverrides, authorizer = Option(authorizer), @@ -12736,7 +12866,7 @@ class KafkaApisTest extends Logging { any[RequestContext], any[ReadShareGroupStateSummaryRequestData] )).thenReturn(future) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = configOverrides, authorizer = Option(authorizer), @@ -12765,7 +12895,7 @@ class KafkaApisTest extends Logging { any[RequestContext], any[WriteShareGroupStateRequestData] )).thenReturn(future) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = configOverrides, authorizer = Option(authorizer), @@ -12794,7 +12924,7 @@ class KafkaApisTest extends Logging { any[RequestContext], any[DeleteShareGroupStateRequestData] )).thenReturn(future) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = configOverrides, authorizer = Option(authorizer), @@ -12823,7 +12953,7 @@ class KafkaApisTest extends Logging { any[RequestContext], any[InitializeShareGroupStateRequestData] )).thenReturn(future) - metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis( overrideProperties = configOverrides, authorizer = Option(authorizer), diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index a8629b6ab5a2c..55c88666c70b7 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -814,7 +814,6 @@ class KafkaConfigTest { case ServerLogConfigs.LOG_DIRS_CONFIG => // ignore string case ServerLogConfigs.LOG_DIR_CONFIG => // ignore string case ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", Records.LOG_OVERHEAD - 1) - case ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG => // no op case ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") @@ -1743,11 +1742,13 @@ class KafkaConfigTest { props.put(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer") var config = KafkaConfig.fromProps(props) assertEquals(Set(GroupType.CLASSIC, GroupType.CONSUMER), config.groupCoordinatorRebalanceProtocols) + assertFalse(config.shareGroupConfig.isShareGroupEnabled) // This is OK. props.put(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer,share") config = KafkaConfig.fromProps(props) assertEquals(Set(GroupType.CLASSIC, GroupType.CONSUMER, GroupType.SHARE), config.groupCoordinatorRebalanceProtocols) + assertTrue(config.shareGroupConfig.isShareGroupEnabled) props.put(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,streams") val config2 = KafkaConfig.fromProps(props) diff --git a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala index 7aba491536fd0..45afae3a0dc13 100644 --- a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala @@ -17,14 +17,13 @@ package kafka.server import kafka.utils.TestUtils -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, ClusterTests, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, ClusterTests, Type} import org.apache.kafka.common.message.ShareFetchResponseData.AcquiredRecords import org.apache.kafka.common.message.{ShareAcknowledgeRequestData, ShareAcknowledgeResponseData, ShareFetchRequestData, ShareFetchResponseData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.requests.{ShareAcknowledgeRequest, ShareAcknowledgeResponse, ShareFetchRequest, ShareFetchResponse, ShareRequestMetadata} import org.apache.kafka.common.test.ClusterInstance -import org.apache.kafka.server.common.Feature import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, Timeout} @@ -43,14 +42,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo @AfterEach def tearDown(): Unit = { closeProducer - closeSockets } - @ClusterTest( - features = Array( - new ClusterFeature(feature = Feature.SHARE_VERSION, version = 0) - ) - ) + @ClusterTest def testShareFetchRequestIsInAccessibleWhenConfigsDisabled(): Unit = { val groupId: String = "group" val metadata: ShareRequestMetadata = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH) @@ -60,23 +54,19 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo ) val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) assertEquals(Errors.UNSUPPORTED_VERSION.code, shareFetchResponse.data.errorCode) assertEquals(0, shareFetchResponse.data.acquisitionLockTimeoutMs) } - @ClusterTest( - features = Array( - new ClusterFeature(feature = Feature.SHARE_VERSION, version = 0) - ) - ) + @ClusterTest def testShareAcknowledgeRequestIsInAccessibleWhenConfigsDisabled(): Unit = { val groupId: String = "group" val metadata: ShareRequestMetadata = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH) val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, Map.empty) - val shareAcknowledgeResponse = connectAndReceiveWithoutClosingSocket[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) assertEquals(Errors.UNSUPPORTED_VERSION.code, shareAcknowledgeResponse.data.errorCode) } @@ -85,6 +75,8 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ), @@ -92,6 +84,8 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -124,8 +118,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // Send the share fetch request to the non-replica and verify the error code val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest, nonReplicaId) + val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest, nonReplicaId) assertEquals(30000, shareFetchResponse.data.acquisitionLockTimeoutMs) + val partitionData = shareFetchResponse.responseData(topicNames).get(topicIdPartition) assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.code, partitionData.errorCode) assertEquals(leader, partitionData.currentLeader().leaderId()) @@ -135,12 +130,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -174,7 +173,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val metadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)) val acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMap) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -197,12 +196,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -245,7 +248,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // as the share partitions might not be initialized yet. So, we retry until we get the response. var responses = Seq[ShareFetchResponseData.PartitionData]() TestUtils.waitUntilTrue(() => { - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) assertEquals(30000, shareFetchResponseData.acquisitionLockTimeoutMs) @@ -293,6 +296,8 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ), @@ -300,6 +305,8 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -340,9 +347,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareFetchRequest2 = createShareFetchRequest(groupId, metadata, send2, Seq.empty, acknowledgementsMap) var shareFetchRequest3 = createShareFetchRequest(groupId, metadata, send3, Seq.empty, acknowledgementsMap) - var shareFetchResponse1 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest1, destination = leader1) - var shareFetchResponse2 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest2, destination = leader2) - var shareFetchResponse3 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest3, destination = leader3) + var shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1, destination = leader1) + var shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2, destination = leader2) + var shareFetchResponse3 = connectAndReceive[ShareFetchResponse](shareFetchRequest3, destination = leader3) initProducer() // Producing 10 records to the topic partitions created above @@ -356,9 +363,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareFetchRequest2 = createShareFetchRequest(groupId, metadata, send2, Seq.empty, acknowledgementsMap) shareFetchRequest3 = createShareFetchRequest(groupId, metadata, send3, Seq.empty, acknowledgementsMap) - shareFetchResponse1 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest1, destination = leader1) - shareFetchResponse2 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest2, destination = leader2) - shareFetchResponse3 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest3, destination = leader3) + shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1, destination = leader1) + shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2, destination = leader2) + shareFetchResponse3 = connectAndReceive[ShareFetchResponse](shareFetchRequest3, destination = leader3) val shareFetchResponseData1 = shareFetchResponse1.data() assertEquals(Errors.NONE.code, shareFetchResponseData1.errorCode) @@ -410,12 +417,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -451,7 +462,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -478,7 +489,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - val shareAcknowledgeResponse = connectAndReceiveWithoutClosingSocket[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -500,7 +511,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -523,6 +534,8 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000") @@ -530,6 +543,8 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -566,7 +581,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -595,7 +610,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -620,7 +635,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -643,12 +658,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -684,7 +703,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -711,7 +730,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(2.toByte))).asJava) // Release the records val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - val shareAcknowledgeResponse = connectAndReceiveWithoutClosingSocket[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -730,7 +749,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -753,12 +772,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -794,7 +817,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -839,7 +862,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo releaseAcknowledgementSent = true } shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -867,12 +890,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -908,7 +935,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -935,7 +962,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(3.toByte))).asJava) // Reject the records val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - val shareAcknowledgeResponse = connectAndReceiveWithoutClosingSocket[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -957,7 +984,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -980,12 +1007,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1021,7 +1052,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1050,7 +1081,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(3.toByte))).asJava) // Reject the records shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1075,7 +1106,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1098,6 +1129,8 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.delivery.count.limit", value = "2") // Setting max delivery count config to 2 @@ -1105,6 +1138,8 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1141,7 +1176,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1168,7 +1203,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(2.toByte))).asJava) // Release the records var shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - var shareAcknowledgeResponse = connectAndReceiveWithoutClosingSocket[ShareAcknowledgeResponse](shareAcknowledgeRequest) + var shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) var shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -1187,7 +1222,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1213,7 +1248,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(2.toByte))).asJava) // Release the records again shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - shareAcknowledgeResponse = connectAndReceiveWithoutClosingSocket[ShareAcknowledgeResponse](shareAcknowledgeRequest) + shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -1235,7 +1270,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1258,12 +1293,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1312,9 +1351,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val acknowledgementsMap3: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty val shareFetchRequest3 = createShareFetchRequest(groupId, metadata3, send, Seq.empty, acknowledgementsMap3, minBytes = 100, maxBytes = 1500) - val shareFetchResponse1 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest1) - val shareFetchResponse2 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest2) - val shareFetchResponse3 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest3) + val shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1) + val shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2) + val shareFetchResponse3 = connectAndReceive[ShareFetchResponse](shareFetchRequest3) val shareFetchResponseData1 = shareFetchResponse1.data() @@ -1350,12 +1389,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1407,9 +1450,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val acknowledgementsMap3: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty val shareFetchRequest3 = createShareFetchRequest(groupId3, metadata3, send, Seq.empty, acknowledgementsMap3) - val shareFetchResponse1 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest1) - val shareFetchResponse2 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest2) - val shareFetchResponse3 = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest3) + val shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1) + val shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2) + val shareFetchResponse3 = connectAndReceive[ShareFetchResponse](shareFetchRequest3) val shareFetchResponseData1 = shareFetchResponse1.data() @@ -1446,12 +1489,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1487,7 +1534,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1516,7 +1563,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1542,7 +1589,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(19) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1554,12 +1601,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1595,7 +1646,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1624,7 +1675,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1651,7 +1702,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(19) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - val shareAcknowledgeResponse = connectAndReceiveWithoutClosingSocket[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -1671,12 +1722,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1711,7 +1766,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Acknowledgements in the Initial Fetch Request val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMap) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() // The response will have a top level error code because this is an Initial Fetch request with acknowledgement data present @@ -1723,12 +1778,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1759,7 +1818,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMap) - val shareAcknowledgeResponse = connectAndReceiveWithoutClosingSocket[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.INVALID_SHARE_SESSION_EPOCH.code, shareAcknowledgeResponseData.errorCode) @@ -1769,12 +1828,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1809,7 +1872,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - var shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1831,7 +1894,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.nextEpoch(shareSessionEpoch)) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.INVALID_SHARE_SESSION_EPOCH.code, shareFetchResponseData.errorCode) @@ -1841,12 +1904,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1881,7 +1948,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1908,7 +1975,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMap) - val shareAcknowledgeResponse = connectAndReceiveWithoutClosingSocket[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.INVALID_SHARE_SESSION_EPOCH.code, shareAcknowledgeResponseData.errorCode) @@ -1918,12 +1985,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1959,7 +2030,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - var shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1981,7 +2052,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(wrongMemberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.SHARE_SESSION_NOT_FOUND.code, shareFetchResponseData.errorCode) @@ -1991,94 +2062,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "group.share.max.share.sessions", value="2"), - new ClusterConfigProperty(key = "group.share.max.size", value="2") - ) - ), - new ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "group.share.max.share.sessions", value="2"), - new ClusterConfigProperty(key = "group.share.max.size", value="2") - ) - ), - ) - ) - def testShareSessionEvictedOnConnectionDrop(): Unit = { - val groupId: String = "group" - val memberId1 = Uuid.randomUuid() - val memberId2 = Uuid.randomUuid() - val memberId3 = Uuid.randomUuid() - - val topic = "topic" - val partition = 0 - - createTopicAndReturnLeaders(topic, numPartitions = 3) - val topicIds = getTopicIds.asJava - val topicId = topicIds.get(topic) - val topicIdPartition = new TopicIdPartition(topicId, new TopicPartition(topic, partition)) - - val send: Seq[TopicIdPartition] = Seq(topicIdPartition) - - // member1 sends share fetch request to register it's share session. Note it does not close the socket connection after. - TestUtils.waitUntilTrue(() => { - val metadata = new ShareRequestMetadata(memberId1, ShareRequestMetadata.INITIAL_EPOCH) - val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) - val shareFetchResponseData = shareFetchResponse.data() - shareFetchResponseData.errorCode == Errors.NONE.code - }, "Share fetch request failed", 5000) - - // member2 sends share fetch request to register it's share session. Note it does not close the socket connection after. - TestUtils.waitUntilTrue(() => { - val metadata = new ShareRequestMetadata(memberId2, ShareRequestMetadata.INITIAL_EPOCH) - val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) - val shareFetchResponseData = shareFetchResponse.data() - shareFetchResponseData.errorCode == Errors.NONE.code - }, "Share fetch request failed", 5000) - - // member3 sends share fetch request to register it's share session. Since the maximum number of share sessions that could - // exist in the share session cache is 2 (group.share.max.share.sessions), the attempt to register a third - // share session with the ShareSessionCache would throw SHARE_SESSION_LIMIT_REACHED - TestUtils.waitUntilTrue(() => { - val metadata = new ShareRequestMetadata(memberId3, ShareRequestMetadata.INITIAL_EPOCH) - val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) - val shareFetchResponseData = shareFetchResponse.data() - shareFetchResponseData.errorCode == Errors.SHARE_SESSION_NOT_FOUND.code - }, "Share fetch request failed", 5000) - - // Now we will close the socket connections for the above three members, mimicking a client disconnection - closeSockets() - - // Since the socket connections were closed before, the corresponding share sessions were dropped from the ShareSessionCache - // on the broker. Now, since the cache is empty, new share sessions can be registered - TestUtils.waitUntilTrue(() => { - val metadata = new ShareRequestMetadata(memberId3, ShareRequestMetadata.INITIAL_EPOCH) - val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) - val shareFetchResponseData = shareFetchResponse.data() - shareFetchResponseData.errorCode == Errors.NONE.code - }, "Share fetch request failed", 5000) - } - - @ClusterTests( - Array( - new ClusterTest( - serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -2114,7 +2107,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -2141,7 +2134,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMap) - val shareAcknowledgeResponse = connectAndReceiveWithoutClosingSocket[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.SHARE_SESSION_NOT_FOUND.code, shareAcknowledgeResponseData.errorCode) @@ -2151,12 +2144,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -2200,7 +2197,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // as the share partitions might not be initialized yet. So, we retry until we get the response. var responses = Seq[ShareFetchResponseData.PartitionData]() TestUtils.waitUntilTrue(() => { - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) assertEquals(30000, shareFetchResponseData.acquisitionLockTimeoutMs) @@ -2226,7 +2223,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val forget: Seq[TopicIdPartition] = Seq(topicIdPartition1) shareFetchRequest = createShareFetchRequest(groupId, metadata, Seq.empty, forget, acknowledgementsMap) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -2249,12 +2246,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -2311,12 +2312,16 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -2376,7 +2381,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo TestUtils.waitUntilTrue(() => { val metadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH) val shareFetchRequest = createShareFetchRequest(groupId, metadata, topicIdPartitions, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceiveWithoutClosingSocket[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala index 408f31db8d15f..6c707c7bae13f 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala @@ -29,7 +29,6 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.common.Feature import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Timeout @@ -42,11 +41,7 @@ import scala.jdk.CollectionConverters._ )) class ShareGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest( - features = Array( - new ClusterFeature(feature = Feature.SHARE_VERSION, version = 0) - ) - ) + @ClusterTest def testShareGroupDescribeIsInAccessibleWhenConfigsDisabled(): Unit = { val shareGroupDescribeRequest = new ShareGroupDescribeRequest.Builder( new ShareGroupDescribeRequestData().setGroupIds(List("grp-1", "grp-2").asJava) @@ -70,8 +65,10 @@ class ShareGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCoord @ClusterTest( serverProperties = Array( + new ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer,share"), new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"), + new ClusterConfigProperty(key = ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG, value = "true"), ) ) def testShareGroupDescribe(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala index 865870eef3beb..77aae6c4d5731 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala @@ -16,7 +16,7 @@ */ package kafka.server -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, NewPartitions} import org.apache.kafka.common.Uuid @@ -24,7 +24,6 @@ import org.apache.kafka.common.message.{ShareGroupHeartbeatRequestData, ShareGro import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse} import org.apache.kafka.common.test.ClusterInstance -import org.apache.kafka.server.common.Feature import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertNotNull, assertNull, assertTrue} import org.junit.jupiter.api.Timeout @@ -38,9 +37,7 @@ import scala.jdk.CollectionConverters._ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { @ClusterTest( - features = Array( - new ClusterFeature(feature = Feature.SHARE_VERSION, version = 0) - ) + types = Array(Type.KRAFT) ) def testShareGroupHeartbeatIsInAccessibleWhenConfigsDisabled(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( @@ -53,7 +50,9 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } @ClusterTest( + types = Array(Type.KRAFT), serverProperties = Array( + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") )) @@ -143,7 +142,9 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } @ClusterTest( + types = Array(Type.KRAFT), serverProperties = Array( + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") )) @@ -297,7 +298,9 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } @ClusterTest( + types = Array(Type.KRAFT), serverProperties = Array( + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") )) @@ -406,7 +409,9 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } @ClusterTest( + types = Array(Type.KRAFT), serverProperties = Array( + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") )) @@ -586,82 +591,10 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "group.share.max.size", value = "2") - )) - def testShareGroupMaxSizeConfigExceeded(): Unit = { - val groupId: String = "group" - val memberId1 = Uuid.randomUuid() - val memberId2 = Uuid.randomUuid() - val memberId3 = Uuid.randomUuid() - - val admin = cluster.admin() - - // Creates the __consumer_offsets topics because it won't be created automatically - // in this test because it does not use FindCoordinator API. - try { - TestUtils.createOffsetsTopicWithAdmin( - admin = admin, - brokers = cluster.brokers.values().asScala.toSeq, - controllers = cluster.controllers().values().asScala.toSeq - ) - - // Heartbeat request to join the group by the first member (memberId1). - var shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( - new ShareGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId1.toString) - .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava) - ).build() - - // Send the request until receiving a successful response. There is a delay - // here because the group coordinator is loaded in the background. - var shareGroupHeartbeatResponse: ShareGroupHeartbeatResponse = null - TestUtils.waitUntilTrue(() => { - shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) - shareGroupHeartbeatResponse.data.errorCode == Errors.NONE.code - }, msg = s"Could not join the group successfully. Last response $shareGroupHeartbeatResponse.") - - // Heartbeat request to join the group by the second member (memberId2). - shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( - new ShareGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId2.toString) - .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava) - ).build() - - // Send the request until receiving a successful response - TestUtils.waitUntilTrue(() => { - shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) - shareGroupHeartbeatResponse.data.errorCode == Errors.NONE.code - }, msg = s"Could not join the group successfully. Last response $shareGroupHeartbeatResponse.") - - // Heartbeat request to join the group by the third member (memberId3). - shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( - new ShareGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId3.toString) - .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava) - ).build() - - shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) - // Since the group.share.max.size config is set to 2, a third member cannot join the same group. - assertEquals(shareGroupHeartbeatResponse.data.errorCode, Errors.GROUP_MAX_SIZE_REACHED.code) - - } finally { - admin.close() - } - } - @ClusterTest( types = Array(Type.KRAFT), serverProperties = Array( + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.heartbeat.interval.ms", value = "500"), @@ -840,7 +773,9 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } @ClusterTest( + types = Array(Type.KRAFT), serverProperties = Array( + new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") )) diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala index 20e5072cdd898..575d3855dcde5 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -196,7 +196,7 @@ class BrokerMetadataPublisherTest { replicaManager, groupCoordinator, mock(classOf[TransactionCoordinator]), - mock(classOf[ShareCoordinator]), + Some(mock(classOf[ShareCoordinator])), mock(classOf[DynamicConfigPublisher]), mock(classOf[DynamicClientQuotaPublisher]), mock(classOf[DynamicTopicClusterQuotaPublisher]), diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index d9de803cc76f4..5b3e9abe1112b 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -545,7 +545,8 @@ class DumpLogSegmentsTest { time, time.scheduler, new MetadataLogConfig( - 1024 * 1024, + 100 * 1024, + 100 * 1024, 10 * 1000, 100 * 1024, 60 * 1000, diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java index faa9b92dff6c4..d686ba1a3427c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.coordinator.group.modern.share; +import org.apache.kafka.common.GroupType; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.utils.Utils; @@ -23,6 +24,8 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; @@ -69,7 +72,7 @@ public class ShareGroupConfig { public static final String SHARE_GROUP_PERSISTER_CLASS_NAME_CONFIG = "group.share.persister.class.name"; public static final String SHARE_GROUP_PERSISTER_CLASS_NAME_DEFAULT = "org.apache.kafka.server.share.persister.DefaultStatePersister"; - public static final String SHARE_GROUP_PERSISTER_CLASS_NAME_DOC = "The fully qualified name of a class which implements " + + public static final String SHARE_GROUP_PERSISTER_CLASS_NAME_DOC = "The class name of share persister for share group. The class should implement " + "the org.apache.kafka.server.share.Persister interface."; public static final ConfigDef CONFIG_DEF = new ConfigDef() @@ -96,8 +99,13 @@ public class ShareGroupConfig { public ShareGroupConfig(AbstractConfig config) { this.config = config; - // The proper way to enable share groups is to use the share.version feature with v1 or later. - isShareGroupEnabled = config.getBoolean(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG); + // Share groups are enabled in either of the two following cases: + // 1. The internal configuration to enable it is explicitly set; or + // 2. the share rebalance protocol is enabled. + Set protocols = config.getList(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG) + .stream().map(String::toUpperCase).collect(Collectors.toSet()); + isShareGroupEnabled = config.getBoolean(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG) || + protocols.contains(GroupType.SHARE.name()); shareGroupPartitionMaxRecordLocks = config.getInt(ShareGroupConfig.SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG); shareGroupDeliveryCountLimit = config.getInt(ShareGroupConfig.SHARE_GROUP_DELIVERY_COUNT_LIMIT_CONFIG); shareGroupRecordLockDurationMs = config.getInt(ShareGroupConfig.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 81b127af16d0d..250b10fec34e2 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -21260,21 +21260,16 @@ public void testShareGroupHeartbeatInitializeOnPartitionUpdate() { .setMemberEpoch(0) .setSubscribedTopicNames(List.of(t1Name, t2Name))); - CoordinatorRecord expected = newShareGroupStatePartitionMetadataRecord( - groupId, - mkShareGroupStateMap(List.of( - mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(0, 1)), - mkShareGroupStateMetadataEntry(t2Uuid, t2Name, List.of(0, 1)) - )), - Map.of(), - Map.of() + assertTrue(result.records().contains( + newShareGroupStatePartitionMetadataRecord(groupId, mkShareGroupStateMap(List.of( + mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(0, 1)), + mkShareGroupStateMetadataEntry(t2Uuid, t2Name, List.of(0, 1)) + )), + Map.of(), + Map.of() + )) ); - Optional actual = result.records().stream().filter(record -> record.key() instanceof ShareGroupStatePartitionMetadataKey) - .findAny(); - assertTrue(actual.isPresent()); - assertRecordEquals(expected, actual.get()); - verifyShareGroupHeartbeatInitializeRequest( result.response().getValue(), Map.of( @@ -21335,23 +21330,18 @@ public void testShareGroupHeartbeatInitializeOnPartitionUpdate() { .setMemberEpoch(1) .setSubscribedTopicNames(null)); - expected = newShareGroupStatePartitionMetadataRecord( - groupId, - mkShareGroupStateMap(List.of( - mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(2, 3)) - )), - mkShareGroupStateMap(List.of( - mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(0, 1)), - mkShareGroupStateMetadataEntry(t2Uuid, t2Name, List.of(0, 1)) - )), - Map.of() + assertTrue(result.records().contains( + newShareGroupStatePartitionMetadataRecord(groupId, mkShareGroupStateMap(List.of( + mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(2, 3)) + )), + mkShareGroupStateMap(List.of( + mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(0, 1)), + mkShareGroupStateMetadataEntry(t2Uuid, t2Name, List.of(0, 1)) + )), + Map.of() + )) ); - actual = result.records().stream().filter(record -> record.key() instanceof ShareGroupStatePartitionMetadataKey) - .findAny(); - assertTrue(actual.isPresent()); - assertRecordEquals(expected, actual.get()); - verifyShareGroupHeartbeatInitializeRequest( result.response().getValue(), Map.of( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java index 73c0c0cc10e68..698dd59380c67 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java @@ -122,6 +122,7 @@ public static ShareGroupConfig createShareGroupConfig( configs.put(ShareGroupConfig.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG, shareGroupRecordLockDurationsMs); configs.put(ShareGroupConfig.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG, shareGroupMinRecordLockDurationMs); configs.put(ShareGroupConfig.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG, shareGroupMaxRecordLockDurationMs); + configs.put(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer,share"); configs.put(GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG, 1); configs.put(GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, 10); diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index b88b8cb11a25b..5005f509cbd1f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -878,9 +878,8 @@ private VoteResponseData handleVoteRequest( Optional voterKey = RaftUtil.voteRequestVoterKey(request, partitionRequest); if (!isValidVoterKey(voterKey)) { logger.info( - "A replica {} sent a voter key ({}) in the VOTE request that doesn't match the " + + "A replica sent a voter key ({}) in the VOTE request that doesn't match the " + "local key ({}, {}); rejecting the vote", - replicaId, voterKey, nodeId, nodeDirectoryId diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index cd276f4c799e4..b0a6f9f045ad6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.record.Records; import org.apache.kafka.server.config.ServerLogConfigs; import java.util.concurrent.TimeUnit; @@ -51,6 +52,10 @@ public class MetadataLogConfig { "configuration. The Kafka node will generate a snapshot when either the maximum time interval is reached or the " + "maximum bytes limit is reached."; + public static final String METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG = "metadata.log.segment.min.bytes"; + public static final String METADATA_LOG_SEGMENT_MIN_BYTES_DOC = "Override the minimum size for a single metadata log file. This should be used for testing only."; + public static final int METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT = 8 * 1024 * 1024; + public static final String METADATA_LOG_SEGMENT_BYTES_CONFIG = "metadata.log.segment.bytes"; public static final String METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file."; public static final int METADATA_LOG_SEGMENT_BYTES_DEFAULT = 1024 * 1024 * 1024; @@ -79,13 +84,15 @@ public class MetadataLogConfig { .define(METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG, LONG, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_DOC) .define(METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC) .define(METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, METADATA_LOG_DIR_DOC) - .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(8 * 1024 * 1024), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) + .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) + .defineInternal(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_MIN_BYTES_DOC) .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); private final int logSegmentBytes; + private final int logSegmentMinBytes; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -96,6 +103,7 @@ public class MetadataLogConfig { /** * Configuration for the metadata log * @param logSegmentBytes The maximum size of a single metadata log file + * @param logSegmentMinBytes The minimum size of a single metadata log file * @param logSegmentMillis The maximum time before a new metadata log file is rolled out * @param retentionMaxBytes The size of the metadata log and snapshots before deleting old snapshots and log files * @param retentionMillis The time to keep a metadata log file or snapshot before deleting it @@ -104,6 +112,7 @@ public class MetadataLogConfig { * @param deleteDelayMillis The amount of time to wait before deleting a file from the filesystem */ public MetadataLogConfig(int logSegmentBytes, + int logSegmentMinBytes, long logSegmentMillis, long retentionMaxBytes, long retentionMillis, @@ -111,6 +120,7 @@ public MetadataLogConfig(int logSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = logSegmentBytes; + this.logSegmentMinBytes = logSegmentMinBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; @@ -121,6 +131,7 @@ public MetadataLogConfig(int logSegmentBytes, public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); + this.logSegmentMinBytes = config.getInt(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); @@ -133,6 +144,10 @@ public int logSegmentBytes() { return logSegmentBytes; } + public int logSegmentMinBytes() { + return logSegmentMinBytes; + } + public long logSegmentMillis() { return logSegmentMillis; } diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index f735370ce8d9a..2c2d44ef012e5 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -43,9 +43,6 @@ public class ServerLogConfigs { public static final String LOG_SEGMENT_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); public static final String LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file"; - public static final String INTERNAL_LOG_SEGMENT_BYTES_CONFIG = "internal." + ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); - public static final String INTERNAL_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; - public static final String LOG_ROLL_TIME_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_MS_CONFIG); public static final String LOG_ROLL_TIME_HOURS_CONFIG = LOG_PREFIX + "roll.hours"; public static final String LOG_ROLL_TIME_MILLIS_DOC = "The maximum time before a new log segment is rolled out (in milliseconds). If not set, the value in " + LOG_ROLL_TIME_HOURS_CONFIG + " is used"; diff --git a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java index 7a612db1478c5..ef61fee154283 100644 --- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java @@ -82,10 +82,6 @@ public int numRecoveryThreadsPerDataDir() { return getInt(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG); } - public Integer internalSegmentSize() { - return getInt(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG); - } - public int backgroundThreads() { return getInt(ServerConfigs.BACKGROUND_THREADS_CONFIG); } diff --git a/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java b/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java index f0f37d9ec7dca..0b06ea535be7e 100644 --- a/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java +++ b/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.requests.ShareRequestMetadata; import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; import org.apache.kafka.server.metrics.KafkaMetricsGroup; -import org.apache.kafka.server.network.ConnectionDisconnectListener; import org.apache.kafka.server.share.CachedSharePartition; import com.yammer.metrics.core.Meter; @@ -54,13 +53,10 @@ public class ShareSessionCache { private final int maxEntries; private long numPartitions = 0; - private final ConnectionDisconnectListener connectionDisconnectListener; // A map of session key to ShareSession. private final Map sessions = new HashMap<>(); - private final Map connectionIdToSessionMap; - @SuppressWarnings("this-escape") public ShareSessionCache(int maxEntries) { this.maxEntries = maxEntries; @@ -68,8 +64,6 @@ public ShareSessionCache(int maxEntries) { KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "ShareSessionCache"); metricsGroup.newGauge(SHARE_SESSIONS_COUNT, this::size); metricsGroup.newGauge(SHARE_PARTITIONS_COUNT, this::totalPartitions); - this.connectionIdToSessionMap = new HashMap<>(); - this.connectionDisconnectListener = new ClientConnectionDisconnectListener(); this.evictionsMeter = metricsGroup.newMeter(SHARE_SESSION_EVICTIONS_PER_SEC, "evictions", TimeUnit.SECONDS); } @@ -129,48 +123,21 @@ public synchronized void updateNumPartitions(ShareSession session) { * @param groupId - The group id in the share fetch request. * @param memberId - The member id in the share fetch request. * @param partitionMap - The topic partitions to be added to the session. - * @param clientConnectionId - The client connection id. * @return - The session key if the session was created, or null if the session was not created. */ - public synchronized ShareSessionKey maybeCreateSession( - String groupId, - Uuid memberId, - ImplicitLinkedHashCollection partitionMap, - String clientConnectionId - ) { + public synchronized ShareSessionKey maybeCreateSession(String groupId, Uuid memberId, ImplicitLinkedHashCollection partitionMap) { if (sessions.size() < maxEntries) { ShareSession session = new ShareSession(new ShareSessionKey(groupId, memberId), partitionMap, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)); sessions.put(session.key(), session); updateNumPartitions(session); - connectionIdToSessionMap.put(clientConnectionId, session.key()); return session.key(); } return null; } - public ConnectionDisconnectListener connectionDisconnectListener() { - return connectionDisconnectListener; - } - // Visible for testing. Meter evictionsMeter() { return evictionsMeter; } - - private final class ClientConnectionDisconnectListener implements ConnectionDisconnectListener { - - // When the client disconnects, the corresponding session should be removed from the cache. - @Override - public void onDisconnect(String connectionId) { - ShareSessionKey shareSessionKey = connectionIdToSessionMap.remove(connectionId); - if (shareSessionKey != null) { - // Remove the session from the cache. - ShareSession removedSession = remove(shareSessionKey); - if (removedSession != null) { - evictionsMeter.mark(); - } - } - } - } } diff --git a/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java b/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java index c9692063b5c99..ca18de5b65c47 100644 --- a/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java +++ b/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java @@ -45,11 +45,11 @@ public void setUp() { public void testShareSessionCache() throws InterruptedException { ShareSessionCache cache = new ShareSessionCache(3); assertEquals(0, cache.size()); - ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(10), "conn-1"); - ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(20), "conn-2"); - ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(30), "conn-3"); - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(40), "conn-4")); - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5), "conn-5")); + ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(10)); + ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(20)); + ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(30)); + assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(40))); + assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5))); assertShareCacheContains(cache, List.of(key1, key2, key3)); assertMetricsValues(3, 60, 0, cache); @@ -60,7 +60,7 @@ public void testResizeCachedSessions() throws InterruptedException { ShareSessionCache cache = new ShareSessionCache(2); assertEquals(0, cache.size()); assertEquals(0, cache.totalPartitions()); - ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(2), "conn-1"); + ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(2)); assertNotNull(key1); assertShareCacheContains(cache, List.of(key1)); ShareSession session1 = cache.get(key1); @@ -70,7 +70,7 @@ public void testResizeCachedSessions() throws InterruptedException { assertMetricsValues(1, 2, 0, cache); - ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(4), "conn-2"); + ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(4)); assertNotNull(key2); assertShareCacheContains(cache, List.of(key1, key2)); ShareSession session2 = cache.get(key2); @@ -81,7 +81,7 @@ public void testResizeCachedSessions() throws InterruptedException { assertMetricsValues(2, 6, 0, cache); - ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5), "conn-3"); + ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5)); assertNull(key3); assertShareCacheContains(cache, List.of(key1, key2)); assertEquals(6, cache.totalPartitions()); @@ -109,36 +109,6 @@ public void testResizeCachedSessions() throws InterruptedException { assertMetricsValues(1, 3, 0, cache); } - @Test - public void testRemoveConnection() throws InterruptedException { - ShareSessionCache cache = new ShareSessionCache(3); - assertEquals(0, cache.size()); - ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(1), "conn-1"); - ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(2), "conn-2"); - ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(3), "conn-3"); - - assertMetricsValues(3, 6, 0, cache); - - // Since cache size is now equal to max entries allowed(3), no new session can be created. - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(40), "conn-4")); - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5), "conn-5")); - assertShareCacheContains(cache, List.of(key1, key2, key3)); - - assertMetricsValues(3, 6, 0, cache); - - // Simulating the disconnection of client with connection id conn-1 - cache.connectionDisconnectListener().onDisconnect("conn-1"); - assertShareCacheContains(cache, List.of(key2, key3)); - - assertMetricsValues(2, 5, 1, cache); - - // Since one client got disconnected, we can add another one now - ShareSessionKey key4 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(4), "conn-6"); - assertShareCacheContains(cache, List.of(key2, key3, key4)); - - assertMetricsValues(3, 9, 1, cache); - } - private ImplicitLinkedHashCollection mockedSharePartitionMap(int size) { ImplicitLinkedHashCollection cacheMap = new ImplicitLinkedHashCollection<>(size); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java index 54ad6a4e79a97..b7d4ccedb2ec1 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java @@ -171,7 +171,7 @@ public Map.Entry doClean(LogToClean cleanable, long currentT List> groupedSegments = groupSegmentsBySize( log.logSegments(0, endOffset), - log.config().segmentSize(), + log.config().segmentSize, log.config().maxIndexSize, cleanable.firstUncleanableOffset() ); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index ebc4ea60e8a57..21c92cd84dff4 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.LegacyRecord; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ConfigUtils; import org.apache.kafka.common.utils.Utils; @@ -185,13 +186,12 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) - .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) - .defineInternal(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_DOC); + .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { CONFIG. - define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(1024 * 1024), MEDIUM, + define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), MEDIUM, TopicConfig.SEGMENT_BYTES_DOC) .define(TopicConfig.SEGMENT_MS_CONFIG, LONG, DEFAULT_SEGMENT_MS, atLeast(1), MEDIUM, TopicConfig.SEGMENT_MS_DOC) .define(TopicConfig.SEGMENT_JITTER_MS_CONFIG, LONG, DEFAULT_SEGMENT_JITTER_MS, atLeast(0), MEDIUM, @@ -262,8 +262,7 @@ public Optional serverConfigName(String configName) { * Important note: Any configuration parameter that is passed along from KafkaConfig to LogConfig * should also be in `KafkaConfig#extractLogConfigMap`. */ - private final int segmentSize; - public final Integer internalSegmentSize; + public final int segmentSize; public final long segmentMs; public final long segmentJitterMs; public final int maxIndexSize; @@ -297,25 +296,16 @@ public Optional serverConfigName(String configName) { private final Map props; public LogConfig(Map props) { - this(props, Set.of(), null); - } - - public LogConfig(Map props, Set overriddenConfigs) { - this(props, overriddenConfigs, null); - } - - public LogConfig(Map props, Integer internalSegmentSize) { - this(props, Set.of(), internalSegmentSize); + this(props, Set.of()); } @SuppressWarnings({"this-escape"}) - public LogConfig(Map props, Set overriddenConfigs, Integer internalSegmentSize) { + public LogConfig(Map props, Set overriddenConfigs) { super(CONFIG, props, false); this.props = Collections.unmodifiableMap(props); this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); - this.internalSegmentSize = internalSegmentSize; this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); @@ -377,11 +367,6 @@ private Optional getCompression() { } } - public int segmentSize() { - if (internalSegmentSize != null) return internalSegmentSize; - return segmentSize; - } - // Exposed as a method so it can be mocked public int maxMessageSize() { return maxMessageSize; @@ -403,7 +388,7 @@ public long maxSegmentMs() { public int initFileSize() { if (preallocate) - return segmentSize(); + return segmentSize; else return 0; } @@ -437,19 +422,15 @@ public String overriddenConfigsAsLoggableString() { return ConfigUtils.configMapToRedactedString(overriddenTopicProps, CONFIG); } - public static LogConfig fromProps(Map defaults, Properties overrides, Integer internalLogSegmentSize) { - Properties props = new Properties(); - props.putAll(defaults); - props.putAll(overrides); - Set overriddenKeys = overrides.keySet().stream().map(k -> (String) k).collect(Collectors.toSet()); - return new LogConfig(props, overriddenKeys, internalLogSegmentSize); - } - /** * Create a log config instance using the given properties and defaults */ public static LogConfig fromProps(Map defaults, Properties overrides) { - return fromProps(defaults, overrides, null); + Properties props = new Properties(); + props.putAll(defaults); + props.putAll(overrides); + Set overriddenKeys = overrides.keySet().stream().map(k -> (String) k).collect(Collectors.toSet()); + return new LogConfig(props, overriddenKeys); } // Visible for testing, return a copy since it's a mutable global variable @@ -647,7 +628,7 @@ public static void validate(Map existingConfigs, @Override public String toString() { return "LogConfig{" + - "segmentSize=" + segmentSize() + + "segmentSize=" + segmentSize + ", segmentMs=" + segmentMs + ", segmentJitterMs=" + segmentJitterMs + ", maxIndexSize=" + maxIndexSize + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index ff689898ecd62..dcef6929d1902 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1177,9 +1177,9 @@ private LogAppendInfo append(MemoryRecords records, }); // check messages size does not exceed config.segmentSize - if (validRecords.sizeInBytes() > config().segmentSize()) { + if (validRecords.sizeInBytes() > config().segmentSize) { throw new RecordBatchTooLargeException("Message batch size is " + validRecords.sizeInBytes() + " bytes in append " + - "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize() + "."); + "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize + "."); } // maybe roll the log if this segment is full @@ -2034,12 +2034,12 @@ private LogSegment maybeRoll(int messagesSize, LogAppendInfo appendInfo) throws long maxTimestampInMessages = appendInfo.maxTimestamp(); long maxOffsetInMessages = appendInfo.lastOffset(); - if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize(), appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { + if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize, appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { logger.debug("Rolling new log segment (log_size = {}/{}}, " + "offset_index_size = {}/{}, " + "time_index_size = {}/{}, " + "inactive_time_ms = {}/{}).", - segment.size(), config().segmentSize(), + segment.size(), config().segmentSize, segment.offsetIndex().entries(), segment.offsetIndex().maxEntries(), segment.timeIndex().entries(), segment.timeIndex().maxEntries(), segment.timeWaitedForRoll(now, maxTimestampInMessages), config().segmentMs - segment.rollJitterMs()); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index 8bbf525df0ba8..7dfba8f7a5954 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -19,12 +19,12 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.util.MockTime; import org.apache.kafka.test.TestUtils; @@ -198,7 +198,7 @@ public void testUpdateConfig() { assertEquals(oldConfig, log.config()); Properties props = new Properties(); - props.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); + props.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldConfig.segmentSize + 1); LogConfig newConfig = new LogConfig(props); log.updateConfig(newConfig); assertEquals(newConfig, log.config()); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index 1c47061565ff2..6023706847f15 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -24,7 +24,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.config.TopicConfig; -import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; @@ -49,7 +49,6 @@ import java.io.IOException; import java.time.Duration; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -68,7 +67,7 @@ public class PurgeRepartitionTopicIntegrationTest { private static Admin adminClient; private static KafkaStreams kafkaStreams; private static final Integer PURGE_INTERVAL_MS = 10; - private static final Integer PURGE_SEGMENT_BYTES = 1024 * 1024; + private static final Integer PURGE_SEGMENT_BYTES = 2000; private static final Integer INITIAL_TASK_DELAY_MS = 0; public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS, new Properties() { @@ -168,8 +167,8 @@ public void setup() { streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, PURGE_INTERVAL_MS); streamsConfiguration.put(StreamsConfig.REPARTITION_PURGE_INTERVAL_MS_CONFIG, PURGE_INTERVAL_MS); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.ByteArray().getClass()); - streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.ByteArray().getClass()); + streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); + streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); @@ -196,17 +195,15 @@ public void shutdown() { @Test public void shouldRestoreState() throws Exception { // produce some data to input topic - final List> messages = new ArrayList<>(); + final List> messages = new ArrayList<>(); for (int i = 0; i < 1000; i++) { - final byte[] data = new byte[1024]; - Arrays.fill(data, (byte) i); - messages.add(new KeyValue<>(data, data)); + messages.add(new KeyValue<>(i, i)); } IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(INPUT_TOPIC, messages, TestUtils.producerConfig(CLUSTER.bootstrapServers(), - ByteArraySerializer.class, - ByteArraySerializer.class), + IntegerSerializer.class, + IntegerSerializer.class), time.milliseconds()); kafkaStreams.start(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java index 6ad61851ddae3..eb164e2b27fd7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java @@ -178,10 +178,10 @@ private static void assignActive(final ApplicationState applicationState, // assign any remaining unassigned tasks final List sortedTasks = new ArrayList<>(unassigned); Collections.sort(sortedTasks); - final Set candidateClients = clients.stream() + for (final TaskId taskId : sortedTasks) { + final Set candidateClients = clients.stream() .map(KafkaStreamsState::processId) .collect(Collectors.toSet()); - for (final TaskId taskId : sortedTasks) { final ProcessId bestClient = assignmentState.findBestClientForTask(taskId, candidateClients); assignmentState.finalizeAssignment(taskId, bestClient, AssignedTask.Type.ACTIVE); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java index 0315120ae1b1e..2bace901779fe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java @@ -44,44 +44,37 @@ public void testSessionWindowedDeserializerConstructor() { assertInstanceOf(StringDeserializer.class, inner, "Inner deserializer type should be StringDeserializer"); } - @Deprecated @Test public void shouldSetSerializerThroughWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); - try (final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>()) { - deserializer.configure(props, false); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); - } + final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>(); + deserializer.configure(props, false); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); } @Test public void shouldSetSerializerThroughWindowedInnerDeserializerClassConfig() { props.put(SessionWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); - try (final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>()) { - deserializer.configure(props, false); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); - } + final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>(); + deserializer.configure(props, false); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); } - @Deprecated @Test public void shouldIgnoreWindowedInnerClassSerdeConfigIfWindowedInnerDeserializerClassConfigIsSet() { props.put(SessionWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); - try (final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>()) { - deserializer.configure(props, false); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); - } + final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>(); + deserializer.configure(props, false); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); } @Test public void shouldThrowErrorIfWindowedInnerClassSerdeAndSessionWindowedDeserializerClassAreNotSet() { - try (final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>()) { - assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); - } + final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>(); + assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); } - @Deprecated @Test public void shouldThrowErrorIfDeserializersConflictInConstructorAndWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); @@ -94,7 +87,6 @@ public void shouldThrowErrorIfDeserializersConflictInConstructorAndWindowedInner assertThrows(IllegalArgumentException.class, () -> sessionWindowedDeserializer.configure(props, false)); } - @Deprecated @Test public void shouldThrowConfigExceptionWhenInvalidWindowedInnerClassSerdeSupplied() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java index 212b0c810e5f7..d7e30bc3fe448 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java @@ -44,44 +44,37 @@ public void testSessionWindowedSerializerConstructor() { assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); } - @Deprecated @Test public void shouldSetSerializerThroughWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); - try (final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>()) { - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); - } + final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>(); + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); } @Test public void shouldSetSerializerThroughWindowedInnerSerializerClassConfig() { props.put(SessionWindowedSerializer.WINDOWED_INNER_SERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); - try (final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>()) { - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); - } + final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>(); + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); } - @Deprecated @Test public void shouldIgnoreWindowedInnerClassSerdeConfigIfWindowedInnerSerializerClassConfigIsSet() { props.put(SessionWindowedSerializer.WINDOWED_INNER_SERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); - try (final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>()) { - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); - } + final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>(); + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); } @Test public void shouldThrowErrorIfWindowedInnerClassSerdeAndWindowedInnerSerializerClassAreNotSet() { - try (final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>()) { - assertThrows(IllegalArgumentException.class, () -> serializer.configure(props, false)); - } + final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>(); + assertThrows(IllegalArgumentException.class, () -> serializer.configure(props, false)); } - @Deprecated @Test public void shouldThrowErrorIfSerializersConflictInConstructorAndWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); @@ -94,7 +87,6 @@ public void shouldThrowErrorIfSerializersConflictInConstructorAndWindowedInnerSe assertThrows(IllegalArgumentException.class, () -> sessionWindowedSerializer.configure(props, false)); } - @Deprecated @Test public void shouldThrowConfigExceptionWhenInvalidWindowedInnerClassSerdeSupplied() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java index d2485c7785d78..bfb8c80cf094d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java @@ -49,49 +49,42 @@ public void testTimeWindowedDeserializerConstructor() { assertThat(timeWindowedDeserializer.getWindowSize(), is(5000000L)); } - @Deprecated @Test public void shouldSetWindowSizeAndDeserializerThroughWindowSizeMsAndWindowedInnerClassSerdeConfigs() { props.put(StreamsConfig.WINDOW_SIZE_MS_CONFIG, "500"); props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); - try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { - deserializer.configure(props, false); - assertThat(deserializer.getWindowSize(), is(500L)); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); - } + final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); + deserializer.configure(props, false); + assertThat(deserializer.getWindowSize(), is(500L)); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); } @Test public void shouldSetWindowSizeAndDeserializerThroughWindowSizeMsAndWindowedInnerDeserializerClassConfigs() { props.put(TimeWindowedDeserializer.WINDOW_SIZE_MS_CONFIG, "500"); props.put(TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); - try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { - deserializer.configure(props, false); - assertThat(deserializer.getWindowSize(), is(500L)); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); - } + final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); + deserializer.configure(props, false); + assertThat(deserializer.getWindowSize(), is(500L)); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); } - @Deprecated @Test public void shouldHaveSameConfigNameForWindowSizeMs() { assertEquals(TimeWindowedDeserializer.WINDOW_SIZE_MS_CONFIG, StreamsConfig.WINDOW_SIZE_MS_CONFIG); } - @Deprecated @Test public void shouldIgnoreWindowedInnerClassSerdeConfigIfWindowedInnerDeserializerClassConfigIsSet() { props.put(TimeWindowedDeserializer.WINDOW_SIZE_MS_CONFIG, "500"); props.put(TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); - try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { - deserializer.configure(props, false); - assertThat(deserializer.getWindowSize(), is(500L)); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); - } + final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); + deserializer.configure(props, false); + assertThat(deserializer.getWindowSize(), is(500L)); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); } - @Deprecated @Test public void shouldThrowErrorIfWindowSizeSetInStreamsConfigAndConstructor() { props.put(StreamsConfig.WINDOW_SIZE_MS_CONFIG, "500"); @@ -104,41 +97,34 @@ public void shouldThrowErrorIfWindowSizeSetInConstructorConfigAndConstructor() { assertThrows(IllegalArgumentException.class, () -> timeWindowedDeserializer.configure(props, false)); } - @Deprecated @Test public void shouldThrowErrorIfWindowSizeIsNotSetAndWindowedInnerClassSerdeIsSet() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); - try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { - assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); - } + final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); + assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); } @Test public void shouldThrowErrorIfWindowSizeIsNotSetAndWindowedInnerDeserializerClassIsSet() { - props.put(TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); - try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { - assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); - } + props.put(TimeWindowedDeserializer.WINDOW_SIZE_MS_CONFIG, Serdes.ByteArraySerde.class.getName()); + final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); + assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); } - @Deprecated @Test public void shouldThrowErrorIfWindowedInnerClassSerdeIsNotSetAndWindowSizeMsInStreamsConfigIsSet() { props.put(StreamsConfig.WINDOW_SIZE_MS_CONFIG, "500"); - try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { - assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); - } + final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); + assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); } @Test public void shouldThrowErrorIfWindowedInnerClassSerdeIsNotSetAndWindowSizeMsInConstructorConfigIsSet() { props.put(TimeWindowedDeserializer.WINDOW_SIZE_MS_CONFIG, "500"); - try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { - assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); - } + final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); + assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); } - @Deprecated @Test public void shouldThrowErrorIfDeserializerConflictInConstructorAndWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); @@ -151,7 +137,6 @@ public void shouldThrowErrorIfDeserializerConflictInConstructorAndWindowedInnerD assertThrows(IllegalArgumentException.class, () -> timeWindowedDeserializer.configure(props, false)); } - @Deprecated @Test public void shouldThrowConfigExceptionWhenInvalidWindowedInnerClassSerdeSupplied() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java index 5fd96f72c1bcf..7a13117db4ad7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java @@ -44,44 +44,37 @@ public void testTimeWindowedSerializerConstructor() { assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); } - @Deprecated @Test public void shouldSetSerializerThroughWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); - try (final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>()) { - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); - } + final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>(); + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); } @Test public void shouldSetSerializerThroughWindowedInnerSerializerClassConfig() { props.put(TimeWindowedSerializer.WINDOWED_INNER_SERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); - try (final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>()) { - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); - } + final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>(); + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); } - @Deprecated @Test public void shouldIgnoreWindowedInnerClassSerdeConfigIfWindowedInnerSerializerClassConfigIsSet() { props.put(TimeWindowedSerializer.WINDOWED_INNER_SERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); - try (final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>()) { - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); - } + final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>(); + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); } @Test public void shouldThrowErrorIfWindowedInnerClassSerdeAndWindowedInnerSerializerClassAreNotSet() { - try (final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>()) { - assertThrows(IllegalArgumentException.class, () -> serializer.configure(props, false)); - } + final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>(); + assertThrows(IllegalArgumentException.class, () -> serializer.configure(props, false)); } - @Deprecated @Test public void shouldThrowErrorIfSerializerConflictInConstructorAndWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); @@ -94,7 +87,6 @@ public void shouldThrowErrorIfSerializerConflictInConstructorAndWindowedInnerSer assertThrows(IllegalArgumentException.class, () -> timeWindowedSerializer.configure(props, false)); } - @Deprecated @Test public void shouldThrowConfigExceptionWhenInvalidWindowedInnerClassSerdeSupplied() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java index ecd2068f5fd09..4bca7f995a52a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java @@ -635,7 +635,6 @@ public static Properties getConfig(final String broker, final String application, final String id, final String processingGuarantee, - final String groupProtocol, final String stateDir) { final Properties properties = mkProperties( @@ -645,8 +644,7 @@ public static Properties getConfig(final String broker, mkEntry(StreamsConfig.CLIENT_ID_CONFIG, id), mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuarantee), mkEntry(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"), - mkEntry(StreamsConfig.STATE_DIR_CONFIG, stateDir), - mkEntry(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol) + mkEntry(StreamsConfig.STATE_DIR_CONFIG, stateDir) ) ); properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L); @@ -658,10 +656,9 @@ public static KafkaStreams startSync(final String broker, final String application, final String id, final String processingGuarantee, - final String groupProtocol, final String stateDir) throws InterruptedException { final KafkaStreams kafkaStreams = - new KafkaStreams(getTopology(), getConfig(broker, application, id, processingGuarantee, groupProtocol, stateDir)); + new KafkaStreams(getTopology(), getConfig(broker, application, id, processingGuarantee, stateDir)); final CountDownLatch startUpLatch = new CountDownLatch(1); kafkaStreams.setStateListener((newState, oldState) -> { if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { @@ -992,9 +989,8 @@ public static void main(final String[] args) { case "application": { final String nodeId = args[2]; final String processingGuarantee = args[3]; - final String groupProtocol = args[4]; - final String stateDir = args[5]; - App.startSync(kafka, UUID.randomUUID().toString(), nodeId, processingGuarantee, groupProtocol, stateDir); + final String stateDir = args[4]; + App.startSync(kafka, UUID.randomUUID().toString(), nodeId, processingGuarantee, stateDir); break; } default: diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTestTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTestTest.java index bbe7d2dad0fdf..4cf19de4c36ac 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTestTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTestTest.java @@ -45,7 +45,6 @@ public void verifySmokeTestLogic() { "test", "test", StreamsConfig.AT_LEAST_ONCE, - "classic", TestUtils.tempDirectory().getAbsolutePath() ))) { diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index 329df370d195a..ab02856c4b3e3 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -74,6 +74,7 @@ DELEGATION_TOKEN_SECRET_KEY="delegation.token.secret.key" SASL_ENABLED_MECHANISMS="sasl.enabled.mechanisms" +GROUP_COORDINATOR_REBALANCE_PROTOCOLS="group.coordinator.rebalance.protocols" CONSUMER_GROUP_MIGRATION_POLICY = "group.consumer.migration.policy" SHARE_COORDINATOR_STATE_TOPIC_REPLICATION_FACTOR ="share.coordinator.state.topic.replication.factor" @@ -81,7 +82,6 @@ SHARE_GROUP_ENABLE = "group.share.enable" UNSTABLE_API_VERSIONS_ENABLE = "unstable.api.versions.enable" -UNSTABLE_FEATURE_VERSIONS_ENABLE = "unstable.feature.versions.enable" """ From KafkaConfig.scala diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index c89eead4b7440..47023ff31c65a 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -784,12 +784,17 @@ def prop_file(self, node): for prop in self.per_node_server_prop_overrides.get(self.idx(node), []): override_configs[prop[0]] = prop[1] + enabledProtocols = 'classic,consumer' + if self.use_share_groups is not None and self.use_share_groups is True: override_configs[config_property.SHARE_GROUP_ENABLE] = str(self.use_share_groups) + enabledProtocols += ',share' if self.use_streams_groups is True: override_configs[config_property.UNSTABLE_API_VERSIONS_ENABLE] = str(True) - override_configs[config_property.UNSTABLE_FEATURE_VERSIONS_ENABLE] = str(True) + enabledProtocols += ',streams' + + override_configs[config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS] = enabledProtocols #update template configs with test override configs configs.update(override_configs) diff --git a/tests/kafkatest/tests/streams/base_streams_test.py b/tests/kafkatest/tests/streams/base_streams_test.py index 96ac192e6067f..00b4c37889705 100644 --- a/tests/kafkatest/tests/streams/base_streams_test.py +++ b/tests/kafkatest/tests/streams/base_streams_test.py @@ -47,7 +47,6 @@ def __init__(self, test_context, topics, num_controllers=1, num_brokers=3): def setUp(self): self.kafka.start() - self.kafka.run_features_command("upgrade", "streams.version", 1) def get_consumer(self, client_id, topic, num_messages): return VerifiableConsumer(self.test_context, diff --git a/tests/kafkatest/tests/streams/streams_relational_smoke_test.py b/tests/kafkatest/tests/streams/streams_relational_smoke_test.py index 72deeded059a1..c174948ef00e1 100644 --- a/tests/kafkatest/tests/streams/streams_relational_smoke_test.py +++ b/tests/kafkatest/tests/streams/streams_relational_smoke_test.py @@ -20,13 +20,13 @@ from kafkatest.services.kafka import quorum from kafkatest.services.kafka.util import get_log4j_config_param, get_log4j_config_for_tools from kafkatest.services.streams import StreamsTestBaseService -from kafkatest.tests.streams.base_streams_test import BaseStreamsTest +from kafkatest.tests.kafka_test import KafkaTest from kafkatest.version import LATEST_4_0 from kafkatest.version import get_version class StreamsRelationalSmokeTestService(StreamsTestBaseService): - def __init__(self, test_context, kafka, mode, nodeId, processing_guarantee, group_protocol): + def __init__(self, test_context, kafka, mode, nodeId, processing_guarantee): super(StreamsRelationalSmokeTestService, self).__init__( test_context, kafka, @@ -36,13 +36,12 @@ def __init__(self, test_context, kafka, mode, nodeId, processing_guarantee, grou self.mode = mode self.nodeId = nodeId self.processing_guarantee = processing_guarantee - self.group_protocol = group_protocol self.log4j_template = "log4j2_template.yaml" if (get_version(self.node) >= LATEST_4_0) else "log4j_template.properties" def start_cmd(self, node): return "( export KAFKA_LOG4J_OPTS=\"%(log4j_param)s%(log4j)s\"; " \ "INCLUDE_TEST_JARS=true %(kafka_run_class)s org.apache.kafka.streams.tests.RelationalSmokeTest " \ - " %(mode)s %(kafka)s %(nodeId)s %(processing_guarantee)s %(group_protocol)s %(state_dir)s" \ + " %(mode)s %(kafka)s %(nodeId)s %(processing_guarantee)s %(state_dir)s" \ " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % { "log4j_param": get_log4j_config_param(node), "log4j": get_log4j_config_for_tools(node), @@ -51,7 +50,6 @@ def start_cmd(self, node): "kafka": self.kafka.bootstrap_servers(), "nodeId": self.nodeId, "processing_guarantee": self.processing_guarantee, - "group_protocol": self.group_protocol, "state_dir": self.PERSISTENT_ROOT, "stdout": self.STDOUT_FILE, "stderr": self.STDERR_FILE, @@ -77,13 +75,13 @@ def await_command(self, command): ) -class StreamsRelationalSmokeTest(BaseStreamsTest): +class StreamsRelationalSmokeTest(KafkaTest): """ Simple test of Kafka Streams. """ def __init__(self, test_context): - super(StreamsRelationalSmokeTest, self).__init__(test_context, num_controllers=1, num_brokers=3, topics={ + super(StreamsRelationalSmokeTest, self).__init__(test_context, num_zk=1, num_brokers=3, topics={ 'in-article': {'partitions': 3, 'replication-factor': 1}, 'in-comment': {'partitions': 5, 'replication-factor': 1}, 'out-augmented-article': {'partitions': 3, 'replication-factor': 1}, @@ -93,16 +91,15 @@ def __init__(self, test_context): @cluster(num_nodes=8) @matrix(crash=[False, True], - metadata_quorum=[quorum.combined_kraft], - group_protocol=["classic", "streams"]) - def test_streams(self, crash, metadata_quorum, group_protocol): - driver = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "driver", "ignored", "ignored", "ignored") + metadata_quorum=[quorum.combined_kraft]) + def test_streams(self, crash, metadata_quorum): + driver = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "driver", "ignored", "ignored") processing_guarantee='exactly_once_v2' LOG_FILE = driver.LOG_FILE # this is the same for all instances of the service, so we can just declare a "constant" - processor1 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor1", processing_guarantee, group_protocol) - processor2 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor2", processing_guarantee, group_protocol) + processor1 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor1", processing_guarantee) + processor2 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor2", processing_guarantee) processor1.start() processor2.start() @@ -117,7 +114,7 @@ def test_streams(self, crash, metadata_quorum, group_protocol): processor1.stop_nodes(not crash) - processor3 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor3", processing_guarantee, group_protocol) + processor3 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor3", processing_guarantee) processor3.start() processor3.await_command("grep -q 'Streams has started' %s" % LOG_FILE) diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommand.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommand.java index cfe3fee581215..dcebff0d3ea62 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommand.java @@ -19,8 +19,6 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AbstractOptions; import org.apache.kafka.clients.admin.Admin; -import org.apache.kafka.clients.admin.DeleteShareGroupOffsetsOptions; -import org.apache.kafka.clients.admin.DeleteShareGroupOffsetsResult; import org.apache.kafka.clients.admin.DeleteShareGroupsOptions; import org.apache.kafka.clients.admin.DescribeShareGroupsOptions; import org.apache.kafka.clients.admin.GroupListing; @@ -35,7 +33,6 @@ import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.GroupNotEmptyException; -import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.util.CommandLineUtils; @@ -92,7 +89,7 @@ public static void run(ShareGroupCommandOptions opts) { } else if (opts.options.has(opts.resetOffsetsOpt)) { throw new UnsupportedOperationException("--reset-offsets option is not yet implemented"); } else if (opts.options.has(opts.deleteOffsetsOpt)) { - shareGroupService.deleteOffsets(); + throw new UnsupportedOperationException("--delete-offsets option is not yet implemented"); } } catch (IllegalArgumentException e) { CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage()); @@ -288,85 +285,6 @@ Map deleteShareGroups() { return failed; } - void deleteOffsets() { - String groupId = opts.options.valueOf(opts.groupOpt); - List topics = opts.options.valuesOf(opts.topicOpt); - - Entry> res = sendDeleteShareGroupOffsetsRequest(groupId, new HashSet<>(topics)); - - Throwable topLevelResult = res.getKey(); - Map topicLevelResult = res.getValue(); - - if (topLevelResult != null) { - Errors topLevelError = Errors.forException(topLevelResult); - switch (topLevelError) { - case INVALID_GROUP_ID: - case GROUP_ID_NOT_FOUND: - case GROUP_AUTHORIZATION_FAILED: - case NON_EMPTY_GROUP: - printError(topLevelResult.getMessage(), Optional.empty()); - break; - case TOPIC_AUTHORIZATION_FAILED: - case UNKNOWN_TOPIC_OR_PARTITION: - // These are expected topic-level errors which will be reported in the topic-level results - break; - default: - printError("Encounter some unknown error: " + topLevelResult, Optional.empty()); - } - } - - if (topicLevelResult != null && !topicLevelResult.isEmpty()) { - int maxTopicLen = 15; - for (String topic : topicLevelResult.keySet()) { - maxTopicLen = Math.max(maxTopicLen, topic.length()); - } - - String format = "%n%" + (-maxTopicLen) + "s %s"; - - System.out.printf(format, "TOPIC", "STATUS"); - topicLevelResult.entrySet().stream() - .sorted(Entry.comparingByKey()) - .forEach(e -> { - String topic = e.getKey(); - Throwable error = e.getValue(); - System.out.printf(format, - topic, - error != null ? "Error: " + error.getMessage() : "Successful" - ); - }); - } - - System.out.println(); - } - - Entry> sendDeleteShareGroupOffsetsRequest(String groupId, Set topics) { - Map topicLevelResult = new HashMap<>(); - - DeleteShareGroupOffsetsResult deleteResult = adminClient.deleteShareGroupOffsets( - groupId, - new HashSet<>(topics), - withTimeoutMs(new DeleteShareGroupOffsetsOptions())); - - Throwable topLevelException = null; - - try { - deleteResult.all().get(); - } catch (ExecutionException | InterruptedException e) { - topLevelException = e.getCause(); - } - - topics.forEach(topic -> { - try { - deleteResult.topicResult(topic).get(); - topicLevelResult.put(topic, null); - } catch (ExecutionException | InterruptedException e) { - topicLevelResult.put(topic, e.getCause()); - } - }); - - return new SimpleImmutableEntry<>(topLevelException, topicLevelResult); - } - private > T withTimeoutMs(T options) { int t = opts.options.valueOf(opts.timeoutMsOpt).intValue(); return options.timeoutMs(t); diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandOptions.java index be99d2946a727..3ba0a707ee58e 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandOptions.java @@ -145,7 +145,7 @@ public ShareGroupCommandOptions(String[] args) { .availableIf(describeOpt); allGroupSelectionScopeOpts = new HashSet<>(Arrays.asList(groupOpt, allGroupsOpt)); - allShareGroupLevelOpts = new HashSet<>(Arrays.asList(listOpt, describeOpt, deleteOpt, resetOffsetsOpt)); + allShareGroupLevelOpts = new HashSet<>(Arrays.asList(listOpt, describeOpt, deleteOpt, deleteOffsetsOpt, resetOffsetsOpt)); allResetOffsetScenarioOpts = new HashSet<>(Arrays.asList(resetToDatetimeOpt, resetToEarliestOpt, resetToLatestOpt)); allDeleteOffsetsOpts = new HashSet<>(Arrays.asList(groupOpt, topicOpt)); @@ -208,7 +208,7 @@ public void checkArgs() { } CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, minus(allGroupSelectionScopeOpts, groupOpt)); - CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, minus(allShareGroupLevelOpts, describeOpt, deleteOpt, resetOffsetsOpt)); + CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, minus(allShareGroupLevelOpts, describeOpt, deleteOpt, deleteOffsetsOpt, resetOffsetsOpt)); CommandLineUtils.checkInvalidArgs(parser, options, topicOpt, minus(allShareGroupLevelOpts, deleteOpt, resetOffsetsOpt)); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 1e96f1d361fea..5bb23cabdd98a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -97,7 +97,7 @@ private void setUpRemoteLogTopics() { Map rlsConfigs = new HashMap<>(); rlsConfigs.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"); rlsConfigs.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "1"); - rlsConfigs.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, "100"); + rlsConfigs.put(TopicConfig.SEGMENT_BYTES_CONFIG, "100"); setupTopics(this::getRemoteLogStorageEnabledTopicName, rlsConfigs); sendProducerRecords(this::getRemoteLogStorageEnabledTopicName); } diff --git a/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java index a17256908fb70..b834c7784a8b8 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java @@ -466,7 +466,7 @@ public void testListGroupsFailsWithException() { @SuppressWarnings({"NPathComplexity", "CyclomaticComplexity"}) @ClusterTest( serverProperties = { - @ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer,streams"), + @ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer,share,streams"), @ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), @ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1") } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java index f1f91217511ac..c3690d953ab37 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClientTestUtils; -import org.apache.kafka.clients.admin.DeleteShareGroupOffsetsResult; import org.apache.kafka.clients.admin.DeleteShareGroupsResult; import org.apache.kafka.clients.admin.DescribeShareGroupsOptions; import org.apache.kafka.clients.admin.DescribeShareGroupsResult; @@ -39,7 +38,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.GroupIdNotFoundException; import org.apache.kafka.common.internals.KafkaFutureImpl; -import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.test.TestUtils; import org.apache.kafka.tools.ToolsTestUtils; @@ -79,7 +77,6 @@ import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyList; -import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -615,204 +612,6 @@ public void testGroupStatesFromString() { assertThrows(IllegalArgumentException.class, () -> ShareGroupCommand.groupStatesFromString(" , ,")); } - @Test - public void testDeleteShareGroupOffsetsArgsWithoutTopic() { - String bootstrapServer = "localhost:9092"; - Admin adminClient = mock(KafkaAdminClient.class); - - // no group spec args - String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", "groupId"}; - AtomicBoolean exited = new AtomicBoolean(false); - Exit.setExitProcedure(((statusCode, message) -> { - assertNotEquals(0, statusCode); - assertTrue(message.contains("Option [delete-offsets] takes the following options: [topic], [group]")); - exited.set(true); - })); - try { - getShareGroupService(cgcArgs, adminClient); - } finally { - assertTrue(exited.get()); - } - } - - @Test - public void testDeleteShareGroupOffsetsArgsWithoutGroup() { - String bootstrapServer = "localhost:9092"; - Admin adminClient = mock(KafkaAdminClient.class); - - // no group spec args - String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServer, "--delete-offsets", "--topic", "t1"}; - AtomicBoolean exited = new AtomicBoolean(false); - Exit.setExitProcedure(((statusCode, message) -> { - assertNotEquals(0, statusCode); - assertTrue(message.contains("Option [delete-offsets] takes the following options: [topic], [group]")); - exited.set(true); - })); - try { - getShareGroupService(cgcArgs, adminClient); - } finally { - assertTrue(exited.get()); - } - } - - @Test - public void testDeleteShareGroupOffsets() throws Exception { - String firstGroup = "first-group"; - String firstTopic = "t1"; - String secondTopic = "t2"; - String bootstrapServer = "localhost:9092"; - - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); - Admin adminClient = mock(KafkaAdminClient.class); - DeleteShareGroupOffsetsResult result = mock(DeleteShareGroupOffsetsResult.class); - - when(result.all()).thenReturn(KafkaFuture.completedFuture(null)); - - when(result.topicResult(eq(firstTopic))).thenReturn(KafkaFuture.completedFuture(null)); - when(result.topicResult(eq(secondTopic))).thenReturn(KafkaFuture.completedFuture(null)); - - when(adminClient.deleteShareGroupOffsets(any(), any(), any())).thenReturn(result); - - try (ShareGroupService service = getShareGroupService(cgcArgs.toArray(new String[0]), adminClient)) { - TestUtils.waitForCondition(() -> { - Entry res = ToolsTestUtils.grabConsoleOutputAndError(deleteOffsets(service)); - String[] lines = res.getKey().trim().split("\n"); - if (lines.length != 3 && !res.getValue().isEmpty()) { - return false; - } - - List expectedResultHeader = List.of("TOPIC", "STATUS"); - List expectedResultValues1 = List.of(firstTopic, "Successful"); - List expectedResultValues2 = List.of(secondTopic, "Successful"); - - return Arrays.stream(lines[0].trim().split("\\s+")).toList().equals(expectedResultHeader) && - Arrays.stream(lines[1].trim().split("\\s+")).toList().equals(expectedResultValues1) && - Arrays.stream(lines[2].trim().split("\\s+")).toList().equals(expectedResultValues2); - }, "Expected a data row and no error in delete offsets result with group: " + firstGroup + " and topic: " + firstTopic); - } - } - - @Test - public void testDeleteShareGroupOffsetsMultipleGroups() { - String firstGroup = "first-group"; - String secondGroup = "second-group"; - String firstTopic = "t1"; - String secondTopic = "t2"; - String bootstrapServer = "localhost:9092"; - - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--group", secondGroup, "--topic", firstTopic, "--topic", secondTopic)); - Admin adminClient = mock(KafkaAdminClient.class); - - try (ShareGroupService service = getShareGroupService(cgcArgs.toArray(new String[0]), adminClient)) { - service.deleteOffsets(); - fail("Expected error was not detected while trying delete offsets multiple groups"); - } catch (Exception e) { - String expectedErrorMessage = "Found multiple arguments for option group, but you asked for only one"; - assertEquals(expectedErrorMessage, e.getMessage()); - } - } - - @Test - public void testDeleteShareGroupOffsetsTopLevelError() throws Exception { - String firstGroup = "first-group"; - String firstTopic = "t1"; - String secondTopic = "t2"; - String bootstrapServer = "localhost:9092"; - - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); - Admin adminClient = mock(KafkaAdminClient.class); - DeleteShareGroupOffsetsResult result = mock(DeleteShareGroupOffsetsResult.class); - - KafkaFutureImpl resultFuture = new KafkaFutureImpl<>(); - String errorMessage = "Group g3 not found."; - GroupIdNotFoundException exception = new GroupIdNotFoundException(errorMessage); - - resultFuture.completeExceptionally(exception); - when(result.all()).thenReturn(resultFuture); - - when(result.topicResult(eq(firstTopic))).thenReturn(resultFuture); - when(result.topicResult(eq(secondTopic))).thenReturn(resultFuture); - - when(adminClient.deleteShareGroupOffsets(any(), any(), any())).thenReturn(result); - - try (ShareGroupService service = getShareGroupService(cgcArgs.toArray(new String[0]), adminClient)) { - TestUtils.waitForCondition(() -> { - Entry res = ToolsTestUtils.grabConsoleOutputAndError(deleteOffsets(service)); - String[] lines = res.getKey().trim().split("\n"); - if (lines.length != 5 && !res.getValue().isEmpty()) { - return false; - } - - List error = Stream.concat( - Stream.of("Error:"), - Arrays.stream(errorMessage.trim().split("\\s+")) - ).toList(); - - List errorLine = new ArrayList<>(error); - List expectedResultHeader = List.of("TOPIC", "STATUS"); - List expectedResultValue1 = new ArrayList<>(); - expectedResultValue1.add(firstTopic); - expectedResultValue1.addAll(error); - List expectedResultValue2 = new ArrayList<>(); - expectedResultValue2.add(secondTopic); - expectedResultValue2.addAll(error); - - return Arrays.stream(lines[0].trim().split("\\s+")).toList().equals(errorLine) && - Arrays.stream(lines[2].trim().split("\\s+")).toList().equals(expectedResultHeader) && - Arrays.stream(lines[3].trim().split("\\s+")).toList().equals(expectedResultValue1) && - Arrays.stream(lines[4].trim().split("\\s+")).toList().equals(expectedResultValue2); - }, "Expected a data row and no error in delete offsets result with group: " + firstGroup + " and topic: " + firstTopic); - } - } - - @Test - public void testDeleteShareGroupOffsetsTopicLevelError() throws Exception { - String firstGroup = "first-group"; - String firstTopic = "t1"; - String secondTopic = "t2"; - String bootstrapServer = "localhost:9092"; - - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); - Admin adminClient = mock(KafkaAdminClient.class); - DeleteShareGroupOffsetsResult result = mock(DeleteShareGroupOffsetsResult.class); - - KafkaFutureImpl resultFuture = new KafkaFutureImpl<>(); - String errorMessage = Errors.UNKNOWN_TOPIC_OR_PARTITION.message(); - - resultFuture.completeExceptionally(Errors.UNKNOWN_TOPIC_OR_PARTITION.exception()); - when(result.all()).thenReturn(resultFuture); - - when(result.topicResult(eq(firstTopic))).thenReturn(KafkaFuture.completedFuture(null)); - when(result.topicResult(eq(secondTopic))).thenReturn(resultFuture); - - when(adminClient.deleteShareGroupOffsets(any(), any(), any())).thenReturn(result); - - try (ShareGroupService service = getShareGroupService(cgcArgs.toArray(new String[0]), adminClient)) { - TestUtils.waitForCondition(() -> { - Entry res = ToolsTestUtils.grabConsoleOutputAndError(deleteOffsets(service)); - String[] lines = res.getKey().trim().split("\n"); - if (lines.length != 5 && !res.getValue().isEmpty()) { - return false; - } - - List error = Stream.concat( - Stream.of("Error:"), - Arrays.stream(errorMessage.trim().split("\\s+")) - ).toList(); - - List expectedResultHeader = List.of("TOPIC", "STATUS"); - List expectedResultValue1 = List.of(firstTopic, "Successful"); - List expectedResultValue2 = new ArrayList<>(); - expectedResultValue2.add(secondTopic); - expectedResultValue2.addAll(error); - - return Arrays.stream(lines[0].trim().split("\\s+")).toList().equals(expectedResultHeader) && - Arrays.stream(lines[1].trim().split("\\s+")).toList().equals(expectedResultValue1) && - Arrays.stream(lines[2].trim().split("\\s+")).toList().equals(expectedResultValue2); - }, "Expected a data row and no error in delete offsets result with group: " + firstGroup + " and topic: " + firstTopic); - } - } - @Test public void testDeleteShareGroupsArgs() { String bootstrapServer = "localhost:9092"; @@ -1074,10 +873,6 @@ private Runnable describeGroups(ShareGroupCommand.ShareGroupService service) { return () -> Assertions.assertDoesNotThrow(service::describeGroups); } - private Runnable deleteOffsets(ShareGroupCommand.ShareGroupService service) { - return () -> Assertions.assertDoesNotThrow(service::deleteOffsets); - } - private boolean checkArgsHeaderOutput(List args, String output) { if (!output.contains("GROUP")) { return false; From 31ff8d0e6dcbd8a71aa58c18fa77b412fe679a87 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 2 May 2025 00:13:26 +0800 Subject: [PATCH 55/74] temp --- .../ConnectWorkerIntegrationTest.java | 4 +- .../scala/kafka/raft/KafkaMetadataLog.scala | 17 ++--- .../api/PlaintextAdminIntegrationTest.scala | 2 +- .../api/SaslSslAdminIntegrationTest.scala | 4 +- .../DynamicBrokerReconfigurationTest.scala | 2 +- .../kafka/raft/KafkaMetadataLogTest.scala | 12 +-- .../kafka/cluster/AbstractPartitionTest.scala | 2 +- .../kafka/cluster/PartitionLockTest.scala | 2 +- .../AbstractLogCleanerIntegrationTest.scala | 2 +- .../kafka/log/LogCleanerManagerTest.scala | 18 ++--- .../scala/unit/kafka/log/LogCleanerTest.scala | 73 ++++++++++--------- .../unit/kafka/log/LogConcurrencyTest.scala | 2 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 2 +- .../scala/unit/kafka/log/LogManagerTest.scala | 4 +- .../scala/unit/kafka/log/LogTestUtils.scala | 2 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 2 +- .../AlterReplicaLogDirsRequestTest.scala | 4 +- .../server/DynamicBrokerConfigTest.scala | 10 --- .../server/DynamicConfigChangeTest.scala | 6 +- .../kafka/tools/DumpLogSegmentsTest.scala | 1 - .../group/GroupCoordinatorServiceTest.java | 3 +- .../kafka/metadata/KafkaConfigSchema.java | 3 + .../apache/kafka/raft/MetadataLogConfig.java | 22 ++---- .../kafka/server/config/ServerLogConfigs.java | 3 + .../kafka/storage/internals/log/Cleaner.java | 2 +- .../storage/internals/log/LogConfig.java | 24 ++++-- .../storage/internals/log/UnifiedLog.java | 8 +- .../storage/internals/log/LocalLogTest.java | 3 +- .../PurgeRepartitionTopicIntegrationTest.java | 2 +- .../kafka/streams/StreamsConfigTest.java | 2 +- .../kafka/tools/GetOffsetShellTest.java | 3 +- 31 files changed, 117 insertions(+), 129 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java index ad8328deabe11..2f6de675219a9 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java @@ -43,6 +43,7 @@ import org.apache.kafka.connect.util.SinkUtils; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.connect.util.clusters.WorkerHandle; +import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -77,7 +78,6 @@ import static org.apache.kafka.clients.CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG; import static org.apache.kafka.common.config.AbstractConfig.CONFIG_PROVIDERS_CONFIG; import static org.apache.kafka.common.config.TopicConfig.DELETE_RETENTION_MS_CONFIG; -import static org.apache.kafka.common.config.TopicConfig.SEGMENT_MS_CONFIG; import static org.apache.kafka.connect.integration.BlockingConnectorTest.TASK_STOP; import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; @@ -1136,7 +1136,7 @@ public void testCompactedDeletedOlderConnectorConfig() throws Exception { final String configTopic = "kafka-16838-configs"; final int offsetCommitIntervalMs = 100; workerProps.put(CONFIG_TOPIC_CONFIG, configTopic); - workerProps.put(CONFIG_STORAGE_PREFIX + SEGMENT_MS_CONFIG, "100"); + workerProps.put(CONFIG_STORAGE_PREFIX + LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "100"); workerProps.put(CONFIG_STORAGE_PREFIX + DELETE_RETENTION_MS_CONFIG, "1"); workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, Integer.toString(offsetCommitIntervalMs)); diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index 01f69b374bc3d..db4841e3d22c9 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -588,7 +588,10 @@ object KafkaMetadataLog extends Logging { 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) + if (config.internalLogSegmentBytes != null) + props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) + else + props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT.toString) // Disable time and byte retention when deleting segments @@ -597,11 +600,7 @@ object KafkaMetadataLog extends Logging { LogConfig.validate(props) val defaultLogConfig = new LogConfig(props) - if (config.logSegmentBytes < config.logSegmentMinBytes) { - throw new InvalidConfigurationException( - s"Cannot set ${MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG} below ${config.logSegmentMinBytes}: ${config.logSegmentBytes}" - ) - } else if (defaultLogConfig.retentionMs >= 0) { + if (defaultLogConfig.retentionMs >= 0) { throw new InvalidConfigurationException( s"Cannot set ${TopicConfig.RETENTION_MS_CONFIG} above -1: ${defaultLogConfig.retentionMs}." ) @@ -637,12 +636,6 @@ object KafkaMetadataLog extends Logging { nodeId ) - // Print a warning if users have overridden the internal config - if (config.logSegmentMinBytes != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) { - metadataLog.error(s"Overriding ${MetadataLogConfig.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.") - } - // 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() diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 0ab2328c53f79..f0353009b6d1b 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -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 diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 66fca0db9badc..4ef32c2f2ebea 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -567,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(TopicConfig.SEGMENT_BYTES_CONFIG -> "3000000").asJava 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)) @@ -581,7 +581,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu val topicConfigs = result.config(topic1).get().entries.asScala assertTrue(topicConfigs.nonEmpty) val segmentBytesConfig = topicConfigs.find(_.name == TopicConfig.SEGMENT_BYTES_CONFIG).get - assertEquals(100000, segmentBytesConfig.value.toLong) + assertEquals(3000000, segmentBytesConfig.value.toLong) assertEquals(ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, segmentBytesConfig.source) val compressionConfig = topicConfigs.find(_.name == TopicConfig.COMPRESSION_TYPE_CONFIG).get assertEquals(ServerLogConfigs.COMPRESSION_TYPE_DEFAULT, compressionConfig.value) diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 3c5cd9396bdba..9d1f699f238ac 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -653,7 +653,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) => diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 1d3b3493cce8e..83914f63a4c98 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -40,10 +40,10 @@ 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.ArgumentsSource - import net.jqwik.api.AfterFailureMode import net.jqwik.api.ForAll import net.jqwik.api.Property +import org.apache.kafka.common.config.ConfigException import java.io.File import java.nio.ByteBuffer @@ -78,13 +78,13 @@ final class KafkaMetadataLogTest { props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240)) props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024)) - assertThrows(classOf[InvalidConfigurationException], () => { + assertThrows(classOf[ConfigException], () => { val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) }) - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) + props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10 * 1024 * 1024)) val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) @@ -689,7 +689,6 @@ final class KafkaMetadataLogTest { val recordSize = 64 val config = new MetadataLogConfig( DefaultMetadataLogConfig.logSegmentBytes, - DefaultMetadataLogConfig.logSegmentMinBytes, DefaultMetadataLogConfig.logSegmentMillis, DefaultMetadataLogConfig.retentionMaxBytes, DefaultMetadataLogConfig.retentionMillis, @@ -908,7 +907,6 @@ final class KafkaMetadataLogTest { @Test def testAdvanceLogStartOffsetAfterCleaning(): Unit = { val config = new MetadataLogConfig( - 512, 512, 10 * 1000, 256, @@ -945,7 +943,6 @@ final class KafkaMetadataLogTest { def testDeleteSnapshots(): Unit = { // Generate some logs and a few snapshots, set retention low and verify that cleaning occurs val config = new MetadataLogConfig( - 1024, 1024, 10 * 1000, 1024, @@ -979,7 +976,6 @@ final class KafkaMetadataLogTest { def testSoftRetentionLimit(): Unit = { // Set retention equal to the segment size and generate slightly more than one segment of logs val config = new MetadataLogConfig( - 10240, 10240, 10 * 1000, 10240, @@ -1023,7 +1019,6 @@ final class KafkaMetadataLogTest { @Test def testSegmentsLessThanLatestSnapshot(): Unit = { val config = new MetadataLogConfig( - 10240, 10240, 10 * 1000, 10240, @@ -1082,7 +1077,6 @@ object KafkaMetadataLogTest { } val DefaultMetadataLogConfig = new MetadataLogConfig( - 100 * 1024, 100 * 1024, 10 * 1000, 100 * 1024, diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index d9eaa7b2aacc0..fff1930a71836 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -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) } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 4a3051ddc9567..77b098cf68298 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -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) } diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index acf21e69ec377..b1e161b975340 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -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) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index f93d703f07777..8445baa7719fb 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -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) @@ -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) @@ -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)) @@ -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)) @@ -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)) @@ -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)) @@ -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)) @@ -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) @@ -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 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 09a9d1c40f140..6c3688f497794 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.errors.CorruptRecordException import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanedTransactionMetadata, Cleaner, CleanerConfig, CleanerStats, LocalLog, LogAppendInfo, LogCleaner, LogCleanerManager, LogCleaningAbortedException, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetMap, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} @@ -56,7 +57,7 @@ class LogCleanerTest extends Logging { val tmpdir = TestUtils.tempDir() val dir = TestUtils.randomPartitionLogDir(tmpdir) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_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 = new LogConfig(logProps) @@ -148,7 +149,7 @@ class LogCleanerTest extends Logging { def testCleanSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -181,7 +182,7 @@ class LogCleanerTest extends Logging { // Construct a log instance. The replaceSegments() method of the log instance is overridden so that // it waits for another thread to execute deleteOldSegments() val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) @@ -271,7 +272,7 @@ class LogCleanerTest extends Logging { val originalMaxFileSize = 1024 val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact": java.lang.String) logProps.put(TopicConfig.PREALLOCATE_CONFIG, "true": java.lang.String) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -295,7 +296,7 @@ class LogCleanerTest extends Logging { def testDuplicateCheckAfterCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) var log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -464,7 +465,7 @@ class LogCleanerTest extends Logging { def testBasicTransactionAwareCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -497,7 +498,7 @@ class LogCleanerTest extends Logging { def testCleanWithTransactionsSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -543,7 +544,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -591,7 +592,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(capacity = Int.MaxValue, maxMessageSize = 100) val logProps = new Properties() logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, 100: java.lang.Integer) - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -619,7 +620,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -682,7 +683,7 @@ class LogCleanerTest extends Logging { def testCleanEmptyControlBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -714,7 +715,7 @@ class LogCleanerTest extends Logging { def testCommittedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -736,7 +737,7 @@ class LogCleanerTest extends Logging { def testAbortedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -765,7 +766,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -799,7 +800,7 @@ class LogCleanerTest extends Logging { val producerId = 1L val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, 0, AppendOrigin.REPLICATION) @@ -832,7 +833,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -896,7 +897,7 @@ class LogCleanerTest extends Logging { // Create cleaner with very small default max message size val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -968,7 +969,7 @@ class LogCleanerTest extends Logging { def createLogWithMessagesLargerThanMaxSize(largeMessageSize: Int): (UnifiedLog, FakeOffsetMap) = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -994,7 +995,7 @@ class LogCleanerTest extends Logging { def testCleaningWithDeletes(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1021,7 +1022,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 3 messages in the map val cleaner = makeCleaner(4) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1050,7 +1051,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsProducerLastSequence(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) log.appendAsLeader(record(0, 0), 0) // offset 0 @@ -1073,7 +1074,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsLastSequenceEvenIfTransactionAborted(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -1107,7 +1108,7 @@ class LogCleanerTest extends Logging { def testCleaningWithKeysConflictingWithTxnMarkerKeys(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val leaderEpoch = 5 val producerEpoch = 0.toShort @@ -1151,7 +1152,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 1 message in the map val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1182,7 +1183,7 @@ class LogCleanerTest extends Logging { def testCleaningWithUncleanableSection(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1225,7 +1226,7 @@ class LogCleanerTest extends Logging { def testLogToClean(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1243,7 +1244,7 @@ class LogCleanerTest extends Logging { def testLogToCleanWithUncleanableSection(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1276,7 +1277,7 @@ class LogCleanerTest extends Logging { // create a log with compaction turned off so we can append unkeyed messages val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1334,7 +1335,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsWithAbort(): Unit = { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1356,7 +1357,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsRetainingLastEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1388,7 +1389,7 @@ class LogCleanerTest extends Logging { def testSegmentGrouping(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1489,7 +1490,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1541,7 +1542,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) //mimic the effect of loading an empty index file logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 400: java.lang.Integer) @@ -1666,7 +1667,7 @@ class LogCleanerTest extends Logging { def testRecoveryAfterCrash(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 10: java.lang.Integer) @@ -1797,7 +1798,7 @@ class LogCleanerTest extends Logging { def testBuildOffsetMapFakeLarge(): Unit = { val map = new FakeOffsetMap(1000) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) @@ -1945,7 +1946,7 @@ class LogCleanerTest extends Logging { @Test def testCleaningBeyondMissingOffsets(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) + logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) val cleaner = makeCleaner(Int.MaxValue) diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 0da8366f443ee..8c49978e6bd9f 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -60,7 +60,7 @@ class LogConcurrencyTest { @Test def testUncommittedDataNotConsumedFrequentSegmentRolls(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 237: Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 237: Integer) val logConfig = new LogConfig(logProps) testUncommittedDataNotConsumed(createLog(logConfig)) } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 8e417a695ee36..1ef21ad81f407 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -245,7 +245,7 @@ class LogLoaderTest { @Test def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "640") + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "640") val logConfig = new LogConfig(logProps) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 67880e0ced54c..04f4acca5debd 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -60,7 +60,7 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 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.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) val logConfig = new LogConfig(logProps) @@ -391,7 +391,7 @@ class LogManagerTest { logManager.shutdown() val segmentBytes = 10 * setSize val properties = new Properties() - properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes.toString) + properties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes.toString) properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString) val configRepository = MockConfigRepository.forTopic(name, properties) diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index 04c91741037bb..0ff68988d76fb 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -75,7 +75,7 @@ object LogTestUtils { remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes: Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs: java.lang.Long) logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 3f73f8f731a71..42c813074aa9a 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -2734,7 +2734,7 @@ class UnifiedLogTest { @Test def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000") + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000") logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") val logConfig = new LogConfig(logProps) diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index 8e2698b0842cf..78f5335caf42d 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AlterReplicaLogDirsRequest, AlterReplicaLogDirsResponse} import org.apache.kafka.server.config.ServerLogConfigs -import org.apache.kafka.storage.internals.log.LogFileUtils +import org.apache.kafka.storage.internals.log.{LogConfig, LogFileUtils} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -148,7 +148,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { // We don't want files with `.deleted` suffix are removed too fast, // so we can validate there will be orphan files and orphan files will be removed eventually. topicProperties.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, "10000") - topicProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1024") + topicProperties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1024") createTopic(topic, partitionNum, 1, topicProperties) assertEquals(logDir1, brokers.head.logManager.getLog(tp).get.dir.getParent) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 5f87b20d1f579..8bf29020eca30 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -670,16 +670,6 @@ class DynamicBrokerConfigTest { assertTrue(m.currentReporters.isEmpty) } - @Test - def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = { - val props = TestUtils.createBrokerConfig(0, port = 8181) - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, "1024") - val config = new KafkaConfig(props) - assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) - config.updateCurrentConfig(new KafkaConfig(props)) - assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) - } - @Test def testDynamicLogLocalRetentionMsConfig(): Unit = { val props = TestUtils.createBrokerConfig(0, port = 8181) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 28c8d694f9d51..cb3a40b9f89cc 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -106,14 +106,14 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { TestUtils.retry(10000) { val logOpt = this.brokers.head.logManager.getLog(tp) assertTrue(logOpt.isDefined) - assertEquals(oldSegmentSize, logOpt.get.config.segmentSize) + assertEquals(oldSegmentSize, logOpt.get.config.segmentSize()) } val newSegmentSize = 2000 val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(TopicConfig.SEGMENT_BYTES_CONFIG, newSegmentSize.toString), + val op = new AlterConfigOp(new ConfigEntry(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), OpType.SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get } finally { @@ -121,7 +121,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } val log = brokers.head.logManager.getLog(tp).get TestUtils.retry(10000) { - assertEquals(newSegmentSize, log.config.segmentSize) + assertEquals(newSegmentSize, log.config.segmentSize()) } (1 to 50).foreach(i => TestUtils.produceMessage(brokers, tp.topic, i.toString)) diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 5b3e9abe1112b..7227c72b9b94b 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -545,7 +545,6 @@ class DumpLogSegmentsTest { time, time.scheduler, new MetadataLogConfig( - 100 * 1024, 100 * 1024, 10 * 1000, 100 * 1024, diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 0fd4d1270e282..f84fcd627c9ac 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -109,6 +109,7 @@ import org.apache.kafka.server.share.persister.TopicData; import org.apache.kafka.server.util.FutureUtils; import org.apache.kafka.server.util.timer.MockTimer; +import org.apache.kafka.storage.internals.log.LogConfig; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -900,7 +901,7 @@ public void testGroupMetadataTopicConfigs() { Properties expectedProperties = new Properties(); expectedProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); expectedProperties.put(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.PRODUCER.name); - expectedProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000"); + expectedProperties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000"); assertEquals(expectedProperties, service.groupMetadataTopicConfigs()); } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java index f61c4c70e0680..13c1743ee5560 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java @@ -166,6 +166,9 @@ public Map resolveEffectiveTopicConfigs( ConfigDef configDef = configDefs.getOrDefault(ConfigResource.Type.TOPIC, EMPTY_CONFIG_DEF); HashMap effectiveConfigs = new HashMap<>(); for (ConfigDef.ConfigKey configKey : configDef.configKeys().values()) { + if (configKey.internalConfig && !dynamicTopicConfigs.containsKey(configKey.name)) { + continue; + } ConfigEntry entry = resolveEffectiveTopicConfig(configKey, staticNodeConfig, dynamicClusterConfigs, dynamicNodeConfigs, dynamicTopicConfigs); effectiveConfigs.put(entry.name(), entry); diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index b0a6f9f045ad6..805d16d361c7b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -52,10 +52,6 @@ public class MetadataLogConfig { "configuration. The Kafka node will generate a snapshot when either the maximum time interval is reached or the " + "maximum bytes limit is reached."; - public static final String METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG = "metadata.log.segment.min.bytes"; - public static final String METADATA_LOG_SEGMENT_MIN_BYTES_DOC = "Override the minimum size for a single metadata log file. This should be used for testing only."; - public static final int METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT = 8 * 1024 * 1024; - public static final String METADATA_LOG_SEGMENT_BYTES_CONFIG = "metadata.log.segment.bytes"; public static final String METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file."; public static final int METADATA_LOG_SEGMENT_BYTES_DEFAULT = 1024 * 1024 * 1024; @@ -85,14 +81,13 @@ public class MetadataLogConfig { .define(METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC) .define(METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, METADATA_LOG_DIR_DOC) .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) - .defineInternal(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_MIN_BYTES_DOC) .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); private final int logSegmentBytes; - private final int logSegmentMinBytes; + private final Integer internalLogSegmentBytes; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -103,7 +98,6 @@ public class MetadataLogConfig { /** * Configuration for the metadata log * @param logSegmentBytes The maximum size of a single metadata log file - * @param logSegmentMinBytes The minimum size of a single metadata log file * @param logSegmentMillis The maximum time before a new metadata log file is rolled out * @param retentionMaxBytes The size of the metadata log and snapshots before deleting old snapshots and log files * @param retentionMillis The time to keep a metadata log file or snapshot before deleting it @@ -112,7 +106,6 @@ public class MetadataLogConfig { * @param deleteDelayMillis The amount of time to wait before deleting a file from the filesystem */ public MetadataLogConfig(int logSegmentBytes, - int logSegmentMinBytes, long logSegmentMillis, long retentionMaxBytes, long retentionMillis, @@ -120,7 +113,7 @@ public MetadataLogConfig(int logSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = logSegmentBytes; - this.logSegmentMinBytes = logSegmentMinBytes; + this.internalLogSegmentBytes = logSegmentBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; @@ -131,7 +124,7 @@ public MetadataLogConfig(int logSegmentBytes, public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); - this.logSegmentMinBytes = config.getInt(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG); + this.internalLogSegmentBytes = config.getInt(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); @@ -141,11 +134,12 @@ public MetadataLogConfig(AbstractConfig config) { } public int logSegmentBytes() { - return logSegmentBytes; + if (internalLogSegmentBytes == null) return logSegmentBytes; + return internalLogSegmentBytes; } - - public int logSegmentMinBytes() { - return logSegmentMinBytes; + + public Integer internalLogSegmentBytes() { + return internalLogSegmentBytes; } public long logSegmentMillis() { diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 2c2d44ef012e5..16b5c7a99a39c 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -43,6 +43,9 @@ public class ServerLogConfigs { public static final String LOG_SEGMENT_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); public static final String LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file"; + public static final String INTERNAL_LOG_SEGMENT_BYTES_CONFIG = "internal." + ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); + public static final String INTERNAL_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; + public static final String LOG_ROLL_TIME_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_MS_CONFIG); public static final String LOG_ROLL_TIME_HOURS_CONFIG = LOG_PREFIX + "roll.hours"; public static final String LOG_ROLL_TIME_MILLIS_DOC = "The maximum time before a new log segment is rolled out (in milliseconds). If not set, the value in " + LOG_ROLL_TIME_HOURS_CONFIG + " is used"; diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java index b7d4ccedb2ec1..54ad6a4e79a97 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java @@ -171,7 +171,7 @@ public Map.Entry doClean(LogToClean cleanable, long currentT List> groupedSegments = groupSegmentsBySize( log.logSegments(0, endOffset), - log.config().segmentSize, + log.config().segmentSize(), log.config().maxIndexSize, cleanable.firstUncleanableOffset() ); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 21c92cd84dff4..88f7b54cd320c 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.record.LegacyRecord; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ConfigUtils; import org.apache.kafka.common.utils.Utils; @@ -140,6 +139,8 @@ public Optional serverConfigName(String configName) { public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs + public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; + // Visible for testing public static final Set CONFIGS_WITH_NO_SERVER_DEFAULTS = Set.of( TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, @@ -186,12 +187,13 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) - .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC); + .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) + .defineInternal(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { CONFIG. - define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), MEDIUM, + define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(1024 * 1024), MEDIUM, TopicConfig.SEGMENT_BYTES_DOC) .define(TopicConfig.SEGMENT_MS_CONFIG, LONG, DEFAULT_SEGMENT_MS, atLeast(1), MEDIUM, TopicConfig.SEGMENT_MS_DOC) .define(TopicConfig.SEGMENT_JITTER_MS_CONFIG, LONG, DEFAULT_SEGMENT_JITTER_MS, atLeast(0), MEDIUM, @@ -253,7 +255,8 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) - .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC); + .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) + .defineInternal(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_DOC); } public final Set overriddenConfigs; @@ -262,7 +265,8 @@ public Optional serverConfigName(String configName) { * Important note: Any configuration parameter that is passed along from KafkaConfig to LogConfig * should also be in `KafkaConfig#extractLogConfigMap`. */ - public final int segmentSize; + private final int segmentSize; + private final Integer internalSegmentSize; public final long segmentMs; public final long segmentJitterMs; public final int maxIndexSize; @@ -306,6 +310,7 @@ public LogConfig(Map props, Set overriddenConfigs) { this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); + this.internalSegmentSize = getInt(INTERNAL_SEGMENT_BYTES_CONFIG); this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); @@ -367,6 +372,11 @@ private Optional getCompression() { } } + public int segmentSize() { + if (internalSegmentSize == null) return segmentSize; + return internalSegmentSize; + } + // Exposed as a method so it can be mocked public int maxMessageSize() { return maxMessageSize; @@ -388,7 +398,7 @@ public long maxSegmentMs() { public int initFileSize() { if (preallocate) - return segmentSize; + return segmentSize(); else return 0; } @@ -628,7 +638,7 @@ public static void validate(Map existingConfigs, @Override public String toString() { return "LogConfig{" + - "segmentSize=" + segmentSize + + "segmentSize=" + segmentSize() + ", segmentMs=" + segmentMs + ", segmentJitterMs=" + segmentJitterMs + ", maxIndexSize=" + maxIndexSize + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index dcef6929d1902..ff689898ecd62 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1177,9 +1177,9 @@ private LogAppendInfo append(MemoryRecords records, }); // check messages size does not exceed config.segmentSize - if (validRecords.sizeInBytes() > config().segmentSize) { + if (validRecords.sizeInBytes() > config().segmentSize()) { throw new RecordBatchTooLargeException("Message batch size is " + validRecords.sizeInBytes() + " bytes in append " + - "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize + "."); + "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize() + "."); } // maybe roll the log if this segment is full @@ -2034,12 +2034,12 @@ private LogSegment maybeRoll(int messagesSize, LogAppendInfo appendInfo) throws long maxTimestampInMessages = appendInfo.maxTimestamp(); long maxOffsetInMessages = appendInfo.lastOffset(); - if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize, appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { + if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize(), appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { logger.debug("Rolling new log segment (log_size = {}/{}}, " + "offset_index_size = {}/{}, " + "time_index_size = {}/{}, " + "inactive_time_ms = {}/{}).", - segment.size(), config().segmentSize, + segment.size(), config().segmentSize(), segment.offsetIndex().entries(), segment.offsetIndex().maxEntries(), segment.timeIndex().entries(), segment.timeIndex().maxEntries(), segment.timeWaitedForRoll(now, maxTimestampInMessages), config().segmentMs - segment.rollJitterMs()); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index 7dfba8f7a5954..d58c520af0972 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; -import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -198,7 +197,7 @@ public void testUpdateConfig() { assertEquals(oldConfig, log.config()); Properties props = new Properties(); - props.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldConfig.segmentSize + 1); + props.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); LogConfig newConfig = new LogConfig(props); log.updateConfig(newConfig); assertEquals(newConfig, log.config()); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index 6023706847f15..21d87556f75df 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -171,7 +171,7 @@ public void setup() { streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); - streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); + streamsConfiguration.put(StreamsConfig.topicPrefix("internal.segment.bytes"), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size final StreamsBuilder builder = new StreamsBuilder(); diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index f081a768815fa..e726debccd274 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -248,7 +248,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { props.put(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, 99_999L); props.put(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, 7L); props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "dummy:host"); - props.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), 100); + props.put(StreamsConfig.topicPrefix("internal.segment.bytes"), 100); final StreamsConfig streamsConfig = new StreamsConfig(props); final Map returnedProps = streamsConfig.getMainConsumerConfigs(groupId, clientId, threadIdx); diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 5bb23cabdd98a..5ee3b38564032 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -40,6 +40,7 @@ import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig; import org.apache.kafka.server.log.remote.storage.LocalTieredStorage; import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig; +import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.test.TestUtils; import java.time.Duration; @@ -97,7 +98,7 @@ private void setUpRemoteLogTopics() { Map rlsConfigs = new HashMap<>(); rlsConfigs.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"); rlsConfigs.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "1"); - rlsConfigs.put(TopicConfig.SEGMENT_BYTES_CONFIG, "100"); + rlsConfigs.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "100"); setupTopics(this::getRemoteLogStorageEnabledTopicName, rlsConfigs); sendProducerRecords(this::getRemoteLogStorageEnabledTopicName); } From c01e9495f87685719bcfddc9999f87ba4d935037 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 10 May 2025 14:06:24 +0800 Subject: [PATCH 56/74] reset to trunk --- LICENSE-binary | 1 - build.gradle | 4 + ...port-control-clients-integration-tests.xml | 2 + checkstyle/import-control-server.xml | 4 + .../MetadataVersionIntegrationTest.java | 88 ++ ...etricsDuringTopicCreationDeletionTest.java | 147 +++ .../clients/TransactionsExpirationTest.java | 4 +- .../DescribeAuthorizedOperationsTest.java | 2 +- .../admin/RackAwareAutoTopicCreationTest.java | 4 +- .../clients/consumer/ShareConsumerTest.java | 187 ++- .../kafka/clients/CommonClientConfigs.java | 14 +- .../org/apache/kafka/clients/Metadata.java | 2 +- .../DescribeUserScramCredentialsResult.java | 2 +- .../kafka/clients/admin/KafkaAdminClient.java | 95 +- .../clients/admin/ListGroupsOptions.java | 22 + .../admin/ListShareGroupOffsetsResult.java | 11 +- .../admin/ListStreamsGroupOffsetsOptions.java | 2 +- .../admin/ListTransactionsOptions.java | 30 +- .../ListConsumerGroupOffsetsHandler.java | 68 +- .../ListShareGroupOffsetsHandler.java | 20 +- .../internals/ListTransactionsHandler.java | 3 + .../kafka/clients/consumer/CloseOptions.java | 21 +- .../kafka/clients/consumer/KafkaConsumer.java | 12 +- .../kafka/clients/consumer/MockConsumer.java | 2 +- .../AbstractHeartbeatRequestManager.java | 5 +- .../internals/AbstractMembershipManager.java | 60 +- .../internals/AsyncKafkaConsumer.java | 5 +- .../internals/CommitRequestManager.java | 96 +- .../internals/ConsumerCoordinator.java | 94 +- .../internals/ConsumerNetworkThread.java | 21 +- .../consumer/internals/RequestManagers.java | 36 +- .../consumer/internals/ShareConsumerImpl.java | 25 +- .../internals/ShareSessionHandler.java | 3 +- .../internals/TopicIdPartitionSet.java | 114 ++ .../kafka/common/config/TopicConfig.java | 10 +- .../apache/kafka/common/protocol/Errors.java | 2 +- .../kafka/common/record/FileRecords.java | 15 +- .../kafka/common/record/MemoryRecords.java | 25 + .../apache/kafka/common/record/Records.java | 16 + .../common/requests/ListGroupsRequest.java | 18 +- .../requests/ListTransactionsRequest.java | 4 + .../common/requests/OffsetFetchRequest.java | 6 +- .../common/requests/OffsetFetchResponse.java | 400 ++----- .../ReadShareGroupStateSummaryResponse.java | 2 + .../message/ListTransactionsRequest.json | 7 +- .../message/ListTransactionsResponse.json | 4 +- .../ReadShareGroupStateSummaryResponse.json | 2 + .../kafka/clients/InFlightRequestsTest.java | 2 +- .../kafka/clients/NetworkClientTest.java | 18 - .../clients/admin/AdminClientTestUtils.java | 4 +- .../DeleteConsumerGroupOffsetsResultTest.java | 2 +- .../clients/admin/KafkaAdminClientTest.java | 1021 +++++++++++++---- .../clients/admin/ListGroupsOptionsTest.java | 114 ++ .../kafka/clients/admin/MockAdminClient.java | 26 +- ...oveMembersFromConsumerGroupResultTest.java | 2 +- .../ListConsumerGroupOffsetsHandlerTest.java | 479 +++++--- .../ListTransactionsHandlerTest.java | 38 + .../clients/consumer/CloseOptionsTest.java | 52 + .../clients/consumer/KafkaConsumerTest.java | 29 +- .../internals/AbstractCoordinatorTest.java | 4 +- .../internals/AsyncKafkaConsumerTest.java | 30 +- .../internals/CommitRequestManagerTest.java | 104 +- .../internals/ConsumerCoordinatorTest.java | 111 +- .../ConsumerHeartbeatRequestManagerTest.java | 11 - .../internals/ConsumerNetworkThreadTest.java | 19 +- .../internals/FetchRequestManagerTest.java | 2 +- .../internals/ShareConsumerImplTest.java | 29 + .../internals/ShareMembershipManagerTest.java | 34 - .../internals/ShareSessionHandlerTest.java | 42 +- .../internals/TopicIdPartitionSetTest.java | 154 +++ .../internals/ProducerInterceptorsTest.java | 14 +- .../internals/RecordAccumulatorTest.java | 18 +- .../producer/internals/SenderTest.java | 2 +- .../internals/TransactionManagerTest.java | 4 +- .../kafka/common/network/NioEchoServer.java | 5 - .../common/network/SslTransportLayerTest.java | 12 +- .../kafka/common/record/FileRecordsTest.java | 24 +- .../record/MemoryRecordsBuilderTest.java | 29 - .../common/record/MemoryRecordsTest.java | 145 +++ .../requests/CreateAclsRequestTest.java | 3 - .../requests/DescribeAclsRequestTest.java | 3 - .../requests/DescribeAclsResponseTest.java | 6 - .../requests/OffsetFetchResponseTest.java | 591 ++++------ .../common/requests/RequestResponseTest.java | 35 +- .../SaslServerAuthenticatorTest.java | 4 +- .../common/security/ssl/SslFactoryTest.java | 13 +- .../org/apache/kafka/test/TestSslUtils.java | 4 +- .../java/org/apache/kafka/test/TestUtils.java | 15 +- .../clients/admin/FakeLocalMetadataStore.java | 2 +- .../ConnectWorkerIntegrationTest.java | 4 +- .../runtime/CoordinatorRuntimeTest.java | 10 +- .../java/kafka/server/TierStateMachine.java | 2 +- .../server/builders/KafkaApisBuilder.java | 10 +- .../kafka/server/share/DelayedShareFetch.java | 317 ++--- .../server/share/PendingRemoteFetches.java | 85 ++ .../kafka/server/share/ShareFetchUtils.java | 19 +- .../kafka/server/share/SharePartition.java | 158 ++- .../server/share/SharePartitionManager.java | 16 +- .../scala/kafka/admin/ConfigCommand.scala | 4 +- .../main/scala/kafka/cluster/Partition.scala | 2 +- .../transaction/TransactionCoordinator.scala | 103 +- .../transaction/TransactionLog.scala | 7 +- .../transaction/TransactionMetadata.scala | 182 +-- .../transaction/TransactionStateManager.scala | 42 +- .../scala/kafka/raft/KafkaMetadataLog.scala | 27 +- .../kafka/server/AbstractFetcherThread.scala | 2 +- .../server/AutoTopicCreationManager.scala | 8 +- .../scala/kafka/server/BrokerServer.scala | 76 +- .../kafka/server/DynamicBrokerConfig.scala | 71 +- .../kafka/server/ForwardingManager.scala | 9 +- .../server/ForwardingManagerMetrics.scala | 100 -- .../main/scala/kafka/server/KafkaApis.scala | 129 +-- .../metadata/BrokerMetadataPublisher.scala | 40 +- .../scala/kafka/tools/DumpLogSegments.scala | 2 +- .../server/share/DelayedShareFetchTest.java | 285 +++-- .../server/share/ShareFetchUtilsTest.java | 77 +- .../share/SharePartitionManagerTest.java | 133 ++- .../server/share/SharePartitionTest.java | 222 +++- .../AbstractAuthorizerIntegrationTest.scala | 2 + .../kafka/api/AuthorizerIntegrationTest.scala | 1014 ++++++++++++++-- .../kafka/api/IntegrationTestHarness.scala | 4 - .../api/PlaintextAdminIntegrationTest.scala | 2 +- .../kafka/api/PlaintextConsumerTest.scala | 19 + .../api/SaslSslAdminIntegrationTest.scala | 4 +- .../DynamicBrokerReconfigurationTest.scala | 2 +- .../MetadataVersionIntegrationTest.scala | 89 -- .../server/RaftClusterSnapshotTest.scala | 4 +- .../kafka/raft/KafkaMetadataLogTest.scala | 14 +- .../kafka/cluster/AbstractPartitionTest.scala | 2 +- .../kafka/cluster/PartitionLockTest.scala | 2 +- ...ransactionCoordinatorConcurrencyTest.scala | 16 +- .../TransactionCoordinatorTest.scala | 170 +-- .../transaction/TransactionLogTest.scala | 31 +- .../TransactionMarkerChannelManagerTest.scala | 11 +- ...onMarkerRequestCompletionHandlerTest.scala | 3 +- .../transaction/TransactionMetadataTest.scala | 132 +-- .../TransactionStateManagerTest.scala | 120 +- ...tricsDuringTopicCreationDeletionTest.scala | 156 --- .../AbstractLogCleanerIntegrationTest.scala | 2 +- .../kafka/log/LogCleanerManagerTest.scala | 18 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 73 +- .../unit/kafka/log/LogConcurrencyTest.scala | 2 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 2 +- .../scala/unit/kafka/log/LogManagerTest.scala | 4 +- .../scala/unit/kafka/log/LogTestUtils.scala | 2 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 2 +- .../AlterReplicaLogDirsRequestTest.scala | 4 +- .../server/AutoTopicCreationManagerTest.scala | 22 +- .../server/DynamicBrokerConfigTest.scala | 10 + .../server/DynamicConfigChangeTest.scala | 6 +- .../server/ForwardingManagerMetricsTest.scala | 114 -- .../kafka/server/ForwardingManagerTest.scala | 2 +- .../GroupCoordinatorBaseRequestTest.scala | 30 + .../unit/kafka/server/KafkaApisTest.scala | 568 ++++----- .../unit/kafka/server/KafkaConfigTest.scala | 2 - .../kafka/server/ReplicaManagerTest.scala | 2 +- .../ShareFetchAcknowledgeRequestTest.scala | 427 ++++--- .../ShareGroupDescribeRequestTest.scala | 9 +- .../ShareGroupHeartbeatRequestTest.scala | 91 +- .../epoch/OffsetsForLeaderEpochTest.scala | 2 +- .../BrokerMetadataPublisherTest.scala | 2 +- .../kafka/tools/DumpLogSegmentsTest.scala | 5 +- .../unit/kafka/utils/SchedulerTest.scala | 234 ---- .../group/GroupCoordinatorService.java | 1 + .../group/GroupMetadataManager.java | 7 +- .../group/modern/share/ShareGroupConfig.java | 14 +- .../group/GroupCoordinatorServiceTest.java | 3 +- .../group/GroupMetadataManagerTest.java | 57 +- .../modern/share/ShareGroupConfigTest.java | 1 - .../controller/OffsetControlManager.java | 2 +- .../controller/PartitionChangeBuilder.java | 8 +- .../kafka/controller/QuorumController.java | 2 +- .../metrics/ControllerMetricsChanges.java | 30 +- .../org/apache/kafka/image/MetadataImage.java | 2 +- .../kafka/image/MetadataProvenance.java | 2 +- .../org/apache/kafka/image/TopicDelta.java | 40 +- .../kafka/metadata/KafkaConfigSchema.java | 3 - .../kafka/metadata/PartitionRegistration.java | 11 +- .../controller/FeatureControlManagerTest.java | 2 +- .../controller/OffsetControlManagerTest.java | 2 +- .../QuorumControllerIntegrationTestUtils.java | 6 +- .../controller/QuorumControllerTest.java | 2 +- .../metrics/ControllerMetricsChangesTest.java | 17 + .../kafka/image/FakeSnapshotWriter.java | 2 +- .../apache/kafka/image/TopicsImageTest.java | 38 + .../image/loader/MetadataLoaderTest.java | 2 +- .../image/publisher/SnapshotEmitterTest.java | 2 +- .../metadata/PartitionRegistrationTest.java | 15 +- .../kafka/metadata/storage/FormatterTest.java | 33 + .../apache/kafka/metalog/LocalLogManager.java | 8 +- .../apache/kafka/raft/KafkaRaftClient.java | 10 +- .../apache/kafka/raft/MetadataLogConfig.java | 22 +- .../org/apache/kafka/raft/OffsetAndEpoch.java | 27 - .../org/apache/kafka/raft/QuorumState.java | 1 + .../org/apache/kafka/raft/RaftClient.java | 1 + .../java/org/apache/kafka/raft/RaftUtil.java | 1 + .../org/apache/kafka/raft/ReplicatedLog.java | 1 + .../kafka/raft/ValidOffsetAndEpoch.java | 2 + .../KRaftControlRecordStateMachine.java | 10 +- .../raft/internals/KafkaRaftMetrics.java | 2 +- .../raft/internals/RecordsBatchReader.java | 6 +- .../kafka/raft/internals/RecordsIterator.java | 16 +- .../kafka/snapshot/FileRawSnapshotReader.java | 2 +- .../kafka/snapshot/FileRawSnapshotWriter.java | 2 +- .../snapshot/NotifyingRawSnapshotWriter.java | 2 +- .../kafka/snapshot/RawSnapshotReader.java | 2 +- .../kafka/snapshot/RawSnapshotWriter.java | 2 +- .../kafka/snapshot/RecordsSnapshotReader.java | 8 +- .../kafka/snapshot/RecordsSnapshotWriter.java | 2 +- .../apache/kafka/snapshot/SnapshotPath.java | 2 +- .../apache/kafka/snapshot/SnapshotReader.java | 2 +- .../apache/kafka/snapshot/SnapshotWriter.java | 2 +- .../org/apache/kafka/snapshot/Snapshots.java | 27 +- .../kafka/raft/KafkaNetworkChannelTest.java | 1 + .../raft/KafkaRaftClientReconfigTest.java | 4 +- .../raft/KafkaRaftClientSnapshotTest.java | 1 + .../java/org/apache/kafka/raft/MockLog.java | 1 + .../org/apache/kafka/raft/MockLogTest.java | 1 + .../apache/kafka/raft/QuorumStateTest.java | 1 + .../kafka/raft/RaftClientTestContext.java | 1 + .../kafka/raft/RaftEventSimulationTest.java | 9 +- .../org/apache/kafka/raft/RaftUtilTest.java | 1 + .../apache/kafka/raft/ReplicatedCounter.java | 1 + .../kafka/raft/ValidOffsetAndEpochTest.java | 2 + .../KRaftControlRecordStateMachineTest.java | 2 +- .../raft/internals/KafkaRaftMetricsTest.java | 2 +- .../internals/RecordsBatchReaderTest.java | 7 +- .../raft/internals/RecordsIteratorTest.java | 12 +- .../kafka/snapshot/FileRawSnapshotTest.java | 2 +- .../kafka/snapshot/MockRawSnapshotReader.java | 2 +- .../kafka/snapshot/MockRawSnapshotWriter.java | 2 +- .../NotifyingRawSnapshotWriterTest.java | 2 +- .../snapshot/RecordsSnapshotWriterTest.java | 12 +- .../snapshot/SnapshotWriterReaderTest.java | 17 +- .../apache/kafka/snapshot/SnapshotsTest.java | 2 +- .../kafka/server/common/MetadataVersion.java | 11 +- .../kafka/server/common/OffsetAndEpoch.java | 41 +- .../kafka/server/common/ShareVersion.java | 2 +- .../kafka/server/config/ServerLogConfigs.java | 3 - .../persister/DefaultStatePersister.java | 2 + .../share/persister/NoOpStatePersister.java | 3 +- .../share/persister/PartitionFactory.java | 8 +- .../persister/PartitionStateSummaryData.java | 2 + .../ReadShareGroupStateSummaryResult.java | 3 +- .../persister/DefaultStatePersisterTest.java | 18 +- .../kafka/server/util/timer/MockTimer.java | 3 + .../kafka/server/AssignmentsManager.java | 2 +- .../metrics/ForwardingManagerMetrics.java | 138 +++ .../fetch/DelayedShareFetchGroupKey.java | 5 + .../share/session/ShareSessionCache.java | 35 +- .../metrics/ForwardingManagerMetricsTest.java | 129 +++ .../share/session/ShareSessionCacheTest.java | 46 +- .../kafka/server/util/SchedulerTest.java | 266 +++++ .../share/ShareCoordinatorService.java | 9 +- .../share/ShareCoordinatorShard.java | 2 + .../share/ShareCoordinatorShardTest.java | 1 + .../kafka/storage/internals/log/Cleaner.java | 2 +- .../storage/internals/log/LogConfig.java | 24 +- .../storage/internals/log/UnifiedLog.java | 34 +- .../storage/internals/log/LocalLogTest.java | 3 +- .../internals/log/RemoteIndexCacheTest.java | 6 +- .../AdjustStreamThreadCountTest.java | 32 +- .../PurgeRepartitionTopicIntegrationTest.java | 2 +- .../BatchingStateRestoreCallback.java | 3 +- .../assignors/StickyTaskAssignor.java | 4 +- .../kafka/streams/StreamsConfigTest.java | 2 +- .../SessionWindowedDeserializerTest.java | 30 +- .../SessionWindowedSerializerTest.java | 30 +- .../kstream/TimeWindowedDeserializerTest.java | 57 +- .../kstream/TimeWindowedSerializerTest.java | 30 +- .../internals/InternalTopicManagerTest.java | 91 +- .../processor/internals/ReadOnlyTaskTest.java | 49 +- ...OptionsColumnFamilyOptionsAdapterTest.java | 70 +- .../streams/tests/RelationalSmokeTest.java | 12 +- .../tests/RelationalSmokeTestTest.java | 1 + .../tests/StreamsStandByReplicaTest.java | 1 - .../kafka/common/test/ClusterInstance.java | 13 + tests/docker/ducker-ak | 25 +- .../services/kafka/config_property.py | 2 +- tests/kafkatest/services/kafka/kafka.py | 14 +- .../services/kafka/templates/kafka.properties | 4 - .../tests/client/share_consumer_test.py | 8 +- .../tests/streams/base_streams_test.py | 14 + .../streams/streams_relational_smoke_test.py | 25 +- .../streams/streams_standby_replica_test.py | 12 +- .../tools}/AdminCommandFailedException.java | 2 +- .../kafka/tools}/AdminOperationException.java | 2 +- .../org/apache/kafka/tools/ClusterTool.java | 4 +- .../kafka/tools/DeleteRecordsCommand.java | 2 - .../kafka/tools/LeaderElectionCommand.java | 2 - .../kafka/tools/ProducerPerformance.java | 15 +- .../org/apache/kafka/tools/TopicCommand.java | 2 - .../kafka/tools/TransactionsCommand.java | 6 + .../consumer/group/ConsumerGroupCommand.java | 30 +- .../consumer/group/ShareGroupCommand.java | 149 ++- .../group/ShareGroupCommandOptions.java | 4 +- .../reassign/ReassignPartitionsCommand.java | 4 +- .../tools/streams/StreamsGroupCommand.java | 8 +- .../kafka/tools/DeleteRecordsCommandTest.java | 2 - .../kafka/tools/FeatureCommandTest.java | 2 +- .../kafka/tools/GetOffsetShellTest.java | 3 +- .../apache/kafka/tools/GroupsCommandTest.java | 2 +- .../tools/LeaderElectionCommandErrorTest.java | 1 - .../tools/LeaderElectionCommandTest.java | 1 - .../kafka/tools/ProducerPerformanceTest.java | 75 ++ .../apache/kafka/tools/TopicCommandTest.java | 2 - .../kafka/tools/TransactionsCommandTest.java | 40 +- .../consumer/group/ListConsumerGroupTest.java | 4 + .../consumer/group/ShareGroupCommandTest.java | 218 +++- .../reassign/ReassignPartitionsUnitTest.java | 4 +- .../transaction/TransactionState.java | 164 +++ 311 files changed, 8217 insertions(+), 4956 deletions(-) create mode 100644 clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/MetadataVersionIntegrationTest.java create mode 100644 clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/MetricsDuringTopicCreationDeletionTest.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSet.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/ListGroupsOptionsTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/CloseOptionsTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSetTest.java create mode 100644 core/src/main/java/kafka/server/share/PendingRemoteFetches.java delete mode 100644 core/src/main/scala/kafka/server/ForwardingManagerMetrics.scala delete mode 100644 core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala delete mode 100644 core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala delete mode 100644 core/src/test/scala/unit/kafka/server/ForwardingManagerMetricsTest.scala delete mode 100644 core/src/test/scala/unit/kafka/utils/SchedulerTest.scala delete mode 100644 raft/src/main/java/org/apache/kafka/raft/OffsetAndEpoch.java create mode 100644 server/src/main/java/org/apache/kafka/server/metrics/ForwardingManagerMetrics.java create mode 100644 server/src/test/java/org/apache/kafka/server/metrics/ForwardingManagerMetricsTest.java create mode 100644 server/src/test/java/org/apache/kafka/server/util/SchedulerTest.java rename {server-common/src/main/java/org/apache/kafka/server/common => tools/src/main/java/org/apache/kafka/tools}/AdminCommandFailedException.java (96%) rename {server-common/src/main/java/org/apache/kafka/server/common => tools/src/main/java/org/apache/kafka/tools}/AdminOperationException.java (96%) create mode 100644 transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionState.java diff --git a/LICENSE-binary b/LICENSE-binary index 6175d3ed7d479..7a35e39889e64 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -212,7 +212,6 @@ License Version 2.0: - commons-lang3-3.12.0 - commons-logging-1.3.2 - commons-validator-1.9.0 -- error_prone_annotations-2.21.1 - jackson-annotations-2.16.2 - jackson-core-2.16.2 - jackson-databind-2.16.2 diff --git a/build.gradle b/build.gradle index 2448bd3eaa2be..d2ac9aa1dc45f 100644 --- a/build.gradle +++ b/build.gradle @@ -911,6 +911,7 @@ project(':server') { testImplementation project(':test-common:test-common-internal-api') testImplementation project(':test-common:test-common-runtime') testImplementation project(':storage:storage-api').sourceSets.test.output + testImplementation project(':server-common').sourceSets.test.output testRuntimeOnly runtimeTestLibs } @@ -1020,6 +1021,7 @@ project(':core') { implementation libs.scalaReflect implementation libs.scalaLogging implementation libs.slf4jApi + implementation libs.re2j testImplementation project(':clients').sourceSets.test.output testImplementation project(':group-coordinator').sourceSets.test.output @@ -1969,6 +1971,7 @@ project(':clients:clients-integration-tests') { } dependencies { + testImplementation libs.metrics testImplementation libs.slf4jApi testImplementation project(':test-common:test-common-internal-api') testImplementation project(':test-common:test-common-runtime') @@ -2232,6 +2235,7 @@ project(':storage') { implementation project(':clients') implementation(libs.caffeine) { exclude group: 'org.checkerframework', module: 'checker-qual' + exclude group: 'com.google.errorprone', module: 'error_prone_annotations' } implementation libs.slf4jApi implementation libs.jacksonDatabind diff --git a/checkstyle/import-control-clients-integration-tests.xml b/checkstyle/import-control-clients-integration-tests.xml index 44cf0dba1fbec..752dcace7f254 100644 --- a/checkstyle/import-control-clients-integration-tests.xml +++ b/checkstyle/import-control-clients-integration-tests.xml @@ -28,6 +28,8 @@ + + diff --git a/checkstyle/import-control-server.xml b/checkstyle/import-control-server.xml index 30a7f5fbe761f..2f96777351a28 100644 --- a/checkstyle/import-control-server.xml +++ b/checkstyle/import-control-server.xml @@ -100,6 +100,10 @@ + + + + diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/MetadataVersionIntegrationTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/MetadataVersionIntegrationTest.java new file mode 100644 index 0000000000000..4e44eef5a216e --- /dev/null +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/MetadataVersionIntegrationTest.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients; + +import org.apache.kafka.clients.admin.FeatureUpdate; +import org.apache.kafka.clients.admin.UpdateFeaturesOptions; +import org.apache.kafka.common.test.ClusterInstance; +import org.apache.kafka.common.test.TestUtils; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.ClusterTests; +import org.apache.kafka.common.test.api.Type; +import org.apache.kafka.server.common.MetadataVersion; + +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class MetadataVersionIntegrationTest { + @ClusterTests(value = { + @ClusterTest(types = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_3_IV3), + @ClusterTest(types = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_4_IV0), + @ClusterTest(types = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_5_IV0), + @ClusterTest(types = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_6_IV0), + @ClusterTest(types = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_6_IV1) + }) + public void testBasicMetadataVersionUpgrade(ClusterInstance clusterInstance) throws Exception { + try (var admin = clusterInstance.admin()) { + var describeResult = admin.describeFeatures(); + var ff = describeResult.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME); + assertEquals(clusterInstance.config().metadataVersion().featureLevel(), ff.minVersionLevel()); + assertEquals(clusterInstance.config().metadataVersion().featureLevel(), ff.maxVersionLevel()); + + // Update to new version + short updateVersion = MetadataVersion.IBP_3_7_IV1.featureLevel(); + var updateResult = admin.updateFeatures( + Map.of("metadata.version", new FeatureUpdate(updateVersion, FeatureUpdate.UpgradeType.UPGRADE)), + new UpdateFeaturesOptions()); + updateResult.all().get(); + + // Verify that new version is visible on broker + TestUtils.waitForCondition(() -> { + try { + var describeResult2 = admin.describeFeatures(); + var ff2 = describeResult2.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME); + return ff2.minVersionLevel() == updateVersion && ff2.maxVersionLevel() == updateVersion; + } catch (Exception e) { + return false; + } + }, "Never saw metadata.version increase on broker"); + } + } + + @ClusterTest(types = Type.KRAFT, metadataVersion = MetadataVersion.IBP_3_9_IV0) + public void testUpgradeSameVersion(ClusterInstance clusterInstance) throws Exception { + try (var admin = clusterInstance.admin()) { + short updateVersion = MetadataVersion.IBP_3_9_IV0.featureLevel(); + var updateResult = admin.updateFeatures( + Map.of("metadata.version", new FeatureUpdate(updateVersion, FeatureUpdate.UpgradeType.UPGRADE)), + new UpdateFeaturesOptions()); + updateResult.all().get(); + } + } + + @ClusterTest(types = Type.KRAFT) + public void testDefaultIsLatestVersion(ClusterInstance clusterInstance) throws Exception { + try (var admin = clusterInstance.admin()) { + var describeResult = admin.describeFeatures(); + var ff = describeResult.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME); + assertEquals(MetadataVersion.latestTesting().featureLevel(), ff.minVersionLevel(), + "If this test fails, check the default MetadataVersion in the @ClusterTest annotation"); + assertEquals(MetadataVersion.latestTesting().featureLevel(), ff.maxVersionLevel()); + } + } +} diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/MetricsDuringTopicCreationDeletionTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/MetricsDuringTopicCreationDeletionTest.java new file mode 100644 index 0000000000000..32a8e5192f9ba --- /dev/null +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/MetricsDuringTopicCreationDeletionTest.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients; + +import org.apache.kafka.common.test.ClusterInstance; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.Type; +import org.apache.kafka.server.config.ReplicationConfigs; +import org.apache.kafka.server.config.ServerConfigs; +import org.apache.kafka.server.config.ServerLogConfigs; +import org.apache.kafka.server.metrics.KafkaYammerMetrics; + +import com.yammer.metrics.core.Gauge; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class MetricsDuringTopicCreationDeletionTest { + + private static final String TOPIC_NAME_PREFIX = "topic"; + private static final int TOPIC_NUM = 2; + private static final int CREATE_DELETE_ITERATIONS = 3; + private static final short REPLICATION_FACTOR = 1; + private static final int PARTITION_NUM = 3; + + private final ClusterInstance clusterInstance; + private final List topics; + private volatile boolean running = true; + + public MetricsDuringTopicCreationDeletionTest(ClusterInstance clusterInstance) { + this.clusterInstance = clusterInstance; + this.topics = new ArrayList<>(); + for (int n = 0; n < TOPIC_NUM; n++) { + topics.add(TOPIC_NAME_PREFIX + n); + } + } + + /* + * Checking all metrics we care in a single test is faster though it would be more elegant to have 3 @Test methods + */ + @ClusterTest( + types = {Type.KRAFT}, + brokers = 1, + serverProperties = { + @ClusterConfigProperty(key = ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, value = "true"), + @ClusterConfigProperty(key = "log.initial.task.delay.ms", value = "100"), + @ClusterConfigProperty(key = "log.segment.delete.delay.ms", value = "1000"), + @ClusterConfigProperty(key = ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, value = "false"), + // speed up the test for UnderReplicatedPartitions, which relies on the ISR expiry thread to execute concurrently with topic creation + // But the replica.lag.time.max.ms value still need to consider the slow testing environment + @ClusterConfigProperty(key = ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, value = "4000") + } + ) + public void testMetricsDuringTopicCreateDelete() throws Exception { + + final int initialOfflinePartitionsCount = getGauge("OfflinePartitionsCount").value(); + final int initialPreferredReplicaImbalanceCount = getGauge("PreferredReplicaImbalanceCount").value(); + final int initialUnderReplicatedPartitionsCount = getGauge("UnderReplicatedPartitions").value(); + + CompletableFuture future = CompletableFuture.runAsync(() -> { + while (running) { + int offlinePartitionsCount = getGauge("OfflinePartitionsCount").value(); + int preferredReplicaImbalanceCount = getGauge("PreferredReplicaImbalanceCount").value(); + int underReplicatedPartitionsCount = getGauge("UnderReplicatedPartitions").value(); + + if (offlinePartitionsCount != initialOfflinePartitionsCount || + preferredReplicaImbalanceCount != initialPreferredReplicaImbalanceCount || + underReplicatedPartitionsCount != initialUnderReplicatedPartitionsCount) { + running = false; + } + + try { + // Avoid busy loop + TimeUnit.MILLISECONDS.sleep(100); + } catch (InterruptedException ignored) { + + } + } + }); + + Closeable runThread = () -> { + running = false; + future.join(); + }; + + try (runThread) { + createAndDeleteTopics(); + } + + final int finalOfflinePartitionsCount = getGauge("OfflinePartitionsCount").value(); + final int finalPreferredReplicaImbalanceCount = getGauge("PreferredReplicaImbalanceCount").value(); + final int finalUnderReplicatedPartitionsCount = getGauge("UnderReplicatedPartitions").value(); + + assertEquals(initialOfflinePartitionsCount, finalOfflinePartitionsCount, + "Expect offlinePartitionsCount to be " + initialOfflinePartitionsCount + ", but got: " + finalOfflinePartitionsCount); + assertEquals(initialPreferredReplicaImbalanceCount, finalPreferredReplicaImbalanceCount, + "Expect PreferredReplicaImbalanceCount to be " + initialPreferredReplicaImbalanceCount + ", but got: " + finalPreferredReplicaImbalanceCount); + assertEquals(initialUnderReplicatedPartitionsCount, finalUnderReplicatedPartitionsCount, + "Expect UnderReplicatedPartitionCount to be " + initialUnderReplicatedPartitionsCount + ", but got: " + finalUnderReplicatedPartitionsCount); + } + + private void createAndDeleteTopics() { + for (int i = 1; i <= CREATE_DELETE_ITERATIONS && running; i++) { + for (String topic : topics) { + if (!running) return; + try { + clusterInstance.createTopic(topic, PARTITION_NUM, REPLICATION_FACTOR); + } catch (Exception ignored) { } + } + + for (String topic : topics) { + if (!running) return; + try { + clusterInstance.deleteTopic(topic); + } catch (Exception ignored) { } + } + } + } + + private Gauge getGauge(String metricName) { + return KafkaYammerMetrics.defaultRegistry().allMetrics().entrySet().stream() + .filter(entry -> entry.getKey().getName().endsWith(metricName)) + .findFirst() + .map(entry -> (Gauge) entry.getValue()) + .orElseThrow(() -> new AssertionError("Unable to find metric " + metricName)); + } +} diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/TransactionsExpirationTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/TransactionsExpirationTest.java index adb55f965afae..1c523a435545b 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/TransactionsExpirationTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/TransactionsExpirationTest.java @@ -165,8 +165,8 @@ private void testFatalErrorAfterInvalidProducerIdMapping(ClusterInstance cluster private void testTransactionAfterProducerIdExpires(ClusterInstance clusterInstance, boolean isTV2Enabled) throws InterruptedException { clusterInstance.createTopic(TOPIC1, 4, (short) 3); - long oldProducerId = 0; - long oldProducerEpoch = 0; + long oldProducerId; + long oldProducerEpoch; try (Producer producer = clusterInstance.producer(Map.of( ProducerConfig.TRANSACTIONAL_ID_CONFIG, TRANSACTION_ID diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/DescribeAuthorizedOperationsTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/DescribeAuthorizedOperationsTest.java index d4ad50a8f74b6..ef7b89a830d17 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/DescribeAuthorizedOperationsTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/DescribeAuthorizedOperationsTest.java @@ -113,7 +113,7 @@ private void setupSecurity(ClusterInstance clusterInstance) throws ExecutionExce public void testConsumerGroupAuthorizedOperations(ClusterInstance clusterInstance) throws ExecutionException, InterruptedException { setupSecurity(clusterInstance); try (Admin admin = clusterInstance.admin(createAdminConfig(JaasUtils.KAFKA_PLAIN_ADMIN, JaasUtils.KAFKA_PLAIN_ADMIN_PASSWORD)); - Admin user1 = clusterInstance.admin(createAdminConfig(JaasUtils.KAFKA_PLAIN_USER1, JaasUtils.KAFKA_PLAIN_USER1_PASSWORD)); + Admin user1 = clusterInstance.admin(createAdminConfig(JaasUtils.KAFKA_PLAIN_USER1, JaasUtils.KAFKA_PLAIN_USER1_PASSWORD)) ) { admin.createTopics(List.of(new NewTopic("topic1", 1, (short) 1))); clusterInstance.waitForTopic("topic1", 1); diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/RackAwareAutoTopicCreationTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/RackAwareAutoTopicCreationTest.java index e3b9efb9b6f94..d394dcb627410 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/RackAwareAutoTopicCreationTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/admin/RackAwareAutoTopicCreationTest.java @@ -112,9 +112,7 @@ private static Map> getTopicAssignment(Admin admin) throw private static Map getBrokerToRackMap(ClusterInstance cluster) throws Exception { Map actualBrokerToRackMap = new HashMap<>(); try (Admin admin = cluster.admin()) { - admin.describeCluster().nodes().get().forEach(node -> { - actualBrokerToRackMap.put(node.id(), node.rack()); - }); + admin.describeCluster().nodes().get().forEach(node -> actualBrokerToRackMap.put(node.id(), node.rack())); } return actualBrokerToRackMap; } diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java index d48f620880ef3..7bb348143d146 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java @@ -36,6 +36,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.errors.GroupMaxSizeReachedException; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.InvalidRecordStateException; @@ -55,7 +56,6 @@ import org.apache.kafka.common.test.api.ClusterConfigProperty; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.coordinator.group.GroupConfig; import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig; @@ -113,8 +113,6 @@ @ClusterTestDefaults( serverProperties = { @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), - @ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - @ClusterConfigProperty(key = "group.share.enable", value = "true"), @ClusterConfigProperty(key = "group.share.partition.max.record.locks", value = "10000"), @ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000"), @ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), @@ -123,8 +121,7 @@ @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), @ClusterConfigProperty(key = "transaction.state.log.min.isr", value = "1"), @ClusterConfigProperty(key = "transaction.state.log.replication.factor", value = "1") - }, - types = {Type.KRAFT} + } ) public class ShareConsumerTest { private final ClusterInstance cluster; @@ -1712,11 +1709,9 @@ public void testShareAutoOffsetResetEarliest() { public void testShareAutoOffsetResetEarliestAfterLsoMovement() { alterShareAutoOffsetReset("group1", "earliest"); try ( - ShareConsumer shareConsumer = createShareConsumer("group1"); Producer producer = createProducer(); Admin adminClient = createAdminClient() ) { - shareConsumer.subscribe(Set.of(tp.topic())); ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); // We write 10 records to the topic, so they would be written from offsets 0-9 on the topic. @@ -1814,8 +1809,7 @@ public void testShareAutoOffsetResetByDuration() throws Exception { // Set the auto offset reset to 3 hours before current time // so the consumer should consume all messages (3 records) alterShareAutoOffsetReset("group2", "by_duration:PT3H"); - try (ShareConsumer shareConsumer = createShareConsumer("group2"); - Producer producer = createProducer()) { + try (ShareConsumer shareConsumer = createShareConsumer("group2")) { shareConsumer.subscribe(Set.of(tp.topic())); List> records = consumeRecords(shareConsumer, 3); @@ -1825,7 +1819,7 @@ public void testShareAutoOffsetResetByDuration() throws Exception { } @ClusterTest - public void testShareAutoOffsetResetByDurationInvalidFormat() throws Exception { + public void testShareAutoOffsetResetByDurationInvalidFormat() { // Test invalid duration format ConfigResource configResource = new ConfigResource(ConfigResource.Type.GROUP, "group1"); Map> alterEntries = new HashMap<>(); @@ -1851,8 +1845,6 @@ public void testShareAutoOffsetResetByDurationInvalidFormat() throws Exception { brokers = 3, serverProperties = { @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), - @ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - @ClusterConfigProperty(key = "group.share.enable", value = "true"), @ClusterConfigProperty(key = "group.share.partition.max.record.locks", value = "10000"), @ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000"), @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "3"), @@ -1973,10 +1965,7 @@ public void testShareConsumerAfterCoordinatorMovement() throws Exception { ); // top the producer after some time (but after coordinator shutdown) - service.schedule(() -> { - prodState.done().set(true); - }, 10L, TimeUnit.SECONDS - ); + service.schedule(() -> prodState.done().set(true), 10L, TimeUnit.SECONDS); // wait for both producer and consumer to finish TestUtils.waitForCondition( @@ -1998,12 +1987,108 @@ public void testShareConsumerAfterCoordinatorMovement() throws Exception { verifyShareGroupStateTopicRecordsProduced(); } + @ClusterTest + public void testDeliveryCountNotIncreaseAfterSessionClose() { + alterShareAutoOffsetReset("group1", "earliest"); + try (Producer producer = createProducer()) { + ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); + // We write 10 records to the topic, so they would be written from offsets 0-9 on the topic. + for (int i = 0; i < 10; i++) { + assertDoesNotThrow(() -> producer.send(record).get(), "Failed to send records"); + } + } + + // Perform the fetch, close in a loop. + for (int count = 0; count < ShareGroupConfig.SHARE_GROUP_DELIVERY_COUNT_LIMIT_DEFAULT; count++) { + consumeMessages(new AtomicInteger(0), 10, "group1", 1, 10, false); + } + + // If the delivery count is increased, consumer will get nothing. + int consumedMessageCount = consumeMessages(new AtomicInteger(0), 10, "group1", 1, 10, true); + // The records returned belong to offsets 0-9. + assertEquals(10, consumedMessageCount); + verifyShareGroupStateTopicRecordsProduced(); + } + + @ClusterTest + public void testDeliveryCountDifferentBehaviorWhenClosingSessionWithExplicitAcknowledgement() { + alterShareAutoOffsetReset("group1", "earliest"); + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer( + "group1", + Map.of(ConsumerConfig.SHARE_ACKNOWLEDGEMENT_MODE_CONFIG, EXPLICIT))) { + + ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, + "key".getBytes(), "value".getBytes()); + producer.send(record); + producer.send(record); + producer.flush(); + + shareConsumer.subscribe(Set.of(tp.topic())); + ConsumerRecords records = waitedPoll(shareConsumer, 2500L, 2); + assertEquals(2, records.count()); + // Acknowledge the first record with AcknowledgeType.RELEASE + shareConsumer.acknowledge(records.records(tp).get(0), AcknowledgeType.RELEASE); + Map> result = shareConsumer.commitSync(); + assertEquals(1, result.size()); + } + + // Test delivery count + try (ShareConsumer shareConsumer = createShareConsumer("group1", Map.of())) { + shareConsumer.subscribe(Set.of(tp.topic())); + ConsumerRecords records = waitedPoll(shareConsumer, 2500L, 2); + assertEquals(2, records.count()); + assertEquals((short) 2, records.records(tp).get(0).deliveryCount().get()); + assertEquals((short) 1, records.records(tp).get(1).deliveryCount().get()); + } + } + + @ClusterTest( + serverProperties = { + @ClusterConfigProperty(key = "group.share.delivery.count.limit", value = "2"), + } + ) + public void testBehaviorOnDeliveryCountBoundary() { + alterShareAutoOffsetReset("group1", "earliest"); + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer( + "group1", + Map.of(ConsumerConfig.SHARE_ACKNOWLEDGEMENT_MODE_CONFIG, EXPLICIT))) { + + ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, + "key".getBytes(), "value".getBytes()); + producer.send(record); + producer.flush(); + + shareConsumer.subscribe(Set.of(tp.topic())); + ConsumerRecords records = waitedPoll(shareConsumer, 2500L, 1); + assertEquals(1, records.count()); + assertEquals((short) 1, records.records(tp).get(0).deliveryCount().get()); + // Acknowledge the record with AcknowledgeType.RELEASE. + shareConsumer.acknowledge(records.records(tp).get(0), AcknowledgeType.RELEASE); + Map> result = shareConsumer.commitSync(); + assertEquals(1, result.size()); + + // Consume again, the delivery count should be 2. + records = waitedPoll(shareConsumer, 2500L, 1); + assertEquals(1, records.count()); + assertEquals((short) 2, records.records(tp).get(0).deliveryCount().get()); + + } + + // Start again and same record should be delivered + try (ShareConsumer shareConsumer = createShareConsumer("group1", Map.of())) { + shareConsumer.subscribe(Set.of(tp.topic())); + ConsumerRecords records = waitedPoll(shareConsumer, 2500L, 1); + assertEquals(1, records.count()); + assertEquals((short) 2, records.records(tp).get(0).deliveryCount().get()); + } + } + @ClusterTest( brokers = 3, serverProperties = { @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), - @ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - @ClusterConfigProperty(key = "group.share.enable", value = "true"), @ClusterConfigProperty(key = "group.share.partition.max.record.locks", value = "10000"), @ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000"), @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "3"), @@ -2050,21 +2135,70 @@ public void testComplexShareConsumer() throws Exception { ); // let the complex consumer read the messages - service.schedule(() -> { - prodState.done().set(true); - }, 10L, TimeUnit.SECONDS - ); + service.schedule(() -> prodState.done().set(true), 10L, TimeUnit.SECONDS); // all messages which can be read are read, some would be redelivered TestUtils.waitForCondition(complexCons1::isDone, 45_000L, () -> "did not close!"); - assertTrue(prodState.count().get() < complexCons1.recordsRead()); + assertTrue(prodState.count().get() < complexCons1.recordsRead(), + String.format("Producer (%d) and share consumer (%d) record count mismatch.", prodState.count().get(), complexCons1.recordsRead())); shutdownExecutorService(service); verifyShareGroupStateTopicRecordsProduced(); } + @ClusterTest( + brokers = 1, + serverProperties = { + @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), + @ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + @ClusterConfigProperty(key = "group.share.enable", value = "true"), + @ClusterConfigProperty(key = "group.share.partition.max.record.locks", value = "10000"), + @ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000"), + @ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + @ClusterConfigProperty(key = "share.coordinator.state.topic.min.isr", value = "1"), + @ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "3"), + @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + @ClusterConfigProperty(key = "transaction.state.log.min.isr", value = "1"), + @ClusterConfigProperty(key = "transaction.state.log.replication.factor", value = "1"), + @ClusterConfigProperty(key = "group.share.max.size", value = "3") // Setting max group size to 3 + } + ) + public void testShareGroupMaxSizeConfigExceeded() throws Exception { + // creating 3 consumers in the group1 + ShareConsumer shareConsumer1 = createShareConsumer("group1"); + ShareConsumer shareConsumer2 = createShareConsumer("group1"); + ShareConsumer shareConsumer3 = createShareConsumer("group1"); + + shareConsumer1.subscribe(Set.of(tp.topic())); + shareConsumer2.subscribe(Set.of(tp.topic())); + shareConsumer3.subscribe(Set.of(tp.topic())); + + shareConsumer1.poll(Duration.ofMillis(5000)); + shareConsumer2.poll(Duration.ofMillis(5000)); + shareConsumer3.poll(Duration.ofMillis(5000)); + + ShareConsumer shareConsumer4 = createShareConsumer("group1"); + shareConsumer4.subscribe(Set.of(tp.topic())); + + TestUtils.waitForCondition(() -> { + try { + shareConsumer4.poll(Duration.ofMillis(5000)); + } catch (GroupMaxSizeReachedException e) { + return true; + } catch (Exception e) { + return false; + } + return false; + }, 30000, 200L, () -> "The 4th consumer was not kicked out of the group"); + + shareConsumer1.close(); + shareConsumer2.close(); + shareConsumer3.close(); + shareConsumer4.close(); + } + @ClusterTest public void testReadCommittedIsolationLevel() { alterShareAutoOffsetReset("group1", "earliest"); @@ -2495,8 +2629,7 @@ private int consumeMessages(AtomicInteger totalMessagesConsumed, int maxPolls, boolean commit) { return assertDoesNotThrow(() -> { - try (ShareConsumer shareConsumer = createShareConsumer( - groupId)) { + try (ShareConsumer shareConsumer = createShareConsumer(groupId)) { shareConsumer.subscribe(Set.of(tp.topic())); return consumeMessages(shareConsumer, totalMessagesConsumed, totalMessages, consumerNumber, maxPolls, commit); } @@ -2763,10 +2896,6 @@ private static class ComplexShareConsumer implements Runnable { this.configs.putIfAbsent(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); } - void stop() { - state.done().set(true); - } - @Override public void run() { try (ShareConsumer consumer = new KafkaShareConsumer<>(configs)) { diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index aa3b5c9d628c9..e2022e0f4d02f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -192,7 +192,8 @@ public class CommonClientConfigs { + "is considered failed and the group will rebalance in order to reassign the partitions to another member. " + "For consumers using a non-null group.instance.id which reach this timeout, partitions will not be immediately reassigned. " + "Instead, the consumer will stop sending heartbeats and partitions will be reassigned " - + "after expiration of session.timeout.ms. This mirrors the behavior of a static consumer which has shutdown."; + + "after expiration of the session timeout (defined by the client config session.timeout.ms if using the Classic rebalance protocol, or by the broker config group.consumer.session.timeout.ms if using the Consumer protocol). " + + "This mirrors the behavior of a static consumer which has shutdown."; public static final String REBALANCE_TIMEOUT_MS_CONFIG = "rebalance.timeout.ms"; public static final String REBALANCE_TIMEOUT_MS_DOC = "The maximum allowed time for each worker to join the group " @@ -206,15 +207,18 @@ public class CommonClientConfigs { + "to the broker. If no heartbeats are received by the broker before the expiration of this session timeout, " + "then the broker will remove this client from the group and initiate a rebalance. Note that the value " + "must be in the allowable range as configured in the broker configuration by group.min.session.timeout.ms " - + "and group.max.session.timeout.ms. Note that this configuration is not supported when group.protocol " - + "is set to \"consumer\"."; + + "and group.max.session.timeout.ms. Note that this client configuration is not supported when group.protocol " + + "is set to \"consumer\". In that case, session timeout is controlled by the broker config group.consumer.session.timeout.ms."; public static final String HEARTBEAT_INTERVAL_MS_CONFIG = "heartbeat.interval.ms"; public static final String HEARTBEAT_INTERVAL_MS_DOC = "The expected time between heartbeats to the consumer " + "coordinator when using Kafka's group management facilities. Heartbeats are used to ensure that the " + "consumer's session stays active and to facilitate rebalancing when new consumers join or leave the group. " - + "The value must be set lower than session.timeout.ms, but typically should be set no higher " - + "than 1/3 of that value. It can be adjusted even lower to control the expected time for normal rebalances."; + + "This config is only supported if group.protocol is set to \"classic\". In that case, " + + "the value must be set lower than session.timeout.ms, but typically should be set no higher " + + "than 1/3 of that value. It can be adjusted even lower to control the expected time for normal rebalances." + + "If group.protocol is set to \"consumer\", this config is not supported, as " + + "the heartbeat interval is controlled by the broker with group.consumer.heartbeat.interval.ms."; public static final String DEFAULT_API_TIMEOUT_MS_CONFIG = "default.api.timeout.ms"; public static final String DEFAULT_API_TIMEOUT_MS_DOC = "Specifies the timeout (in milliseconds) for client APIs. " + diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index b60156aae0066..4c567b7d466b5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -381,7 +381,7 @@ public synchronized void update(int requestVersion, MetadataResponse response, b public synchronized Set updatePartitionLeadership(Map partitionLeaders, List leaderNodes) { Map newNodes = leaderNodes.stream().collect(Collectors.toMap(Node::id, node -> node)); // Insert non-overlapping nodes from existing-nodes into new-nodes. - this.metadataSnapshot.cluster().nodes().stream().forEach(node -> newNodes.putIfAbsent(node.id(), node)); + this.metadataSnapshot.cluster().nodes().forEach(node -> newNodes.putIfAbsent(node.id(), node)); // Create partition-metadata for all updated partitions. Exclude updates for partitions - // 1. for which the corresponding partition has newer leader in existing metadata. diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java index 5a2f55c544fe1..fad56892f4596 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeUserScramCredentialsResult.java @@ -69,7 +69,7 @@ public KafkaFuture> all() { retval.completeExceptionally(Errors.forCode(optionalFirstFailedDescribe.get().errorCode()).exception(optionalFirstFailedDescribe.get().errorMessage())); } else { Map retvalMap = new HashMap<>(); - data.results().stream().forEach(userResult -> + data.results().forEach(userResult -> retvalMap.put(userResult.user(), new UserScramCredentialsDescription(userResult.user(), getScramCredentialInfosFor(userResult)))); retval.complete(retvalMap); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index 5cb5cc292ea2a..87a951fe26ecd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -825,36 +825,6 @@ public boolean supportsUseControllers() { } } - /** - * Provides the least loaded broker, or the active kcontroller if we're using - * bootstrap.controllers. - */ - private class ConstantBrokerOrActiveKController implements NodeProvider { - private final int nodeId; - - ConstantBrokerOrActiveKController(int nodeId) { - this.nodeId = nodeId; - } - - @Override - public Node provide() { - if (metadataManager.isReady()) { - if (metadataManager.usingBootstrapControllers()) { - return metadataManager.controller(); - } else if (metadataManager.nodeById(nodeId) != null) { - return metadataManager.nodeById(nodeId); - } - } - metadataManager.requestUpdate(); - return null; - } - - @Override - public boolean supportsUseControllers() { - return true; - } - } - /** * Provides the least loaded broker, or the active kcontroller if we're using * bootstrap.controllers. @@ -3541,27 +3511,29 @@ ListGroupsRequest.Builder createRequest(int timeoutMs) { } private void maybeAddGroup(ListGroupsResponseData.ListedGroup group) { - final String groupId = group.groupId(); - final Optional type; - if (group.groupType() == null || group.groupType().isEmpty()) { - type = Optional.empty(); - } else { - type = Optional.of(GroupType.parse(group.groupType())); - } - final String protocolType = group.protocolType(); - final Optional groupState; - if (group.groupState() == null || group.groupState().isEmpty()) { - groupState = Optional.empty(); - } else { - groupState = Optional.of(GroupState.parse(group.groupState())); + String protocolType = group.protocolType(); + if (options.protocolTypes().isEmpty() || options.protocolTypes().contains(protocolType)) { + final String groupId = group.groupId(); + final Optional type; + if (group.groupType() == null || group.groupType().isEmpty()) { + type = Optional.empty(); + } else { + type = Optional.of(GroupType.parse(group.groupType())); + } + final Optional groupState; + if (group.groupState() == null || group.groupState().isEmpty()) { + groupState = Optional.empty(); + } else { + groupState = Optional.of(GroupState.parse(group.groupState())); + } + final GroupListing groupListing = new GroupListing( + groupId, + type, + protocolType, + groupState + ); + results.addListing(groupListing); } - final GroupListing groupListing = new GroupListing( - groupId, - type, - protocolType, - groupState - ); - results.addListing(groupListing); } @Override @@ -3779,7 +3751,10 @@ public ListStreamsGroupOffsetsResult listStreamsGroupOffsets(Map new ListConsumerGroupOffsetsSpec().topicPartitions(entry.getValue().topicPartitions()) )); - return new ListStreamsGroupOffsetsResult(listConsumerGroupOffsets(consumerGroupSpecs, new ListConsumerGroupOffsetsOptions())); + ListConsumerGroupOffsetsOptions consumerGroupOptions = new ListConsumerGroupOffsetsOptions() + .requireStable(options.requireStable()) + .timeoutMs(options.timeoutMs()); + return new ListStreamsGroupOffsetsResult(listConsumerGroupOffsets(consumerGroupSpecs, consumerGroupOptions)); } @Override @@ -3794,7 +3769,9 @@ public DeleteConsumerGroupsResult deleteConsumerGroups(Collection groupI @Override public DeleteStreamsGroupsResult deleteStreamsGroups(Collection groupIds, DeleteStreamsGroupsOptions options) { - return new DeleteStreamsGroupsResult(deleteConsumerGroups(groupIds, new DeleteConsumerGroupsOptions())); + DeleteConsumerGroupsOptions consumerGroupOptions = new DeleteConsumerGroupsOptions() + .timeoutMs(options.timeoutMs()); + return new DeleteStreamsGroupsResult(deleteConsumerGroups(groupIds, consumerGroupOptions)); } @Override @@ -3814,7 +3791,9 @@ public DeleteStreamsGroupOffsetsResult deleteStreamsGroupOffsets( String groupId, Set partitions, DeleteStreamsGroupOffsetsOptions options) { - return new DeleteStreamsGroupOffsetsResult(deleteConsumerGroupOffsets(groupId, partitions, new DeleteConsumerGroupOffsetsOptions())); + DeleteConsumerGroupOffsetsOptions consumerGroupOptions = new DeleteConsumerGroupOffsetsOptions() + .timeoutMs(options.timeoutMs()); + return new DeleteStreamsGroupOffsetsResult(deleteConsumerGroupOffsets(groupId, partitions, consumerGroupOptions)); } @Override @@ -3839,7 +3818,7 @@ public AlterShareGroupOffsetsResult alterShareGroupOffsets(String groupId, Map groupSpecs, final ListShareGroupOffsetsOptions options) { - SimpleAdminApiFuture> future = ListShareGroupOffsetsHandler.newFuture(groupSpecs.keySet()); + SimpleAdminApiFuture> future = ListShareGroupOffsetsHandler.newFuture(groupSpecs.keySet()); ListShareGroupOffsetsHandler handler = new ListShareGroupOffsetsHandler(groupSpecs, logContext); invokeDriver(handler, future, options.timeoutMs); return new ListShareGroupOffsetsResult(future.all()); @@ -4273,7 +4252,9 @@ public AlterStreamsGroupOffsetsResult alterStreamsGroupOffsets( Map offsets, AlterStreamsGroupOffsetsOptions options ) { - return new AlterStreamsGroupOffsetsResult(alterConsumerGroupOffsets(groupId, offsets, new AlterConsumerGroupOffsetsOptions())); + AlterConsumerGroupOffsetsOptions consumerGroupOptions = new AlterConsumerGroupOffsetsOptions() + .timeoutMs(options.timeoutMs()); + return new AlterStreamsGroupOffsetsResult(alterConsumerGroupOffsets(groupId, offsets, consumerGroupOptions)); } @Override @@ -4487,8 +4468,8 @@ public void handleResponse(AbstractResponse abstractResponse) { * Be sure to do this after the NOT_CONTROLLER error check above * so that all errors are consistent in that case. */ - userIllegalAlterationExceptions.entrySet().stream().forEach(entry -> - futures.get(entry.getKey()).completeExceptionally(entry.getValue()) + userIllegalAlterationExceptions.forEach((key, value) -> + futures.get(key).completeExceptionally(value) ); response.data().results().forEach(result -> { KafkaFutureImpl future = futures.get(result.user()); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListGroupsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListGroupsOptions.java index e5d70133186cd..7d7083f46c5c7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListGroupsOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListGroupsOptions.java @@ -46,6 +46,24 @@ public static ListGroupsOptions forConsumerGroups() { .withProtocolTypes(Set.of("", ConsumerProtocol.PROTOCOL_TYPE)); } + /** + * Only share groups will be returned by listGroups(). + * This operation sets a filter on group type which select share groups. + */ + public static ListGroupsOptions forShareGroups() { + return new ListGroupsOptions() + .withTypes(Set.of(GroupType.SHARE)); + } + + /** + * Only streams groups will be returned by listGroups(). + * This operation sets a filter on group type which select streams groups. + */ + public static ListGroupsOptions forStreamsGroups() { + return new ListGroupsOptions() + .withTypes(Set.of(GroupType.STREAMS)); + } + /** * If groupStates is set, only groups in these states will be returned by listGroups(). * Otherwise, all groups are returned. @@ -56,6 +74,10 @@ public ListGroupsOptions inGroupStates(Set groupStates) { return this; } + /** + * If protocol types is set, only groups of these protocol types will be returned by listGroups(). + * Otherwise, all groups are returned. + */ public ListGroupsOptions withProtocolTypes(Set protocolTypes) { this.protocolTypes = (protocolTypes == null || protocolTypes.isEmpty()) ? Set.of() : Set.copyOf(protocolTypes); return this; diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java index d39f3711f4c65..e1dcd932309bf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java @@ -18,6 +18,7 @@ package org.apache.kafka.clients.admin; import org.apache.kafka.clients.admin.internals.CoordinatorKey; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.annotation.InterfaceStability; @@ -35,9 +36,9 @@ @InterfaceStability.Evolving public class ListShareGroupOffsetsResult { - private final Map>> futures; + private final Map>> futures; - ListShareGroupOffsetsResult(final Map>> futures) { + ListShareGroupOffsetsResult(final Map>> futures) { this.futures = futures.entrySet().stream() .collect(Collectors.toMap(e -> e.getKey().idValue, Map.Entry::getValue)); } @@ -47,10 +48,10 @@ public class ListShareGroupOffsetsResult { * * @return Future which yields all {@code Map>} objects, if requests for all the groups succeed. */ - public KafkaFuture>> all() { + public KafkaFuture>> all() { return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).thenApply( nil -> { - Map> offsets = new HashMap<>(futures.size()); + Map> offsets = new HashMap<>(futures.size()); futures.forEach((groupId, future) -> { try { offsets.put(groupId, future.get()); @@ -70,7 +71,7 @@ public KafkaFuture>> all() { * @param groupId The group ID. * @return Future which yields a map of topic partitions to offsets for the specified group. */ - public KafkaFuture> partitionsToOffset(String groupId) { + public KafkaFuture> partitionsToOffsetAndMetadata(String groupId) { if (!futures.containsKey(groupId)) { throw new IllegalArgumentException("Group ID not found: " + groupId); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListStreamsGroupOffsetsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListStreamsGroupOffsetsOptions.java index 05caf1dee1120..08835d817e63e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListStreamsGroupOffsetsOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListStreamsGroupOffsetsOptions.java @@ -21,7 +21,7 @@ /** - * Options for {@link Admin#listStreamsGroupOffsets(Map, ListStreamsGroupOffsetsOptions)}. + * Options for {@link Admin#listStreamsGroupOffsets(java.util.Map, ListStreamsGroupOffsetsOptions)}. *

* The API of this class is evolving, see {@link Admin} for details. */ diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsOptions.java index 0768b1a75a71c..72a796308d45b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListTransactionsOptions.java @@ -31,6 +31,7 @@ public class ListTransactionsOptions extends AbstractOptions filteredProducerIds = Collections.emptySet(); private long filteredDuration = -1L; + private String filteredTransactionalIdPattern; /** * Filter only the transactions that are in a specific set of states. If no filter * is specified or if the passed set of states is empty, then transactions in all @@ -70,6 +71,19 @@ public ListTransactionsOptions filterOnDuration(long durationMs) { return this; } + /** + * Filter only the transactions that match with the given transactional ID pattern. + * If the filter is null or if the passed string is empty, + * then all the transactions will be returned. + * + * @param pattern the transactional ID regular expression pattern to filter by + * @return this object + */ + public ListTransactionsOptions filterOnTransactionalIdPattern(String pattern) { + this.filteredTransactionalIdPattern = pattern; + return this; + } + /** * Returns the set of states to be filtered or empty if no states have been specified. * @@ -99,12 +113,23 @@ public long filteredDuration() { return filteredDuration; } + /** + * Returns transactional ID being filtered. + * + * @return the current transactional ID pattern filter (empty means no transactional IDs are filtered and all + * transactions will be returned) + */ + public String filteredTransactionalIdPattern() { + return filteredTransactionalIdPattern; + } + @Override public String toString() { return "ListTransactionsOptions(" + "filteredStates=" + filteredStates + ", filteredProducerIds=" + filteredProducerIds + ", filteredDuration=" + filteredDuration + + ", filteredTransactionalIdPattern=" + filteredTransactionalIdPattern + ", timeoutMs=" + timeoutMs + ')'; } @@ -116,11 +141,12 @@ public boolean equals(Object o) { ListTransactionsOptions that = (ListTransactionsOptions) o; return Objects.equals(filteredStates, that.filteredStates) && Objects.equals(filteredProducerIds, that.filteredProducerIds) && - Objects.equals(filteredDuration, that.filteredDuration); + Objects.equals(filteredDuration, that.filteredDuration) && + Objects.equals(filteredTransactionalIdPattern, that.filteredTransactionalIdPattern); } @Override public int hashCode() { - return Objects.hash(filteredStates, filteredProducerIds, filteredDuration); + return Objects.hash(filteredStates, filteredProducerIds, filteredDuration, filteredTransactionalIdPattern); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java index 36ff25ebb7991..86c56167ad314 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandler.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.RequestUtils; import org.apache.kafka.common.utils.LogContext; import org.slf4j.Logger; @@ -36,7 +37,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -139,40 +139,52 @@ public ApiResult> handleR ) { validateKeys(groupIds); - final OffsetFetchResponse response = (OffsetFetchResponse) abstractResponse; + var response = (OffsetFetchResponse) abstractResponse; + var completed = new HashMap>(); + var failed = new HashMap(); + var unmapped = new ArrayList(); - Map> completed = new HashMap<>(); - Map failed = new HashMap<>(); - List unmapped = new ArrayList<>(); for (CoordinatorKey coordinatorKey : groupIds) { - String group = coordinatorKey.idValue; - if (response.groupHasError(group)) { - handleGroupError(CoordinatorKey.byGroupId(group), response.groupLevelError(group), failed, unmapped); + var groupId = coordinatorKey.idValue; + var group = response.group(groupId); + var error = Errors.forCode(group.errorCode()); + + if (error != Errors.NONE) { + handleGroupError( + coordinatorKey, + error, + failed, + unmapped + ); } else { - final Map groupOffsetsListing = new HashMap<>(); - Map responseData = response.partitionDataMap(group); - for (Map.Entry partitionEntry : responseData.entrySet()) { - final TopicPartition topicPartition = partitionEntry.getKey(); - OffsetFetchResponse.PartitionData partitionData = partitionEntry.getValue(); - final Errors error = partitionData.error; - - if (error == Errors.NONE) { - final long offset = partitionData.offset; - final String metadata = partitionData.metadata; - final Optional leaderEpoch = partitionData.leaderEpoch; - // Negative offset indicates that the group has no committed offset for this partition - if (offset < 0) { - groupOffsetsListing.put(topicPartition, null); + var offsets = new HashMap(); + + group.topics().forEach(topic -> + topic.partitions().forEach(partition -> { + var tp = new TopicPartition(topic.name(), partition.partitionIndex()); + var partitionError = Errors.forCode(partition.errorCode()); + + if (partitionError == Errors.NONE) { + // Negative offset indicates that the group has no committed offset for this partition. + if (partition.committedOffset() < 0) { + offsets.put(tp, null); + } else { + offsets.put(tp, new OffsetAndMetadata( + partition.committedOffset(), + RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()), + partition.metadata() + )); + } } else { - groupOffsetsListing.put(topicPartition, new OffsetAndMetadata(offset, leaderEpoch, metadata)); + log.warn("Skipping return offset for {} due to error {}.", tp, partitionError); } - } else { - log.warn("Skipping return offset for {} due to error {}.", topicPartition, error); - } - } - completed.put(CoordinatorKey.byGroupId(group), groupOffsetsListing); + }) + ); + + completed.put(coordinatorKey, offsets); } } + return new ApiResult<>(completed, failed, unmapped); } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListShareGroupOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListShareGroupOffsetsHandler.java index fcaba5a67e612..f9b9e987930bd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListShareGroupOffsetsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListShareGroupOffsetsHandler.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.admin.KafkaAdminClient; import org.apache.kafka.clients.admin.ListShareGroupOffsetsOptions; import org.apache.kafka.clients.admin.ListShareGroupOffsetsSpec; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.DescribeShareGroupOffsetsRequestData; @@ -39,13 +40,14 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; /** * This class is the handler for {@link KafkaAdminClient#listShareGroupOffsets(Map, ListShareGroupOffsetsOptions)} call */ -public class ListShareGroupOffsetsHandler extends AdminApiHandler.Batched> { +public class ListShareGroupOffsetsHandler extends AdminApiHandler.Batched> { private final Map groupSpecs; private final Logger log; @@ -58,7 +60,7 @@ public ListShareGroupOffsetsHandler(Map group this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, logContext); } - public static AdminApiFuture.SimpleAdminApiFuture> newFuture(Collection groupIds) { + public static AdminApiFuture.SimpleAdminApiFuture> newFuture(Collection groupIds) { return AdminApiFuture.forKeys(coordinatorKeys(groupIds)); } @@ -108,13 +110,13 @@ public DescribeShareGroupOffsetsRequest.Builder buildBatchedRequest(int coordina } @Override - public ApiResult> handleResponse(Node coordinator, - Set groupIds, - AbstractResponse abstractResponse) { + public ApiResult> handleResponse(Node coordinator, + Set groupIds, + AbstractResponse abstractResponse) { validateKeys(groupIds); final DescribeShareGroupOffsetsResponse response = (DescribeShareGroupOffsetsResponse) abstractResponse; - final Map> completed = new HashMap<>(); + final Map> completed = new HashMap<>(); final Map failed = new HashMap<>(); final List unmapped = new ArrayList<>(); @@ -123,17 +125,19 @@ public ApiResult> handleResponse(Node if (response.hasGroupError(groupId)) { handleGroupError(coordinatorKey, response.groupError(groupId), failed, unmapped); } else { - Map groupOffsetsListing = new HashMap<>(); + Map groupOffsetsListing = new HashMap<>(); response.data().groups().stream().filter(g -> g.groupId().equals(groupId)).forEach(groupResponse -> { for (DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic topicResponse : groupResponse.topics()) { for (DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition partitionResponse : topicResponse.partitions()) { TopicPartition tp = new TopicPartition(topicResponse.topicName(), partitionResponse.partitionIndex()); if (partitionResponse.errorCode() == Errors.NONE.code()) { + final long startOffset = partitionResponse.startOffset(); + final Optional leaderEpoch = partitionResponse.leaderEpoch() < 0 ? Optional.empty() : Optional.of(partitionResponse.leaderEpoch()); // Negative offset indicates there is no start offset for this partition if (partitionResponse.startOffset() < 0) { groupOffsetsListing.put(tp, null); } else { - groupOffsetsListing.put(tp, partitionResponse.startOffset()); + groupOffsetsListing.put(tp, new OffsetAndMetadata(startOffset, leaderEpoch, "")); } } else { log.warn("Skipping return offset for {} due to error {}: {}.", tp, partitionResponse.errorCode(), partitionResponse.errorMessage()); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandler.java index 71b8e1a7c5607..f47d9f90189a5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandler.java @@ -75,6 +75,9 @@ public ListTransactionsRequest.Builder buildBatchedRequest( .map(TransactionState::toString) .collect(Collectors.toList())); request.setDurationFilter(options.filteredDuration()); + if (options.filteredTransactionalIdPattern() != null && !options.filteredTransactionalIdPattern().isEmpty()) { + request.setTransactionalIdPattern(options.filteredTransactionalIdPattern()); + } return new ListTransactionsRequest.Builder(request); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/CloseOptions.java b/clients/src/main/java/org/apache/kafka/clients/consumer/CloseOptions.java index 0cbbcbca54d81..0aae57d39cfc0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/CloseOptions.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/CloseOptions.java @@ -47,23 +47,18 @@ public enum GroupMembershipOperation { * Specifies the group membership operation upon shutdown. * By default, {@code GroupMembershipOperation.DEFAULT} will be applied, which follows the consumer's default behavior. */ - protected GroupMembershipOperation operation = GroupMembershipOperation.DEFAULT; + private GroupMembershipOperation operation = GroupMembershipOperation.DEFAULT; /** * Specifies the maximum amount of time to wait for the close process to complete. * This allows users to define a custom timeout for gracefully stopping the consumer. * If no value is set, the default timeout {@link ConsumerUtils#DEFAULT_CLOSE_TIMEOUT_MS} will be applied. */ - protected Optional timeout = Optional.empty(); + private Optional timeout = Optional.empty(); private CloseOptions() { } - protected CloseOptions(final CloseOptions option) { - this.operation = option.operation; - this.timeout = option.timeout; - } - /** * Static method to create a {@code CloseOptions} with a custom timeout. * @@ -71,9 +66,7 @@ protected CloseOptions(final CloseOptions option) { * @return a new {@code CloseOptions} instance with the specified timeout. */ public static CloseOptions timeout(final Duration timeout) { - CloseOptions option = new CloseOptions(); - option.timeout = Optional.ofNullable(timeout); - return option; + return new CloseOptions().withTimeout(timeout); } /** @@ -84,10 +77,7 @@ public static CloseOptions timeout(final Duration timeout) { * @return a new {@code CloseOptions} instance with the specified group membership operation. */ public static CloseOptions groupMembershipOperation(final GroupMembershipOperation operation) { - Objects.requireNonNull(operation, "operation should not be null"); - CloseOptions option = new CloseOptions(); - option.operation = operation; - return option; + return new CloseOptions().withGroupMembershipOperation(operation); } /** @@ -108,8 +98,7 @@ public CloseOptions withTimeout(final Duration timeout) { * @return this {@code CloseOptions} instance. */ public CloseOptions withGroupMembershipOperation(final GroupMembershipOperation operation) { - Objects.requireNonNull(operation, "operation should not be null"); - this.operation = operation; + this.operation = Objects.requireNonNull(operation, "operation should not be null"); return this; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index c4c462e7f9451..3a06e71335d75 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1563,8 +1563,8 @@ public Set paused() { * @param timestampsToSearch the mapping from partition to the timestamp to look up. * * @return a mapping from partition to the timestamp and offset of the first message with timestamp greater - * than or equal to the target timestamp. {@code null} will be returned for the partition if there is no - * such message. + * than or equal to the target timestamp. If the timestamp and offset for a specific partition cannot be found within + * the default timeout, and no corresponding message exists, the entry in the returned map will be {@code null} * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic(s). See the exception for more details * @throws IllegalArgumentException if the target timestamp is negative @@ -1590,8 +1590,8 @@ public Map offsetsForTimes(Map beginningOffsets(Collection par * @param partitions the partitions to get the earliest offsets * @param timeout The maximum amount of time to await retrieval of the beginning offsets * - * @return The earliest available offsets for the given partitions + * @return The earliest available offsets for the given partitions, and it will return empty map if zero timeout is provided * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic(s). See the exception for more details * @throws org.apache.kafka.common.errors.TimeoutException if the offset metadata could not be fetched before @@ -1684,7 +1684,7 @@ public Map endOffsets(Collection partition * @param partitions the partitions to get the end offsets. * @param timeout The maximum amount of time to await retrieval of the end offsets * - * @return The end offsets for the given partitions. + * @return The end offsets for the given partitions, and it will return empty map if zero timeout is provided * @throws org.apache.kafka.common.errors.AuthenticationException if authentication fails. See the exception for more details * @throws org.apache.kafka.common.errors.AuthorizationException if not authorized to the topic(s). See the exception for more details * @throws org.apache.kafka.common.errors.TimeoutException if the offsets could not be fetched before diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index 7b268fe0f4918..303f8e5f1ddc0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -335,7 +335,7 @@ public synchronized void addRecord(ConsumerRecord record) { * @param maxPollRecords the max.poll.records. */ public synchronized void setMaxPollRecords(long maxPollRecords) { - if (this.maxPollRecords < 1) { + if (maxPollRecords < 1) { throw new IllegalArgumentException("MaxPollRecords must be strictly superior to 0"); } this.maxPollRecords = maxPollRecords; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java index 9d219907926d4..3998d672006a3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractHeartbeatRequestManager.java @@ -63,8 +63,9 @@ public abstract class AbstractHeartbeatRequestManager impl */ static final Utils.TopicPartitionComparator TOPIC_PARTITION_COMPARATOR = new Utils.TopicPartitionComparator(); - /** - * TopicIdPartition comparator based on topic name and partition (ignoring topic ID while sorting, - * as this is sorted mainly for logging purposes). - */ - static final Utils.TopicIdPartitionComparator TOPIC_ID_PARTITION_COMPARATOR = new Utils.TopicIdPartitionComparator(); - /** * Group ID of the consumer group the member will be part of, provided when creating the current * membership manager. @@ -376,8 +370,8 @@ protected void processAssignmentReceived(Map> assignmen */ private void replaceTargetAssignmentWithNewAssignment(Map> assignment) { currentTargetAssignment.updateWith(assignment).ifPresent(updatedAssignment -> { - log.debug("Target assignment updated from {} to {}. Member will reconcile it on the next poll.", - currentTargetAssignment, updatedAssignment); + log.debug("Member {} updated its target assignment from {} to {}. Member will reconcile it on the next poll.", + memberId, currentTargetAssignment, updatedAssignment); currentTargetAssignment = updatedAssignment; }); } @@ -517,11 +511,10 @@ private void clearAssignment() { * @param assignedPartitions Full assignment, to update in the subscription state * @param addedPartitions Newly added partitions */ - private void updateSubscriptionAwaitingCallback(SortedSet assignedPartitions, + private void updateSubscriptionAwaitingCallback(TopicIdPartitionSet assignedPartitions, SortedSet addedPartitions) { - Set assignedTopicPartitions = toTopicPartitionSet(assignedPartitions); - subscriptions.assignFromSubscribedAwaitingCallback(assignedTopicPartitions, addedPartitions); - notifyAssignmentChange(assignedTopicPartitions); + subscriptions.assignFromSubscribedAwaitingCallback(assignedPartitions.topicPartitions(), addedPartitions); + notifyAssignmentChange(assignedPartitions.topicPartitions()); } /** @@ -541,6 +534,7 @@ public void transitionToJoining() { } resetEpoch(); transitionTo(MemberState.JOINING); + log.debug("Member {} will join the group on the next call to poll.", memberId); clearPendingAssignmentsAndLocalNamesCache(); } @@ -618,6 +612,8 @@ protected CompletableFuture leaveGroup(boolean runCallbacks) { clearAssignmentAndLeaveGroup(); }); } else { + log.debug("Member {} attempting to leave has no rebalance callbacks, " + + "so it will clear assignments and transition to send heartbeat to leave group.", memberId); clearAssignmentAndLeaveGroup(); } @@ -708,8 +704,10 @@ public void onHeartbeatRequestGenerated() { transitionTo(MemberState.STABLE); } else { log.debug("Member {} with epoch {} transitioned to {} after a heartbeat was sent " + - "to ack a previous reconciliation. New assignments are ready to " + - "be reconciled.", memberId, memberEpoch, MemberState.RECONCILING); + "to ack a previous reconciliation. \n" + + "\t\tCurrent assignment: {} \n" + + "\t\tTarget assignment: {}\n", + memberId, memberEpoch, MemberState.RECONCILING, currentAssignment, currentTargetAssignment); transitionTo(MemberState.RECONCILING); } } else if (state == MemberState.LEAVING) { @@ -839,7 +837,7 @@ public void maybeReconcile(boolean canCommit) { // Find the subset of the target assignment that can be resolved to topic names, and trigger a metadata update // if some topic IDs are not resolvable. - SortedSet assignedTopicIdPartitions = findResolvableAssignmentAndTriggerMetadataUpdate(); + TopicIdPartitionSet assignedTopicIdPartitions = findResolvableAssignmentAndTriggerMetadataUpdate(); final LocalAssignment resolvedAssignment = new LocalAssignment(currentTargetAssignment.localEpoch, assignedTopicIdPartitions); if (!currentAssignment.isNone() && resolvedAssignment.partitions.equals(currentAssignment.partitions)) { @@ -857,7 +855,7 @@ public void maybeReconcile(boolean canCommit) { // Keep copy of assigned TopicPartitions created from the TopicIdPartitions that are // being reconciled. Needed for interactions with the centralized subscription state that // does not support topic IDs yet, and for the callbacks. - SortedSet assignedTopicPartitions = toTopicPartitionSet(assignedTopicIdPartitions); + SortedSet assignedTopicPartitions = assignedTopicIdPartitions.toTopicNamePartitionSet(); SortedSet ownedPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); ownedPartitions.addAll(subscriptions.assignedPartitions()); @@ -934,7 +932,7 @@ long getDeadlineMsForTimeout(final long timeoutMs) { * transition. Note that if any of the 2 callbacks fails, the reconciliation should fail. */ private void revokeAndAssign(LocalAssignment resolvedAssignment, - SortedSet assignedTopicIdPartitions, + TopicIdPartitionSet assignedTopicIdPartitions, SortedSet revokedPartitions, SortedSet addedPartitions) { CompletableFuture revocationResult; @@ -1031,15 +1029,6 @@ protected CompletableFuture signalPartitionsLost(Set parti return CompletableFuture.completedFuture(null); } - /** - * Build set of {@link TopicPartition} from the given set of {@link TopicIdPartition}. - */ - protected SortedSet toTopicPartitionSet(SortedSet topicIdPartitions) { - SortedSet result = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); - topicIdPartitions.forEach(topicIdPartition -> result.add(topicIdPartition.topicPartition())); - return result; - } - /** * Visible for testing. */ @@ -1073,8 +1062,8 @@ void markReconciliationCompleted() { * * */ - private SortedSet findResolvableAssignmentAndTriggerMetadataUpdate() { - final SortedSet assignmentReadyToReconcile = new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR); + private TopicIdPartitionSet findResolvableAssignmentAndTriggerMetadataUpdate() { + final TopicIdPartitionSet assignmentReadyToReconcile = new TopicIdPartitionSet(); final HashMap> unresolved = new HashMap<>(currentTargetAssignment.partitions); // Try to resolve topic names from metadata cache or subscription cache, and move @@ -1200,7 +1189,7 @@ CompletableFuture revokePartitions(Set partitionsToRevoke) * @return Future that will complete when the callback execution completes. */ private CompletableFuture assignPartitions( - SortedSet assignedPartitions, + TopicIdPartitionSet assignedPartitions, SortedSet addedPartitions) { // Update assignment in the subscription state, and ensure that no fetching or positions @@ -1218,7 +1207,7 @@ private CompletableFuture assignPartitions( // returning no records, as no topic partitions are marked as fetchable. In contrast, with the classic consumer, // if the first callback fails but the next one succeeds, polling can still retrieve data. To align with // this behavior, we rely on assignedPartitions to avoid such scenarios. - subscriptions.enablePartitionsAwaitingCallback(toTopicPartitionSet(assignedPartitions)); + subscriptions.enablePartitionsAwaitingCallback(assignedPartitions.topicPartitions()); } else { // Keeping newly added partitions as non-fetchable after the callback failure. // They will be retried on the next reconciliation loop, until it succeeds or the @@ -1232,7 +1221,7 @@ private CompletableFuture assignPartitions( }); // Clear topic names cache, removing topics that are not assigned to the member anymore. - Set assignedTopics = assignedPartitions.stream().map(TopicIdPartition::topic).collect(Collectors.toSet()); + Set assignedTopics = assignedPartitions.topicNames(); assignedTopicNamesCache.values().retainAll(assignedTopics); return result; @@ -1450,16 +1439,13 @@ public LocalAssignment(long localEpoch, Map> partitions } } - public LocalAssignment(long localEpoch, SortedSet topicIdPartitions) { + public LocalAssignment(long localEpoch, TopicIdPartitionSet topicIdPartitions) { + Objects.requireNonNull(topicIdPartitions); this.localEpoch = localEpoch; - this.partitions = new HashMap<>(); if (localEpoch == NONE_EPOCH && !topicIdPartitions.isEmpty()) { throw new IllegalArgumentException("Local epoch must be set if there are partitions"); } - topicIdPartitions.forEach(topicIdPartition -> { - Uuid topicId = topicIdPartition.topicId(); - partitions.computeIfAbsent(topicId, k -> new TreeSet<>()).add(topicIdPartition.partition()); - }); + this.partitions = topicIdPartitions.toTopicIdPartitionMap(); } public String toString() { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index db1d239c0ae84..e301b6855c6f3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -1306,7 +1306,10 @@ private Map beginningOrEndOffset(Collection(); } Map offsetAndTimestampMap; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index 4b22a5711b330..bae8ad679aa79 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -35,6 +35,7 @@ import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitResponseData; import org.apache.kafka.common.message.OffsetFetchRequestData; +import org.apache.kafka.common.message.OffsetFetchResponseData; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.RecordBatch; @@ -43,6 +44,7 @@ import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.RequestUtils; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; @@ -1012,13 +1014,14 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest() { @Override void onResponse(final ClientResponse response) { long currentTimeMs = response.receivedTimeMs(); - OffsetFetchResponse fetchResponse = (OffsetFetchResponse) response.responseBody(); - Errors responseError = fetchResponse.groupLevelError(groupId); - if (responseError != Errors.NONE) { - onFailure(currentTimeMs, responseError); + var fetchResponse = (OffsetFetchResponse) response.responseBody(); + var groupResponse = fetchResponse.group(groupId); + var error = Errors.forCode(groupResponse.errorCode()); + if (error != Errors.NONE) { + onFailure(currentTimeMs, error); return; } - onSuccess(currentTimeMs, fetchResponse); + onSuccess(currentTimeMs, groupResponse); } /** @@ -1083,53 +1086,58 @@ void removeRequest() { * offsets contained in the response, and record a successful request attempt. */ private void onSuccess(final long currentTimeMs, - final OffsetFetchResponse response) { - Set unauthorizedTopics = null; - Map responseData = - response.partitionDataMap(groupId); - Map offsets = new HashMap<>(responseData.size()); - Set unstableTxnOffsetTopicPartitions = new HashSet<>(); - boolean failedRequestRegistered = false; - for (Map.Entry entry : responseData.entrySet()) { - TopicPartition tp = entry.getKey(); - OffsetFetchResponse.PartitionData partitionData = entry.getValue(); - if (partitionData.hasError()) { - Errors error = partitionData.error; - log.debug("Failed to fetch offset for partition {}: {}", tp, error.message()); + final OffsetFetchResponseData.OffsetFetchResponseGroup response) { + var offsets = new HashMap(); + var unstableTxnOffsetTopicPartitions = new HashSet(); + var unauthorizedTopics = new HashSet(); + var failedRequestRegistered = false; + + for (var topic : response.topics()) { + for (var partition : topic.partitions()) { + var tp = new TopicPartition( + topic.name(), + partition.partitionIndex() + ); + var error = Errors.forCode(partition.errorCode()); + if (error != Errors.NONE) { + log.debug("Failed to fetch offset for partition {}: {}", tp, error.message()); - if (!failedRequestRegistered) { - onFailedAttempt(currentTimeMs); - failedRequestRegistered = true; - } + if (!failedRequestRegistered) { + onFailedAttempt(currentTimeMs); + failedRequestRegistered = true; + } - if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) { - future.completeExceptionally(new KafkaException("Topic or Partition " + tp + " does not exist")); - return; - } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) { - if (unauthorizedTopics == null) { - unauthorizedTopics = new HashSet<>(); + if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) { + future.completeExceptionally(new KafkaException("Topic or Partition " + tp + " does not exist")); + return; + } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) { + unauthorizedTopics.add(tp.topic()); + } else if (error == Errors.UNSTABLE_OFFSET_COMMIT) { + unstableTxnOffsetTopicPartitions.add(tp); + } else { + // Fail with a non-retriable KafkaException for all unexpected partition + // errors (even if they are retriable) + future.completeExceptionally(new KafkaException("Unexpected error in fetch offset " + + "response for partition " + tp + ": " + error.message())); + return; } - unauthorizedTopics.add(tp.topic()); - } else if (error == Errors.UNSTABLE_OFFSET_COMMIT) { - unstableTxnOffsetTopicPartitions.add(tp); + } else if (partition.committedOffset() >= 0) { + // record the position with the offset (-1 indicates no committed offset to fetch); + // if there's no committed offset, record as null + offsets.put(tp, new OffsetAndMetadata( + partition.committedOffset(), + RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()), + partition.metadata() + )); } else { - // Fail with a non-retriable KafkaException for all unexpected partition - // errors (even if they are retriable) - future.completeExceptionally(new KafkaException("Unexpected error in fetch offset " + - "response for partition " + tp + ": " + error.message())); - return; + log.info("Found no committed offset for partition {}", tp); + offsets.put(tp, null); } - } else if (partitionData.offset >= 0) { - // record the position with the offset (-1 indicates no committed offset to fetch); - // if there's no committed offset, record as null - offsets.put(tp, new OffsetAndMetadata(partitionData.offset, partitionData.leaderEpoch, partitionData.metadata)); - } else { - log.info("Found no committed offset for partition {}", tp); - offsets.put(tp, null); + } } - if (unauthorizedTopics != null) { + if (!unauthorizedTopics.isEmpty()) { future.completeExceptionally(new TopicAuthorizationException(unauthorizedTopics)); } else if (!unstableTxnOffsetTopicPartitions.isEmpty()) { // TODO: Optimization question: Do we need to retry all partitions upon a single partition error? diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 8829654c7a8b5..9abed8f7fb417 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -63,6 +63,7 @@ import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.RequestUtils; import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; @@ -1512,64 +1513,71 @@ private OffsetFetchResponseHandler() { @Override public void handle(OffsetFetchResponse response, RequestFuture> future) { - Errors responseError = response.groupLevelError(rebalanceConfig.groupId); - if (responseError != Errors.NONE) { - log.debug("Offset fetch failed: {}", responseError.message()); + var group = response.group(rebalanceConfig.groupId); + var groupError = Errors.forCode(group.errorCode()); - if (responseError == Errors.COORDINATOR_NOT_AVAILABLE || - responseError == Errors.NOT_COORDINATOR) { + if (groupError != Errors.NONE) { + log.debug("Offset fetch failed: {}", groupError.message()); + + if (groupError == Errors.COORDINATOR_NOT_AVAILABLE || + groupError == Errors.NOT_COORDINATOR) { // re-discover the coordinator and retry - markCoordinatorUnknown(responseError); - future.raise(responseError); - } else if (responseError == Errors.GROUP_AUTHORIZATION_FAILED) { + markCoordinatorUnknown(groupError); + future.raise(groupError); + } else if (groupError == Errors.GROUP_AUTHORIZATION_FAILED) { future.raise(GroupAuthorizationException.forGroupId(rebalanceConfig.groupId)); - } else if (responseError.exception() instanceof RetriableException) { + } else if (groupError.exception() instanceof RetriableException) { // retry - future.raise(responseError); + future.raise(groupError); } else { - future.raise(new KafkaException("Unexpected error in fetch offset response: " + responseError.message())); + future.raise(new KafkaException("Unexpected error in fetch offset response: " + groupError.message())); } return; } - Set unauthorizedTopics = null; - Map responseData = - response.partitionDataMap(rebalanceConfig.groupId); - Map offsets = new HashMap<>(responseData.size()); - Set unstableTxnOffsetTopicPartitions = new HashSet<>(); - for (Map.Entry entry : responseData.entrySet()) { - TopicPartition tp = entry.getKey(); - OffsetFetchResponse.PartitionData partitionData = entry.getValue(); - if (partitionData.hasError()) { - Errors error = partitionData.error; - log.debug("Failed to fetch offset for partition {}: {}", tp, error.message()); - - if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) { - future.raise(new KafkaException("Topic or Partition " + tp + " does not exist")); - return; - } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) { - if (unauthorizedTopics == null) { - unauthorizedTopics = new HashSet<>(); + var offsets = new HashMap(); + var unstableTxnOffsetTopicPartitions = new HashSet(); + var unauthorizedTopics = new HashSet(); + + for (var topic : group.topics()) { + for (var partition : topic.partitions()) { + var tp = new TopicPartition( + topic.name(), + partition.partitionIndex() + ); + var error = Errors.forCode(partition.errorCode()); + + if (error != Errors.NONE) { + log.debug("Failed to fetch offset for partition {}: {}", tp, error.message()); + + if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) { + future.raise(new KafkaException("Topic or Partition " + tp + " does not exist")); + return; + } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) { + unauthorizedTopics.add(tp.topic()); + } else if (error == Errors.UNSTABLE_OFFSET_COMMIT) { + unstableTxnOffsetTopicPartitions.add(tp); + } else { + future.raise(new KafkaException("Unexpected error in fetch offset response for partition " + + tp + ": " + error.message())); + return; } - unauthorizedTopics.add(tp.topic()); - } else if (error == Errors.UNSTABLE_OFFSET_COMMIT) { - unstableTxnOffsetTopicPartitions.add(tp); + } else if (partition.committedOffset() >= 0) { + // record the position with the offset (-1 indicates no committed offset to fetch); + // if there's no committed offset, record as null + offsets.put(tp, new OffsetAndMetadata( + partition.committedOffset(), + RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()), + partition.metadata() + )); } else { - future.raise(new KafkaException("Unexpected error in fetch offset response for partition " + - tp + ": " + error.message())); - return; + log.info("Found no committed offset for partition {}", tp); + offsets.put(tp, null); } - } else if (partitionData.offset >= 0) { - // record the position with the offset (-1 indicates no committed offset to fetch); - // if there's no committed offset, record as null - offsets.put(tp, new OffsetAndMetadata(partitionData.offset, partitionData.leaderEpoch, partitionData.metadata)); - } else { - log.info("Found no committed offset for partition {}", tp); - offsets.put(tp, null); } } - if (unauthorizedTopics != null) { + if (!unauthorizedTopics.isEmpty()) { future.raise(new TopicAuthorizationException(unauthorizedTopics)); } else if (!unstableTxnOffsetTopicPartitions.isEmpty()) { // just retry diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java index a48289919b023..de146f29e8277 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java @@ -39,7 +39,6 @@ import java.util.LinkedList; import java.util.List; import java.util.Objects; -import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -154,18 +153,18 @@ void runOnce() { lastPollTimeMs = currentTimeMs; final long pollWaitTimeMs = requestManagers.entries().stream() - .filter(Optional::isPresent) - .map(Optional::get) .map(rm -> rm.poll(currentTimeMs)) - .map(networkClientDelegate::addAll) - .reduce(MAX_POLL_TIMEOUT_MS, Math::min); + .mapToLong(networkClientDelegate::addAll) + .filter(ms -> ms <= MAX_POLL_TIMEOUT_MS) + .min() + .orElse(MAX_POLL_TIMEOUT_MS); + networkClientDelegate.poll(pollWaitTimeMs, currentTimeMs); cachedMaximumTimeToWait = requestManagers.entries().stream() - .filter(Optional::isPresent) - .map(Optional::get) - .map(rm -> rm.maximumTimeToWait(currentTimeMs)) - .reduce(Long.MAX_VALUE, Math::min); + .mapToLong(rm -> rm.maximumTimeToWait(currentTimeMs)) + .min() + .orElse(Long.MAX_VALUE); reapExpiredApplicationEvents(currentTimeMs); List> uncompletedEvents = applicationEventReaper.uncompletedEvents(); @@ -233,13 +232,11 @@ private void reapExpiredApplicationEvents(long currentTimeMs) { * */ // Visible for testing - static void runAtClose(final Collection> requestManagers, + static void runAtClose(final Collection requestManagers, final NetworkClientDelegate networkClientDelegate, final long currentTimeMs) { // These are the optional outgoing requests at the requestManagers.stream() - .filter(Optional::isPresent) - .map(Optional::get) .map(rm -> rm.pollOnClose(currentTimeMs)) .forEach(networkClientDelegate::addAll); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java index cab7d804cadd6..f341dc35a4a53 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestManagers.java @@ -60,7 +60,7 @@ public class RequestManagers implements Closeable { public final FetchRequestManager fetchRequestManager; public final Optional shareConsumeRequestManager; public final Optional streamsGroupHeartbeatRequestManager; - private final List> entries; + private final List entries; private final IdempotentCloser closer = new IdempotentCloser(); public RequestManagers(LogContext logContext, @@ -87,16 +87,16 @@ public RequestManagers(LogContext logContext, this.streamsMembershipManager = streamsMembershipManager; this.shareMembershipManager = Optional.empty(); - List> list = new ArrayList<>(); - list.add(coordinatorRequestManager); - list.add(commitRequestManager); - list.add(heartbeatRequestManager); - list.add(membershipManager); - list.add(streamsGroupHeartbeatRequestManager); - list.add(streamsMembershipManager); - list.add(Optional.of(offsetsRequestManager)); - list.add(Optional.of(topicMetadataRequestManager)); - list.add(Optional.of(fetchRequestManager)); + List list = new ArrayList<>(); + coordinatorRequestManager.ifPresent(list::add); + commitRequestManager.ifPresent(list::add); + heartbeatRequestManager.ifPresent(list::add); + membershipManager.ifPresent(list::add); + streamsGroupHeartbeatRequestManager.ifPresent(list::add); + streamsMembershipManager.ifPresent(list::add); + list.add(offsetsRequestManager); + list.add(topicMetadataRequestManager); + list.add(fetchRequestManager); entries = Collections.unmodifiableList(list); } @@ -119,15 +119,15 @@ public RequestManagers(LogContext logContext, this.topicMetadataRequestManager = null; this.fetchRequestManager = null; - List> list = new ArrayList<>(); - list.add(coordinatorRequestManager); - list.add(shareHeartbeatRequestManager); - list.add(shareMembershipManager); - list.add(Optional.of(shareConsumeRequestManager)); + List list = new ArrayList<>(); + coordinatorRequestManager.ifPresent(list::add); + shareHeartbeatRequestManager.ifPresent(list::add); + shareMembershipManager.ifPresent(list::add); + list.add(shareConsumeRequestManager); entries = Collections.unmodifiableList(list); } - public List> entries() { + public List entries() { return entries; } @@ -138,8 +138,6 @@ public void close() { log.debug("Closing RequestManagers"); entries.stream() - .filter(Optional::isPresent) - .map(Optional::get) .filter(rm -> rm instanceof Closeable) .map(rm -> (Closeable) rm) .forEach(c -> closeQuietly(c, c.getClass().getSimpleName())); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index bb5193f8dc6b9..081fecf78db5b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -567,7 +567,7 @@ public synchronized ConsumerRecords poll(final Duration timeout) { acquireAndEnsureOpen(); try { // Handle any completed acknowledgements for which we already have the responses - handleCompletedAcknowledgements(); + handleCompletedAcknowledgements(false); // If using implicit acknowledgement, acknowledge the previously fetched records acknowledgeBatchIfImplicitAcknowledgement(); @@ -708,7 +708,7 @@ public Map> commitSync(final Duration acquireAndEnsureOpen(); try { // Handle any completed acknowledgements for which we already have the responses - handleCompletedAcknowledgements(); + handleCompletedAcknowledgements(false); // If using implicit acknowledgement, acknowledge the previously fetched records acknowledgeBatchIfImplicitAcknowledgement(); @@ -752,7 +752,7 @@ public void commitAsync() { acquireAndEnsureOpen(); try { // Handle any completed acknowledgements for which we already have the responses - handleCompletedAcknowledgements(); + handleCompletedAcknowledgements(false); // If using implicit acknowledgement, acknowledge the previously fetched records acknowledgeBatchIfImplicitAcknowledgement(); @@ -883,7 +883,7 @@ private void close(final Duration timeout, final boolean swallowException) { swallow(log, Level.ERROR, "Failed to stop finding coordinator", this::stopFindCoordinatorOnClose, firstException); swallow(log, Level.ERROR, "Failed invoking acknowledgement commit callback", - this::handleCompletedAcknowledgements, firstException); + () -> handleCompletedAcknowledgements(true), firstException); if (applicationEventHandler != null) closeQuietly(() -> applicationEventHandler.close(Duration.ofMillis(closeTimer.remainingMs())), "Failed shutting down network thread", firstException); closeTimer.update(); @@ -1017,8 +1017,12 @@ private void maybeThrowInvalidGroupIdException() { *

* If the acknowledgement commit callback throws an exception, this method will throw an exception. */ - private void handleCompletedAcknowledgements() { - processBackgroundEvents(); + private void handleCompletedAcknowledgements(boolean onClose) { + // If the user gets any fatal errors, they will get these exceptions in the background queue. + // While closing, we ignore these exceptions so that the consumers close successfully. + processBackgroundEvents(onClose ? e -> (e instanceof GroupAuthorizationException + || e instanceof TopicAuthorizationException + || e instanceof InvalidTopicException) : e -> false); if (!completedAcknowledgements.isEmpty()) { try { @@ -1065,6 +1069,15 @@ private static ShareAcknowledgementMode initializeAcknowledgementMode(ConsumerCo return ShareAcknowledgementMode.fromString(s); } + private void processBackgroundEvents(final Predicate ignoreErrorEventException) { + try { + processBackgroundEvents(); + } catch (Exception e) { + if (!ignoreErrorEventException.test(e)) + throw e; + } + } + /** * Process the events—if any—that were produced by the {@link ConsumerNetworkThread network thread}. * It is possible that {@link ErrorEvent an error} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandler.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandler.java index 34a109944bea5..634a9839c5d00 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandler.java @@ -219,7 +219,8 @@ private String topicIdPartitionsToLogString(Collection partiti */ public boolean handleResponse(ShareFetchResponse response, short version) { if ((response.error() == Errors.SHARE_SESSION_NOT_FOUND) || - (response.error() == Errors.INVALID_SHARE_SESSION_EPOCH)) { + (response.error() == Errors.INVALID_SHARE_SESSION_EPOCH) || + (response.error() == Errors.SHARE_SESSION_LIMIT_REACHED)) { log.info("Node {} was unable to process the ShareFetch request with {}: {}.", node, nextMetadata, response.error()); nextMetadata = nextMetadata.nextCloseExistingAttemptNew(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSet.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSet.java new file mode 100644 index 0000000000000..920fb63515df3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSet.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; + +/** + * Represents a set of topic partitions, where each entry contains topic ID, topic name and partition number. + * Keeps in-memory references to provide easy access to this data in different forms. + * (ex. retrieve topic IDs only, topic names, partitions with topic names, partitions with topic IDs) + * Data is kept sorted by topic name and partition number, for improved logging. + */ +public class TopicIdPartitionSet { + + /** + * TopicPartition comparator based on topic name and partition. + */ + static final Utils.TopicPartitionComparator TOPIC_PARTITION_COMPARATOR = new Utils.TopicPartitionComparator(); + + /** + * TopicIdPartition comparator based on topic name and partition. + * (Ignoring topic ID while sorting, as this is sorted mainly for logging purposes). + */ + static final Utils.TopicIdPartitionComparator TOPIC_ID_PARTITION_COMPARATOR = new Utils.TopicIdPartitionComparator(); + + private final SortedSet topicIdPartitions; + private final SortedSet topicPartitions; + private final Set topicIds; + private final SortedSet topicNames; + + public TopicIdPartitionSet() { + this.topicIdPartitions = new TreeSet<>(TOPIC_ID_PARTITION_COMPARATOR); + this.topicPartitions = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); + this.topicIds = new HashSet<>(); + this.topicNames = new TreeSet<>(); + } + + /** + * Add a new topic (id+name) and partition. This will keep it, and also save references to the topic ID, topic name and partition. + */ + public void add(TopicIdPartition topicIdPartition) { + topicIdPartitions.add(topicIdPartition); + topicPartitions.add(topicIdPartition.topicPartition()); + topicIds.add(topicIdPartition.topicId()); + topicNames.add(topicIdPartition.topicPartition().topic()); + } + + public boolean isEmpty() { + return this.topicIdPartitions.isEmpty(); + } + + public SortedSet topicPartitions() { + return Collections.unmodifiableSortedSet(topicPartitions); + } + + public Set topicIds() { + return Collections.unmodifiableSet(topicIds); + } + + public SortedSet topicNames() { + return Collections.unmodifiableSortedSet(topicNames); + } + + /** + * @return Map of partition numbers per topic ID, sorted by topic names (for improved logging). + */ + public Map> toTopicIdPartitionMap() { + Map> partitions = new HashMap<>(); + topicIdPartitions.forEach(topicIdPartition -> { + Uuid topicId = topicIdPartition.topicId(); + partitions.computeIfAbsent(topicId, k -> new TreeSet<>()).add(topicIdPartition.partition()); + }); + return partitions; + } + + /** + * @return Set of topic partitions (with topic name and partition number) + */ + protected SortedSet toTopicNamePartitionSet() { + SortedSet result = new TreeSet<>(TOPIC_PARTITION_COMPARATOR); + topicIdPartitions.forEach(topicIdPartition -> result.add(topicIdPartition.topicPartition())); + return result; + } + + @Override + public String toString() { + return this.topicIdPartitions.toString(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index ec3721389a489..3fcea1968b8e1 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -111,10 +111,12 @@ public class TopicConfig { "The largest record batch size allowed by Kafka (after compression if compression is enabled)."; public static final String INDEX_INTERVAL_BYTES_CONFIG = "index.interval.bytes"; - public static final String INDEX_INTERVAL_BYTES_DOC = "This setting controls how frequently " + - "Kafka adds an index entry to its offset index. The default setting ensures that we index a " + - "message roughly every 4096 bytes. More indexing allows reads to jump closer to the exact " + - "position in the log but makes the index larger. You probably don't need to change this."; + public static final String INDEX_INTERVAL_BYTES_DOC = "This setting controls how frequently Kafka " + + "adds entries to its offset index and, conditionally, to its time index. " + + "The default setting ensures that we index a message roughly every 4096 bytes. " + + "More frequent indexing allows reads to jump closer to the exact position in the log " + + "but results in larger index files. You probably don't need to change this." + + "

Note: the time index will be inserted only when the timestamp is greater than the last indexed timestamp.

"; public static final String FILE_DELETE_DELAY_MS_CONFIG = "file.delete.delay.ms"; public static final String FILE_DELETE_DELAY_MS_DOC = "The time to wait before deleting a file from the " + diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 9092db41783ba..a27a7fcf23c77 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -229,7 +229,7 @@ public enum Errors { "The group member's supported protocols are incompatible with those of existing members " + "or first group member tried to join with empty protocol type or empty protocol list.", InconsistentGroupProtocolException::new), - INVALID_GROUP_ID(24, "The configured groupId is invalid.", + INVALID_GROUP_ID(24, "The group id is invalid.", InvalidGroupIdException::new), UNKNOWN_MEMBER_ID(25, "The coordinator is not aware of this member.", UnknownMemberIdException::new), diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java index 7f78235ab70d4..6a42a52d2e0c9 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java @@ -120,19 +120,8 @@ public void readInto(ByteBuffer buffer, int position) throws IOException { buffer.flip(); } - /** - * Return a slice of records from this instance, which is a view into this set starting from the given position - * and with the given size limit. - * - * If the size is beyond the end of the file, the end will be based on the size of the file at the time of the read. - * - * If this message set is already sliced, the position will be taken relative to that slicing. - * - * @param position The start position to begin the read from - * @param size The number of bytes after the start position to include - * @return A sliced wrapper on this message set limited based on the given position and size - */ - public FileRecords slice(int position, int size) throws IOException { + @Override + public Records slice(int position, int size) throws IOException { int availableBytes = availableBytes(position, size); int startPosition = this.start + position; return new FileRecords(file, channel, startPosition, startPosition + availableBytes, true); diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index c2fd231e4b7cd..1786f61d18796 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -300,6 +300,31 @@ public ByteBuffer buffer() { return buffer.duplicate(); } + @Override + public Records slice(int position, int size) { + if (position < 0) + throw new IllegalArgumentException("Invalid position: " + position + " in read from " + this); + if (position > buffer.limit()) + throw new IllegalArgumentException("Slice from position " + position + " exceeds end position of " + this); + if (size < 0) + throw new IllegalArgumentException("Invalid size: " + size + " in read from " + this); + + int availableBytes = Math.min(size, buffer.limit() - position); + // As of now, clients module support Java11 hence can't use ByteBuffer::slice(position, size) method. + // So we need to create a duplicate buffer and set the position and limit. Duplicate buffer + // is backed by original bytes hence not the content but only the relative position and limit + // are changed in the duplicate buffer. Once the position and limit are set, we can call the + // slice method to get the sliced buffer, which is a backed by the original buffer with the + // position reset to 0 and limit set to the size of the slice. + ByteBuffer slicedBuffer = buffer.duplicate(); + slicedBuffer.position(position); + slicedBuffer.limit(position + availableBytes); + // Reset the position to 0 so that the sliced view has a relative position. + slicedBuffer = slicedBuffer.slice(); + + return readableRecords(slicedBuffer); + } + @Override public Iterable batches() { return batches; diff --git a/clients/src/main/java/org/apache/kafka/common/record/Records.java b/clients/src/main/java/org/apache/kafka/common/record/Records.java index ec710394bec09..3d45762e81518 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Records.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Records.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.utils.AbstractIterator; +import java.io.IOException; import java.util.Iterator; import java.util.Optional; @@ -90,4 +91,19 @@ public interface Records extends TransferableRecords { * @return The record iterator */ Iterable records(); + + /** + * Return a slice of records from this instance, which is a view into this set starting from the given position + * and with the given size limit. + * + * If the size is beyond the end of the records, the end will be based on the size of the records at the time of the read. + * + * If this records set is already sliced, the position will be taken relative to that slicing. + * + * @param position The start position to begin the read from. The position should be aligned to + * the batch boundary, else the returned records can't be iterated. + * @param size The number of bytes after the start position to include + * @return A sliced wrapper on this message set limited based on the given position and size + */ + Records slice(int position, int size) throws IOException; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java index 6dfbcca955a7e..84f7cc2a72d69 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsRequest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.common.GroupType; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.ListGroupsRequestData; import org.apache.kafka.common.message.ListGroupsResponseData; @@ -24,6 +25,8 @@ import org.apache.kafka.common.protocol.Readable; import java.util.Collections; +import java.util.HashSet; +import java.util.List; /** * Possible error codes: @@ -50,8 +53,19 @@ public ListGroupsRequest build(short version) { "v" + version + ", but we need v4 or newer to request groups by states."); } if (!data.typesFilter().isEmpty() && version < 5) { - throw new UnsupportedVersionException("The broker only supports ListGroups " + - "v" + version + ", but we need v5 or newer to request groups by type."); + // Types filter is supported by brokers with version 3.8.0 or later. Older brokers only support + // classic groups, so listing consumer groups on an older broker does not need to use a types filter. + // If the types filter is only for consumer and classic, or just classic groups, it can be safely omitted. + // This allows a modern admin client to list consumer groups on older brokers in a straightforward way. + HashSet typesCopy = new HashSet<>(data.typesFilter()); + boolean containedClassic = typesCopy.remove(GroupType.CLASSIC.toString()); + boolean containedConsumer = typesCopy.remove(GroupType.CONSUMER.toString()); + if (!typesCopy.isEmpty() || (!containedClassic && containedConsumer)) { + throw new UnsupportedVersionException("The broker only supports ListGroups " + + "v" + version + ", but we need v5 or newer to request groups by type. " + + "Requested group types: [" + String.join(", ", data.typesFilter()) + "]."); + } + return new ListGroupsRequest(data.duplicate().setTypesFilter(List.of()), version); } return new ListGroupsRequest(data, version); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java index 5f7ea5b157d7c..34c39625972c9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsRequest.java @@ -38,6 +38,10 @@ public ListTransactionsRequest build(short version) { throw new UnsupportedVersionException("Duration filter can be set only when using API version 1 or higher." + " If client is connected to an older broker, do not specify duration filter or set duration filter to -1."); } + if (data.transactionalIdPattern() != null && version < 2) { + throw new UnsupportedVersionException("Transactional ID pattern filter can be set only when using API version 2 or higher." + + " If client is connected to an older broker, do not specify the pattern filter."); + } return new ListTransactionsRequest(data, version); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index 2a26e7940d39d..377169ed1c652 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -41,9 +41,9 @@ public class OffsetFetchRequest extends AbstractRequest { private static final Logger log = LoggerFactory.getLogger(OffsetFetchRequest.class); - private static final short TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION = 2; - private static final short REQUIRE_STABLE_OFFSET_MIN_VERSION = 7; - private static final short BATCH_MIN_VERSION = 8; + public static final short TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION = 2; + public static final short REQUIRE_STABLE_OFFSET_MIN_VERSION = 7; + public static final short BATCH_MIN_VERSION = 8; private final OffsetFetchRequestData data; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 7b60ba59c5873..e76ea8f7f3b58 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -16,30 +16,25 @@ */ package org.apache.kafka.common.requests; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.OffsetFetchResponseData; import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseGroup; import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponsePartition; -import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponsePartitions; import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopic; -import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopics; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Readable; -import java.util.ArrayList; import java.util.Arrays; import java.util.EnumMap; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; -import java.util.Objects; -import java.util.Optional; +import java.util.function.Function; import java.util.stream.Collectors; import static org.apache.kafka.common.record.RecordBatch.NO_PARTITION_LEADER_EPOCH; +import static org.apache.kafka.common.requests.OffsetFetchRequest.BATCH_MIN_VERSION; +import static org.apache.kafka.common.requests.OffsetFetchRequest.TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION; /** * Possible error codes: @@ -60,221 +55,127 @@ public class OffsetFetchResponse extends AbstractResponse { public static final long INVALID_OFFSET = -1L; public static final String NO_METADATA = ""; - public static final PartitionData UNKNOWN_PARTITION = new PartitionData(INVALID_OFFSET, - Optional.empty(), - NO_METADATA, - Errors.UNKNOWN_TOPIC_OR_PARTITION); - public static final PartitionData UNAUTHORIZED_PARTITION = new PartitionData(INVALID_OFFSET, - Optional.empty(), - NO_METADATA, - Errors.TOPIC_AUTHORIZATION_FAILED); + + // We only need to track the partition errors returned in version 1. This + // is used to identify group level errors when the response is normalized. private static final List PARTITION_ERRORS = Arrays.asList( - Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.TOPIC_AUTHORIZATION_FAILED); + Errors.UNKNOWN_TOPIC_OR_PARTITION, + Errors.TOPIC_AUTHORIZATION_FAILED + ); + private final short version; private final OffsetFetchResponseData data; - private final Errors error; - private final Map groupLevelErrors = new HashMap<>(); - - public static final class PartitionData { - public final long offset; - public final String metadata; - public final Errors error; - public final Optional leaderEpoch; + // Lazily initialized when OffsetFetchResponse#group is called. + private Map groups = null; - public PartitionData(long offset, - Optional leaderEpoch, - String metadata, - Errors error) { - this.offset = offset; - this.leaderEpoch = leaderEpoch; - this.metadata = metadata; - this.error = error; - } - - public boolean hasError() { - return this.error != Errors.NONE; - } - - @Override - public boolean equals(Object other) { - if (!(other instanceof PartitionData)) - return false; - PartitionData otherPartition = (PartitionData) other; - return Objects.equals(this.offset, otherPartition.offset) - && Objects.equals(this.leaderEpoch, otherPartition.leaderEpoch) - && Objects.equals(this.metadata, otherPartition.metadata) - && Objects.equals(this.error, otherPartition.error); - } - - @Override - public String toString() { - return "PartitionData(" - + "offset=" + offset - + ", leaderEpoch=" + leaderEpoch.orElse(NO_PARTITION_LEADER_EPOCH) - + ", metadata=" + metadata - + ", error='" + error.toString() - + ")"; - } + public static class Builder { + private final List groups; - @Override - public int hashCode() { - return Objects.hash(offset, leaderEpoch, metadata, error); + public Builder(OffsetFetchResponseGroup group) { + this(List.of(group)); } - } - - /** - * Constructor without throttle time. - * @param error Potential coordinator or group level error code (for api version 2 and later) - * @param responseData Fetched offset information grouped by topic-partition - */ - public OffsetFetchResponse(Errors error, Map responseData) { - this(DEFAULT_THROTTLE_TIME, error, responseData); - } - /** - * Constructor with throttle time for version 0 to 7 - * @param throttleTimeMs The time in milliseconds that this response was throttled - * @param error Potential coordinator or group level error code (for api version 2 and later) - * @param responseData Fetched offset information grouped by topic-partition - */ - public OffsetFetchResponse(int throttleTimeMs, Errors error, Map responseData) { - super(ApiKeys.OFFSET_FETCH); - Map offsetFetchResponseTopicMap = new HashMap<>(); - for (Map.Entry entry : responseData.entrySet()) { - String topicName = entry.getKey().topic(); - OffsetFetchResponseTopic topic = offsetFetchResponseTopicMap.getOrDefault( - topicName, new OffsetFetchResponseTopic().setName(topicName)); - PartitionData partitionData = entry.getValue(); - topic.partitions().add(new OffsetFetchResponsePartition() - .setPartitionIndex(entry.getKey().partition()) - .setErrorCode(partitionData.error.code()) - .setCommittedOffset(partitionData.offset) - .setCommittedLeaderEpoch( - partitionData.leaderEpoch.orElse(NO_PARTITION_LEADER_EPOCH)) - .setMetadata(partitionData.metadata) - ); - offsetFetchResponseTopicMap.put(topicName, topic); + public Builder(List groups) { + this.groups = groups; } - this.data = new OffsetFetchResponseData() - .setTopics(new ArrayList<>(offsetFetchResponseTopicMap.values())) - .setErrorCode(error.code()) - .setThrottleTimeMs(throttleTimeMs); - this.error = error; - } + public OffsetFetchResponse build(short version) { + var data = new OffsetFetchResponseData(); - /** - * Constructor with throttle time for version 8 and above. - * @param throttleTimeMs The time in milliseconds that this response was throttled - * @param errors Potential coordinator or group level error code - * @param responseData Fetched offset information grouped by topic-partition and by group - */ - public OffsetFetchResponse(int throttleTimeMs, - Map errors, - Map> responseData) { - super(ApiKeys.OFFSET_FETCH); - List groupList = new ArrayList<>(); - for (Entry> entry : responseData.entrySet()) { - String groupName = entry.getKey(); - Map partitionDataMap = entry.getValue(); - Map offsetFetchResponseTopicsMap = new HashMap<>(); - for (Entry partitionEntry : partitionDataMap.entrySet()) { - String topicName = partitionEntry.getKey().topic(); - OffsetFetchResponseTopics topic = - offsetFetchResponseTopicsMap.getOrDefault(topicName, - new OffsetFetchResponseTopics().setName(topicName)); - PartitionData partitionData = partitionEntry.getValue(); - topic.partitions().add(new OffsetFetchResponsePartitions() - .setPartitionIndex(partitionEntry.getKey().partition()) - .setErrorCode(partitionData.error.code()) - .setCommittedOffset(partitionData.offset) - .setCommittedLeaderEpoch( - partitionData.leaderEpoch.orElse(NO_PARTITION_LEADER_EPOCH)) - .setMetadata(partitionData.metadata)); - offsetFetchResponseTopicsMap.put(topicName, topic); - } - groupList.add(new OffsetFetchResponseGroup() - .setGroupId(groupName) - .setTopics(new ArrayList<>(offsetFetchResponseTopicsMap.values())) - .setErrorCode(errors.get(groupName).code())); - groupLevelErrors.put(groupName, errors.get(groupName)); - } - this.data = new OffsetFetchResponseData() - .setGroups(groupList) - .setThrottleTimeMs(throttleTimeMs); - this.error = null; - } - - public OffsetFetchResponse(List groups, short version) { - super(ApiKeys.OFFSET_FETCH); - data = new OffsetFetchResponseData(); - - if (version >= 8) { - data.setGroups(groups); - error = null; + if (version >= BATCH_MIN_VERSION) { + data.setGroups(groups); + } else { + if (groups.size() != 1) { + throw new UnsupportedVersionException( + "Version " + version + " of OffsetFetchResponse only supports one group." + ); + } - for (OffsetFetchResponseGroup group : data.groups()) { - this.groupLevelErrors.put(group.groupId(), Errors.forCode(group.errorCode())); - } - } else { - if (groups.size() != 1) { - throw new UnsupportedVersionException( - "Version " + version + " of OffsetFetchResponse only supports one group." - ); + OffsetFetchResponseGroup group = groups.get(0); + data.setErrorCode(group.errorCode()); + + group.topics().forEach(topic -> { + OffsetFetchResponseTopic newTopic = new OffsetFetchResponseTopic().setName(topic.name()); + data.topics().add(newTopic); + + topic.partitions().forEach(partition -> { + OffsetFetchResponsePartition newPartition; + + if (version < TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION && group.errorCode() != Errors.NONE.code()) { + // Versions prior to version 2 do not support a top level error. Therefore, + // we put it at the partition level. + newPartition = new OffsetFetchResponsePartition() + .setPartitionIndex(partition.partitionIndex()) + .setErrorCode(group.errorCode()) + .setCommittedOffset(INVALID_OFFSET) + .setMetadata(NO_METADATA) + .setCommittedLeaderEpoch(NO_PARTITION_LEADER_EPOCH); + } else { + newPartition = new OffsetFetchResponsePartition() + .setPartitionIndex(partition.partitionIndex()) + .setErrorCode(partition.errorCode()) + .setCommittedOffset(partition.committedOffset()) + .setMetadata(partition.metadata()) + .setCommittedLeaderEpoch(partition.committedLeaderEpoch()); + } + + newTopic.partitions().add(newPartition); + }); + }); } - OffsetFetchResponseGroup group = groups.get(0); - data.setErrorCode(group.errorCode()); - error = Errors.forCode(group.errorCode()); - - group.topics().forEach(topic -> { - OffsetFetchResponseTopic newTopic = new OffsetFetchResponseTopic().setName(topic.name()); - data.topics().add(newTopic); - - topic.partitions().forEach(partition -> { - OffsetFetchResponsePartition newPartition; - - if (version < 2 && group.errorCode() != Errors.NONE.code()) { - // Versions prior to version 2 do not support a top level error. Therefore, - // we put it at the partition level. - newPartition = new OffsetFetchResponsePartition() - .setPartitionIndex(partition.partitionIndex()) - .setErrorCode(group.errorCode()) - .setCommittedOffset(INVALID_OFFSET) - .setMetadata(NO_METADATA) - .setCommittedLeaderEpoch(NO_PARTITION_LEADER_EPOCH); - } else { - newPartition = new OffsetFetchResponsePartition() - .setPartitionIndex(partition.partitionIndex()) - .setErrorCode(partition.errorCode()) - .setCommittedOffset(partition.committedOffset()) - .setMetadata(partition.metadata()) - .setCommittedLeaderEpoch(partition.committedLeaderEpoch()); - } - - newTopic.partitions().add(newPartition); - }); - }); + return new OffsetFetchResponse(data, version); } } public OffsetFetchResponse(OffsetFetchResponseData data, short version) { super(ApiKeys.OFFSET_FETCH); this.data = data; - // for version 2 and later use the top-level error code (in ERROR_CODE_KEY_NAME) from the response. - // for older versions there is no top-level error in the response and all errors are partition errors, - // so if there is a group or coordinator error at the partition level use that as the top-level error. - // this way clients can depend on the top-level error regardless of the offset fetch version. - // we return the error differently starting with version 8, so we will only populate the - // error field if we are between version 2 and 7. if we are in version 8 or greater, then - // we will populate the map of group id to error codes. - if (version < 8) { - this.error = version >= 2 ? Errors.forCode(data.errorCode()) : topLevelError(data); + this.version = version; + } + + public OffsetFetchResponseData.OffsetFetchResponseGroup group(String groupId) { + if (version < BATCH_MIN_VERSION) { + // for version 2 and later use the top-level error code from the response. + // for older versions there is no top-level error in the response and all errors are partition errors, + // so if there is a group or coordinator error at the partition level use that as the top-level error. + // this way clients can depend on the top-level error regardless of the offset fetch version. + // we return the error differently starting with version 8, so we will only populate the + // error field if we are between version 2 and 7. if we are in version 8 or greater, then + // we will populate the map of group id to error codes. + short topLevelError = version < TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION ? topLevelError(data).code() : data.errorCode(); + if (topLevelError != Errors.NONE.code()) { + return new OffsetFetchResponseGroup() + .setGroupId(groupId) + .setErrorCode(topLevelError); + } else { + return new OffsetFetchResponseGroup() + .setGroupId(groupId) + .setTopics(data.topics().stream().map(topic -> + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topic.name()) + .setPartitions(topic.partitions().stream().map(partition -> + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(partition.partitionIndex()) + .setErrorCode(partition.errorCode()) + .setCommittedOffset(partition.committedOffset()) + .setMetadata(partition.metadata()) + .setCommittedLeaderEpoch(partition.committedLeaderEpoch()) + ).collect(Collectors.toList())) + ).collect(Collectors.toList())); + } } else { - for (OffsetFetchResponseGroup group : data.groups()) { - this.groupLevelErrors.put(group.groupId(), Errors.forCode(group.errorCode())); + if (groups == null) { + groups = data.groups().stream().collect(Collectors.toMap( + OffsetFetchResponseData.OffsetFetchResponseGroup::groupId, + Function.identity() + )); + } + var group = groups.get(groupId); + if (group == null) { + throw new IllegalArgumentException("Group " + groupId + " not found in the response"); } - this.error = null; + return group; } } @@ -300,94 +201,29 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { data.setThrottleTimeMs(throttleTimeMs); } - public boolean hasError() { - return error != Errors.NONE; - } - - public boolean groupHasError(String groupId) { - Errors error = groupLevelErrors.get(groupId); - if (error == null) { - return this.error != null && this.error != Errors.NONE; - } - return error != Errors.NONE; - } - - public Errors error() { - return error; - } - - public Errors groupLevelError(String groupId) { - if (error != null) { - return error; - } - return groupLevelErrors.get(groupId); - } - @Override public Map errorCounts() { Map counts = new EnumMap<>(Errors.class); - if (!groupLevelErrors.isEmpty()) { - // built response with v8 or above - for (Map.Entry entry : groupLevelErrors.entrySet()) { - updateErrorCounts(counts, entry.getValue()); + if (version < BATCH_MIN_VERSION) { + if (version >= TOP_LEVEL_ERROR_AND_NULL_TOPICS_MIN_VERSION) { + updateErrorCounts(counts, Errors.forCode(data.errorCode())); } - for (OffsetFetchResponseGroup group : data.groups()) { - group.topics().forEach(topic -> - topic.partitions().forEach(partition -> - updateErrorCounts(counts, Errors.forCode(partition.errorCode())))); - } - } else { - // built response with v0-v7 - updateErrorCounts(counts, error); data.topics().forEach(topic -> topic.partitions().forEach(partition -> - updateErrorCounts(counts, Errors.forCode(partition.errorCode())))); - } - return counts; - } - - // package-private for testing purposes - Map responseDataV0ToV7() { - Map responseData = new HashMap<>(); - for (OffsetFetchResponseTopic topic : data.topics()) { - for (OffsetFetchResponsePartition partition : topic.partitions()) { - responseData.put(new TopicPartition(topic.name(), partition.partitionIndex()), - new PartitionData(partition.committedOffset(), - RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()), - partition.metadata(), - Errors.forCode(partition.errorCode())) - ); - } - } - return responseData; - } - - private Map buildResponseData(String groupId) { - Map responseData = new HashMap<>(); - OffsetFetchResponseGroup group = data - .groups() - .stream() - .filter(g -> g.groupId().equals(groupId)) - .collect(Collectors.toList()) - .get(0); - for (OffsetFetchResponseTopics topic : group.topics()) { - for (OffsetFetchResponsePartitions partition : topic.partitions()) { - responseData.put(new TopicPartition(topic.name(), partition.partitionIndex()), - new PartitionData(partition.committedOffset(), - RequestUtils.getLeaderEpoch(partition.committedLeaderEpoch()), - partition.metadata(), - Errors.forCode(partition.errorCode())) + updateErrorCounts(counts, Errors.forCode(partition.errorCode())) + ) + ); + } else { + data.groups().forEach(group -> { + updateErrorCounts(counts, Errors.forCode(group.errorCode())); + group.topics().forEach(topic -> + topic.partitions().forEach(partition -> + updateErrorCounts(counts, Errors.forCode(partition.errorCode())) + ) ); - } - } - return responseData; - } - - public Map partitionDataMap(String groupId) { - if (groupLevelErrors.isEmpty()) { - return responseDataV0ToV7(); + }); } - return buildResponseData(groupId); + return counts; } public static OffsetFetchResponse parse(Readable readable, short version) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ReadShareGroupStateSummaryResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ReadShareGroupStateSummaryResponse.java index 86363add1e10d..a2787ff82c96e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ReadShareGroupStateSummaryResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ReadShareGroupStateSummaryResponse.java @@ -99,6 +99,7 @@ public static ReadShareGroupStateSummaryResponseData toResponseData( Uuid topicId, int partition, long startOffset, + int leaderEpoch, int stateEpoch ) { return new ReadShareGroupStateSummaryResponseData() @@ -109,6 +110,7 @@ public static ReadShareGroupStateSummaryResponseData toResponseData( new ReadShareGroupStateSummaryResponseData.PartitionResult() .setPartition(partition) .setStartOffset(startOffset) + .setLeaderEpoch(leaderEpoch) .setStateEpoch(stateEpoch) )) )); diff --git a/clients/src/main/resources/common/message/ListTransactionsRequest.json b/clients/src/main/resources/common/message/ListTransactionsRequest.json index 5d7c688da2213..57d42e6b99c41 100644 --- a/clients/src/main/resources/common/message/ListTransactionsRequest.json +++ b/clients/src/main/resources/common/message/ListTransactionsRequest.json @@ -19,7 +19,9 @@ "listeners": ["broker"], "name": "ListTransactionsRequest", // Version 1: adds DurationFilter to list transactions older than specified duration - "validVersions": "0-1", + + // Version 2: adds TransactionalIdPattern to list transactions with the same pattern(KIP-1152) + "validVersions": "0-2", "flexibleVersions": "0+", "fields": [ { "name": "StateFilters", "type": "[]string", "versions": "0+", @@ -30,6 +32,9 @@ }, { "name": "DurationFilter", "type": "int64", "versions": "1+", "default": -1, "about": "Duration (in millis) to filter by: if < 0, all transactions will be returned; otherwise, only transactions running longer than this duration will be returned." + }, + { "name": "TransactionalIdPattern", "type": "string", "versions": "2+", "nullableVersions": "2+", "default": "null", + "about": "The transactional ID regular expression pattern to filter by: if it is empty or null, all transactions are returned; Otherwise then only the transactions matching the given regular expression will be returned." } ] } diff --git a/clients/src/main/resources/common/message/ListTransactionsResponse.json b/clients/src/main/resources/common/message/ListTransactionsResponse.json index 3872cf24a3075..0af1be699b435 100644 --- a/clients/src/main/resources/common/message/ListTransactionsResponse.json +++ b/clients/src/main/resources/common/message/ListTransactionsResponse.json @@ -18,7 +18,9 @@ "type": "response", "name": "ListTransactionsResponse", // Version 1 is the same as version 0 (KIP-994). - "validVersions": "0-1", + + // This API can return InvalidRegularExpression (KIP-1152). + "validVersions": "0-2", "flexibleVersions": "0+", "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", diff --git a/clients/src/main/resources/common/message/ReadShareGroupStateSummaryResponse.json b/clients/src/main/resources/common/message/ReadShareGroupStateSummaryResponse.json index ddf9d7044a6a3..81e3edc554ece 100644 --- a/clients/src/main/resources/common/message/ReadShareGroupStateSummaryResponse.json +++ b/clients/src/main/resources/common/message/ReadShareGroupStateSummaryResponse.json @@ -41,6 +41,8 @@ "about": "The error message, or null if there was no error." }, { "name": "StateEpoch", "type": "int32", "versions": "0+", "about": "The state epoch of the share-partition." }, + { "name": "LeaderEpoch", "type": "int32", "versions": "0+", + "about": "The leader epoch of the share-partition." }, { "name": "StartOffset", "type": "int64", "versions": "0+", "about": "The share-partition start offset." } ]} diff --git a/clients/src/test/java/org/apache/kafka/clients/InFlightRequestsTest.java b/clients/src/test/java/org/apache/kafka/clients/InFlightRequestsTest.java index 006de9d06d987..b60efe8950ff8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/InFlightRequestsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/InFlightRequestsTest.java @@ -102,7 +102,7 @@ public void testCompleteNext() { } @Test - public void testCompleteNextThrowsIfNoInflights() { + public void testCompleteNextThrowsIfNoInFlights() { assertThrows(IllegalStateException.class, () -> inFlightRequests.completeNext(dest)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index cef48b65bb69f..e8dcf5843dcb8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -748,24 +748,6 @@ public void testConnectionThrottling() { assertEquals(0, client.throttleDelayMs(node, time.milliseconds())); } - // Creates expected ApiVersionsResponse from the specified node, where the max protocol version for the specified - // key is set to the specified version. - private ApiVersionsResponse createExpectedApiVersionsResponse(ApiKeys key, short maxVersion) { - ApiVersionCollection versionList = new ApiVersionCollection(); - for (ApiKeys apiKey : ApiKeys.values()) { - if (apiKey == key) { - versionList.add(new ApiVersion() - .setApiKey(apiKey.id) - .setMinVersion((short) 0) - .setMaxVersion(maxVersion)); - } else versionList.add(ApiVersionsResponse.toApiVersion(apiKey)); - } - return new ApiVersionsResponse(new ApiVersionsResponseData() - .setErrorCode(Errors.NONE.code()) - .setThrottleTimeMs(0) - .setApiKeys(versionList)); - } - private int sendEmptyProduceRequest() { return sendEmptyProduceRequest(client, node.idString()); } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java index 061982e34d360..c98ffb9483f41 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/AdminClientTestUtils.java @@ -176,8 +176,8 @@ public static ListClientMetricsResourcesResult listClientMetricsResourcesResult( return new ListClientMetricsResourcesResult(future); } - public static ListShareGroupOffsetsResult createListShareGroupOffsetsResult(Map>> groupOffsets) { - Map>> coordinatorFutures = groupOffsets.entrySet().stream() + public static ListShareGroupOffsetsResult createListShareGroupOffsetsResult(Map>> groupOffsets) { + Map>> coordinatorFutures = groupOffsets.entrySet().stream() .collect(Collectors.toMap( entry -> CoordinatorKey.byGroupId(entry.getKey()), Map.Entry::getValue diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/DeleteConsumerGroupOffsetsResultTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/DeleteConsumerGroupOffsetsResultTest.java index cd2250d365fad..1e577b8319a42 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/DeleteConsumerGroupOffsetsResultTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/DeleteConsumerGroupOffsetsResultTest.java @@ -59,7 +59,7 @@ public void setUp() { } @Test - public void testTopLevelErrorConstructor() throws InterruptedException { + public void testTopLevelErrorConstructor() { partitionFutures.completeExceptionally(Errors.GROUP_AUTHORIZATION_FAILED.exception()); DeleteConsumerGroupOffsetsResult topLevelErrorResult = new DeleteConsumerGroupOffsetsResult(partitionFutures, partitions); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 319a91d740746..3df76fffd3674 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -161,6 +161,7 @@ import org.apache.kafka.common.message.OffsetFetchRequestData; import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestGroup; import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestTopics; +import org.apache.kafka.common.message.OffsetFetchResponseData; import org.apache.kafka.common.message.RemoveRaftVoterRequestData; import org.apache.kafka.common.message.RemoveRaftVoterResponseData; import org.apache.kafka.common.message.ShareGroupDescribeResponseData; @@ -239,7 +240,6 @@ import org.apache.kafka.common.requests.OffsetDeleteResponse; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; -import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; import org.apache.kafka.common.requests.RemoveRaftVoterRequest; import org.apache.kafka.common.requests.RemoveRaftVoterResponse; import org.apache.kafka.common.requests.RequestTestUtils; @@ -337,7 +337,6 @@ public class KafkaAdminClientTest { private static final Logger log = LoggerFactory.getLogger(KafkaAdminClientTest.class); private static final String GROUP_ID = "group-0"; - private static final int THROTTLE = 10; public static final Uuid REPLICA_DIRECTORY_ID = Uuid.randomUuid(); @Test @@ -503,7 +502,7 @@ public void testExplicitlyEnableTelemetryReporter() { .map(r -> (ClientTelemetryReporter) r) .collect(Collectors.toList()); - assertEquals(telemetryReporterList.size(), 1); + assertEquals(1, telemetryReporterList.size()); } } @@ -2086,7 +2085,7 @@ public void testElectLeaders() throws Exception { ElectLeadersResult results = env.adminClient().electLeaders( electionType, new HashSet<>(asList(topic1, topic2))); - assertEquals(results.partitions().get().get(topic2).get().getClass(), ClusterAuthorizationException.class); + assertEquals(ClusterAuthorizationException.class, results.partitions().get().get(topic2).get().getClass()); // Test a call where there are no errors. By mutating the internal of election results partition1Result.setErrorCode(ApiError.NONE.error().code()); @@ -2292,7 +2291,7 @@ private static DescribeLogDirsResponse prepareDescribeLogDirsResponse(Errors err private static DescribeLogDirsResponse prepareEmptyDescribeLogDirsResponse(Optional error) { DescribeLogDirsResponseData data = new DescribeLogDirsResponseData(); - if (error.isPresent()) data.setErrorCode(error.get().code()); + error.ifPresent(e -> data.setErrorCode(e.code())); return new DescribeLogDirsResponse(data); } @@ -3190,19 +3189,391 @@ public void testListGroupsEmptyGroupType() throws Exception { } } + @Test + public void testListGroupsWithProtocolTypes() throws Exception { + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + // Test with list group options. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(Set.of(), Set.of()), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable") + .setGroupType(GroupType.CONSUMER.toString()), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-2") + .setGroupState("Empty") + .setGroupType(GroupType.CONSUMER.toString())))), + env.cluster().nodeById(0)); + + final ListGroupsOptions options = new ListGroupsOptions().withProtocolTypes(Set.of("")); + final ListGroupsResult result = env.adminClient().listGroups(options); + Collection listing = result.valid().get(); + + assertEquals(1, listing.size()); + List expected = new ArrayList<>(); + expected.add(new GroupListing("group-2", Optional.of(GroupType.CONSUMER), "", Optional.of(GroupState.EMPTY))); + assertEquals(expected, listing); + assertEquals(0, result.errors().get().size()); + } + } + @Test public void testListGroupsWithTypes() throws Exception { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); - // Test with list group options. + // Test with list group options. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(Collections.emptySet(), singleton(GroupType.CONSUMER.toString())), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(asList( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable") + .setGroupType(GroupType.CONSUMER.toString()), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-2") + .setGroupState("Empty") + .setGroupType(GroupType.CONSUMER.toString())))), + env.cluster().nodeById(0)); + + final ListGroupsOptions options = new ListGroupsOptions().withTypes(singleton(GroupType.CONSUMER)); + final ListGroupsResult result = env.adminClient().listGroups(options); + Collection listing = result.valid().get(); + + assertEquals(2, listing.size()); + List expected = new ArrayList<>(); + expected.add(new GroupListing("group-2", Optional.of(GroupType.CONSUMER), "", Optional.of(GroupState.EMPTY))); + expected.add(new GroupListing("group-1", Optional.of(GroupType.CONSUMER), ConsumerProtocol.PROTOCOL_TYPE, Optional.of(GroupState.STABLE))); + assertEquals(expected, listing); + assertEquals(0, result.errors().get().size()); + } + } + + @Test + public void testListGroupsWithTypesOlderBrokerVersion() throws Exception { + ApiVersion listGroupV4 = new ApiVersion() + .setApiKey(ApiKeys.LIST_GROUPS.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 4); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(List.of(listGroupV4))); + + // Check that we cannot set a type filter with an older broker. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + env.kafkaClient().prepareUnsupportedVersionResponse(request -> + request instanceof ListGroupsRequest && !((ListGroupsRequest) request).data().typesFilter().isEmpty() + ); + + ListGroupsOptions options = new ListGroupsOptions().withTypes(Set.of(GroupType.SHARE)); + ListGroupsResult result = env.adminClient().listGroups(options); + TestUtils.assertFutureThrows(UnsupportedVersionException.class, result.all()); + + // But a type filter which is just classic groups is permitted with an older broker, because they + // only know about classic groups so the types filter can be omitted. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(Set.of(), Set.of()), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState(GroupState.STABLE.toString())))), + env.cluster().nodeById(0)); + + options = new ListGroupsOptions().withTypes(Set.of(GroupType.CLASSIC)); + result = env.adminClient().listGroups(options); + + Collection listing = result.all().get(); + assertEquals(1, listing.size()); + List expected = List.of( + new GroupListing("group-1", Optional.empty(), ConsumerProtocol.PROTOCOL_TYPE, Optional.of(GroupState.STABLE)) + ); + assertEquals(expected, listing); + + // But a type filter which is just consumer groups without classic groups is not permitted with an older broker. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + env.kafkaClient().prepareUnsupportedVersionResponse(request -> + request instanceof ListGroupsRequest && !((ListGroupsRequest) request).data().typesFilter().isEmpty() + ); + + options = new ListGroupsOptions().withTypes(Set.of(GroupType.CONSUMER)); + result = env.adminClient().listGroups(options); + TestUtils.assertFutureThrows(UnsupportedVersionException.class, result.all()); + } + } + + @Test + public void testDescribeClusterHandleUnsupportedVersionForIncludingFencedBrokers() { + ApiVersion describeClusterV1 = new ApiVersion() + .setApiKey(ApiKeys.DESCRIBE_CLUSTER.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Collections.singletonList(describeClusterV1))); + + env.kafkaClient().prepareUnsupportedVersionResponse( + request -> request instanceof DescribeClusterRequest); + + final DescribeClusterResult result = env.adminClient().describeCluster(new DescribeClusterOptions().includeFencedBrokers(true)); + TestUtils.assertFutureThrows(UnsupportedVersionException.class, result.nodes()); + } + } + + @Test + public void testListConsumerGroups() throws Exception { + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(4, 0), + AdminClientConfig.RETRIES_CONFIG, "2")) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + // Empty metadata response should be retried + env.kafkaClient().prepareResponse( + RequestTestUtils.metadataResponse( + List.of(), + env.cluster().clusterResource().clusterId(), + -1, + List.of())); + + env.kafkaClient().prepareResponse( + RequestTestUtils.metadataResponse( + env.cluster().nodes(), + env.cluster().clusterResource().clusterId(), + env.cluster().controller().id(), + List.of())); + + env.kafkaClient().prepareResponseFrom( + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable"), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-connect-1") + .setProtocolType("connector") + .setGroupState("Stable") + ))), + env.cluster().nodeById(0)); + + // handle retriable errors + env.kafkaClient().prepareResponseFrom( + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) + .setGroups(Collections.emptyList()) + ), + env.cluster().nodeById(1)); + env.kafkaClient().prepareResponseFrom( + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) + .setGroups(Collections.emptyList()) + ), + env.cluster().nodeById(1)); + env.kafkaClient().prepareResponseFrom( + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(asList( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-2") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable"), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-connect-2") + .setProtocolType("connector") + .setGroupState("Stable") + ))), + env.cluster().nodeById(1)); + + env.kafkaClient().prepareResponseFrom( + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-3") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable"), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-connect-3") + .setProtocolType("connector") + .setGroupState("Stable") + ))), + env.cluster().nodeById(2)); + + // fatal error + env.kafkaClient().prepareResponseFrom( + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()) + .setGroups(Collections.emptyList())), + env.cluster().nodeById(3)); + + final ListGroupsResult result = env.adminClient().listGroups(ListGroupsOptions.forConsumerGroups()); + TestUtils.assertFutureThrows(UnknownServerException.class, result.all()); + + Collection listings = result.valid().get(); + assertEquals(3, listings.size()); + + Set groupIds = new HashSet<>(); + for (GroupListing listing : listings) { + groupIds.add(listing.groupId()); + assertTrue(listing.groupState().isPresent()); + } + + assertEquals(Set.of("group-1", "group-2", "group-3"), groupIds); + assertEquals(1, result.errors().get().size()); + } + } + + @Test + public void testListConsumerGroupsMetadataFailure() throws Exception { + final Cluster cluster = mockCluster(3, 0); + final Time time = new MockTime(); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, + AdminClientConfig.RETRIES_CONFIG, "0")) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + // Empty metadata causes the request to fail since we have no list of brokers + // to send the ListGroups requests to + env.kafkaClient().prepareResponse( + RequestTestUtils.metadataResponse( + List.of(), + env.cluster().clusterResource().clusterId(), + -1, + List.of())); + + final ListGroupsResult result = env.adminClient().listGroups(ListGroupsOptions.forConsumerGroups()); + TestUtils.assertFutureThrows(KafkaException.class, result.all()); + } + } + + @Test + public void testListConsumerGroupsWithStates() throws Exception { + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.kafkaClient().prepareResponseFrom( + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable"), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-2") + .setGroupState("Empty")))), + env.cluster().nodeById(0)); + + final ListGroupsOptions options = ListGroupsOptions.forConsumerGroups(); + final ListGroupsResult result = env.adminClient().listGroups(options); + Collection listings = result.valid().get(); + + assertEquals(2, listings.size()); + List expected = new ArrayList<>(); + expected.add(new GroupListing("group-2", Optional.empty(), "", Optional.of(GroupState.EMPTY))); + expected.add(new GroupListing("group-1", Optional.empty(), ConsumerProtocol.PROTOCOL_TYPE, Optional.of(GroupState.STABLE))); + assertEquals(expected, listings); + assertEquals(0, result.errors().get().size()); + } + } + + @Test + public void testListConsumerGroupsWithProtocolTypes() throws Exception { + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + // Test with a specific protocol type filter in list consumer group options. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(Set.of(), Set.of(GroupType.CONSUMER.toString(), GroupType.CLASSIC.toString())), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable") + .setGroupType(GroupType.CONSUMER.toString()), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-2") + .setGroupState("Empty") + .setGroupType(GroupType.CONSUMER.toString())))), + env.cluster().nodeById(0)); + + final ListGroupsOptions options = ListGroupsOptions.forConsumerGroups().withProtocolTypes(Set.of(ConsumerProtocol.PROTOCOL_TYPE)); + final ListGroupsResult result = env.adminClient().listGroups(options); + Collection listings = result.valid().get(); + + assertEquals(1, listings.size()); + List expected = new ArrayList<>(); + expected.add(new GroupListing("group-1", Optional.of(GroupType.CONSUMER), ConsumerProtocol.PROTOCOL_TYPE, Optional.of(GroupState.STABLE))); + assertEquals(expected, listings); + assertEquals(0, result.errors().get().size()); + } + } + + @Test + public void testListConsumerGroupsWithTypes() throws Exception { + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + + // Test with a specific state filter but no type filter in list consumer group options. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(Set.of(GroupState.STABLE.toString()), Set.of(GroupType.CONSUMER.toString(), GroupType.CLASSIC.toString())), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable") + .setGroupType(GroupType.CLASSIC.toString())))), + env.cluster().nodeById(0)); + + final ListGroupsOptions options = ListGroupsOptions.forConsumerGroups().inGroupStates(Set.of(GroupState.STABLE)); + final ListGroupsResult result = env.adminClient().listGroups(options); + Collection listings = result.valid().get(); + + assertEquals(1, listings.size()); + List expected = new ArrayList<>(); + expected.add(new GroupListing("group-1", Optional.of(GroupType.CLASSIC), ConsumerProtocol.PROTOCOL_TYPE, Optional.of(GroupState.STABLE))); + assertEquals(expected, listings); + assertEquals(0, result.errors().get().size()); + + // Test with list consumer group options. env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); env.kafkaClient().prepareResponseFrom( - expectListGroupsRequestWithFilters(Collections.emptySet(), singleton(GroupType.CONSUMER.toString())), + expectListGroupsRequestWithFilters(Set.of(), Set.of(GroupType.CONSUMER.toString())), new ListGroupsResponse(new ListGroupsResponseData() .setErrorCode(Errors.NONE.code()) - .setGroups(asList( + .setGroups(List.of( new ListGroupsResponseData.ListedGroup() .setGroupId("group-1") .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) @@ -3214,151 +3585,199 @@ public void testListGroupsWithTypes() throws Exception { .setGroupType(GroupType.CONSUMER.toString())))), env.cluster().nodeById(0)); - final ListGroupsOptions options = new ListGroupsOptions().withTypes(singleton(GroupType.CONSUMER)); - final ListGroupsResult result = env.adminClient().listGroups(options); - Collection listing = result.valid().get(); + final ListGroupsOptions options2 = ListGroupsOptions.forConsumerGroups().withTypes(Set.of(GroupType.CONSUMER)); + final ListGroupsResult result2 = env.adminClient().listGroups(options2); + Collection listings2 = result2.valid().get(); - assertEquals(2, listing.size()); - List expected = new ArrayList<>(); - expected.add(new GroupListing("group-2", Optional.of(GroupType.CONSUMER), "", Optional.of(GroupState.EMPTY))); - expected.add(new GroupListing("group-1", Optional.of(GroupType.CONSUMER), ConsumerProtocol.PROTOCOL_TYPE, Optional.of(GroupState.STABLE))); - assertEquals(expected, listing); + assertEquals(2, listings2.size()); + List expected2 = new ArrayList<>(); + expected2.add(new GroupListing("group-2", Optional.of(GroupType.CONSUMER), "", Optional.of(GroupState.EMPTY))); + expected2.add(new GroupListing("group-1", Optional.of(GroupType.CONSUMER), ConsumerProtocol.PROTOCOL_TYPE, Optional.of(GroupState.STABLE))); + assertEquals(expected2, listings2); assertEquals(0, result.errors().get().size()); } } @Test - public void testListGroupsWithTypesOlderBrokerVersion() { - ApiVersion listGroupV4 = new ApiVersion() - .setApiKey(ApiKeys.LIST_GROUPS.id) - .setMinVersion((short) 0) - .setMaxVersion((short) 4); + public void testListConsumerGroupsWithStatesOlderBrokerVersion() throws Exception { + ApiVersion listGroupV3 = new ApiVersion() + .setApiKey(ApiKeys.LIST_GROUPS.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 3); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { - env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Collections.singletonList(listGroupV4))); + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(List.of(listGroupV3))); env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); - // Check that we cannot set a type filter with an older broker. + // Check we can list groups v3 with older broker if we don't specify states, and use just consumer group types which can be omitted. + env.kafkaClient().prepareResponseFrom( + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE)))), + env.cluster().nodeById(0)); + + ListGroupsOptions options = ListGroupsOptions.forConsumerGroups(); + ListGroupsResult result = env.adminClient().listGroups(options); + Collection listing = result.all().get(); + assertEquals(1, listing.size()); + List expected = List.of(new GroupListing("group-1", Optional.empty(), ConsumerProtocol.PROTOCOL_TYPE, Optional.empty())); + assertEquals(expected, listing); + + // But we cannot set a state filter with older broker env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + env.kafkaClient().prepareUnsupportedVersionResponse(request -> - request instanceof ListGroupsRequest && !((ListGroupsRequest) request).data().typesFilter().isEmpty() + request instanceof ListGroupsRequest && + !((ListGroupsRequest) request).data().statesFilter().isEmpty() ); - ListGroupsOptions options = new ListGroupsOptions().withTypes(singleton(GroupType.CLASSIC)); - ListGroupsResult result = env.adminClient().listGroups(options); + options = ListGroupsOptions.forConsumerGroups().inGroupStates(Set.of(GroupState.STABLE)); + result = env.adminClient().listGroups(options); TestUtils.assertFutureThrows(UnsupportedVersionException.class, result.all()); } } @Test - public void testDescribeClusterHandleUnsupportedVersionForIncludingFencedBrokers() { - ApiVersion describeClusterV1 = new ApiVersion() - .setApiKey(ApiKeys.DESCRIBE_CLUSTER.id) + public void testListConsumerGroupsWithTypesOlderBrokerVersion() throws Exception { + ApiVersion listGroupV4 = new ApiVersion() + .setApiKey(ApiKeys.LIST_GROUPS.id) .setMinVersion((short) 0) - .setMaxVersion((short) 1); + .setMaxVersion((short) 4); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { - env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Collections.singletonList(describeClusterV1))); + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(List.of(listGroupV4))); - env.kafkaClient().prepareUnsupportedVersionResponse( - request -> request instanceof DescribeClusterRequest); + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); - final DescribeClusterResult result = env.adminClient().describeCluster(new DescribeClusterOptions().includeFencedBrokers(true)); - TestUtils.assertFutureThrows(UnsupportedVersionException.class, result.nodes()); + // Check if we can list groups v4 with older broker if we specify states and don't specify types. + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(Set.of(GroupState.STABLE.toString()), Set.of()), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState(GroupState.STABLE.toString())))), + env.cluster().nodeById(0)); + + ListGroupsOptions options = ListGroupsOptions.forConsumerGroups().inGroupStates(Set.of(GroupState.STABLE)); + ListGroupsResult result = env.adminClient().listGroups(options); + + Collection listing = result.all().get(); + assertEquals(1, listing.size()); + List expected = List.of( + new GroupListing("group-1", Optional.empty(), ConsumerProtocol.PROTOCOL_TYPE, Optional.of(GroupState.STABLE)) + ); + assertEquals(expected, listing); + + // Check that we cannot set a type filter with an older broker. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + // First attempt to build request will require v5 (type filter), but the broker only supports v4 + env.kafkaClient().prepareUnsupportedVersionResponse(request -> + request instanceof ListGroupsRequest && !((ListGroupsRequest) request).data().typesFilter().isEmpty() + ); + + options = ListGroupsOptions.forConsumerGroups().withTypes(Set.of(GroupType.SHARE)); + result = env.adminClient().listGroups(options); + TestUtils.assertFutureThrows(UnsupportedVersionException.class, result.all()); } } @Test @SuppressWarnings("removal") - public void testListConsumerGroups() throws Exception { + public void testListConsumerGroupsDeprecated() throws Exception { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(4, 0), - AdminClientConfig.RETRIES_CONFIG, "2")) { + AdminClientConfig.RETRIES_CONFIG, "2")) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); // Empty metadata response should be retried env.kafkaClient().prepareResponse( - RequestTestUtils.metadataResponse( - Collections.emptyList(), - env.cluster().clusterResource().clusterId(), - -1, - Collections.emptyList())); + RequestTestUtils.metadataResponse( + List.of(), + env.cluster().clusterResource().clusterId(), + -1, + List.of())); env.kafkaClient().prepareResponse( - RequestTestUtils.metadataResponse( - env.cluster().nodes(), - env.cluster().clusterResource().clusterId(), - env.cluster().controller().id(), - Collections.emptyList())); + RequestTestUtils.metadataResponse( + env.cluster().nodes(), + env.cluster().clusterResource().clusterId(), + env.cluster().controller().id(), + List.of())); env.kafkaClient().prepareResponseFrom( - new ListGroupsResponse( - new ListGroupsResponseData() - .setErrorCode(Errors.NONE.code()) - .setGroups(asList( - new ListGroupsResponseData.ListedGroup() - .setGroupId("group-1") - .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) - .setGroupState("Stable"), - new ListGroupsResponseData.ListedGroup() - .setGroupId("group-connect-1") - .setProtocolType("connector") - .setGroupState("Stable") - ))), - env.cluster().nodeById(0)); + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable"), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-connect-1") + .setProtocolType("connector") + .setGroupState("Stable") + ))), + env.cluster().nodeById(0)); // handle retriable errors env.kafkaClient().prepareResponseFrom( - new ListGroupsResponse( - new ListGroupsResponseData() - .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) - .setGroups(Collections.emptyList()) - ), - env.cluster().nodeById(1)); + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) + .setGroups(List.of()) + ), + env.cluster().nodeById(1)); env.kafkaClient().prepareResponseFrom( - new ListGroupsResponse( - new ListGroupsResponseData() - .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) - .setGroups(Collections.emptyList()) - ), - env.cluster().nodeById(1)); + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) + .setGroups(List.of()) + ), + env.cluster().nodeById(1)); env.kafkaClient().prepareResponseFrom( - new ListGroupsResponse( - new ListGroupsResponseData() - .setErrorCode(Errors.NONE.code()) - .setGroups(asList( - new ListGroupsResponseData.ListedGroup() - .setGroupId("group-2") - .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) - .setGroupState("Stable"), - new ListGroupsResponseData.ListedGroup() - .setGroupId("group-connect-2") - .setProtocolType("connector") - .setGroupState("Stable") - ))), - env.cluster().nodeById(1)); + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-2") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable"), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-connect-2") + .setProtocolType("connector") + .setGroupState("Stable") + ))), + env.cluster().nodeById(1)); env.kafkaClient().prepareResponseFrom( - new ListGroupsResponse( - new ListGroupsResponseData() - .setErrorCode(Errors.NONE.code()) - .setGroups(asList( - new ListGroupsResponseData.ListedGroup() - .setGroupId("group-3") - .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) - .setGroupState("Stable"), - new ListGroupsResponseData.ListedGroup() - .setGroupId("group-connect-3") - .setProtocolType("connector") - .setGroupState("Stable") - ))), - env.cluster().nodeById(2)); + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-3") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable"), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-connect-3") + .setProtocolType("connector") + .setGroupState("Stable") + ))), + env.cluster().nodeById(2)); // fatal error env.kafkaClient().prepareResponseFrom( - new ListGroupsResponse( - new ListGroupsResponseData() - .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()) - .setGroups(Collections.emptyList())), - env.cluster().nodeById(3)); + new ListGroupsResponse( + new ListGroupsResponseData() + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()) + .setGroups(List.of())), + env.cluster().nodeById(3)); final ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(); TestUtils.assertFutureThrows(UnknownServerException.class, result.all()); @@ -3379,22 +3798,22 @@ public void testListConsumerGroups() throws Exception { @Test @SuppressWarnings("removal") - public void testListConsumerGroupsMetadataFailure() throws Exception { + public void testListConsumerGroupsDeprecatedMetadataFailure() throws Exception { final Cluster cluster = mockCluster(3, 0); final Time time = new MockTime(); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, - AdminClientConfig.RETRIES_CONFIG, "0")) { + AdminClientConfig.RETRIES_CONFIG, "0")) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); // Empty metadata causes the request to fail since we have no list of brokers // to send the ListGroups requests to env.kafkaClient().prepareResponse( - RequestTestUtils.metadataResponse( - Collections.emptyList(), - env.cluster().clusterResource().clusterId(), - -1, - Collections.emptyList())); + RequestTestUtils.metadataResponse( + List.of(), + env.cluster().clusterResource().clusterId(), + -1, + List.of())); final ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(); TestUtils.assertFutureThrows(KafkaException.class, result.all()); @@ -3403,7 +3822,7 @@ public void testListConsumerGroupsMetadataFailure() throws Exception { @Test @SuppressWarnings("removal") - public void testListConsumerGroupsWithStates() throws Exception { + public void testListConsumerGroupsDeprecatedWithStates() throws Exception { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); @@ -3412,14 +3831,14 @@ public void testListConsumerGroupsWithStates() throws Exception { env.kafkaClient().prepareResponseFrom( new ListGroupsResponse(new ListGroupsResponseData() .setErrorCode(Errors.NONE.code()) - .setGroups(asList( - new ListGroupsResponseData.ListedGroup() - .setGroupId("group-1") - .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) - .setGroupState("Stable"), - new ListGroupsResponseData.ListedGroup() - .setGroupId("group-2") - .setGroupState("Empty")))), + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState("Stable"), + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-2") + .setGroupState("Empty")))), env.cluster().nodeById(0)); final ListConsumerGroupsOptions options = new ListConsumerGroupsOptions(); @@ -3437,7 +3856,7 @@ public void testListConsumerGroupsWithStates() throws Exception { @Test @SuppressWarnings("removal") - public void testListConsumerGroupsWithTypes() throws Exception { + public void testListConsumerGroupsDeprecatedWithTypes() throws Exception { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); @@ -3445,10 +3864,10 @@ public void testListConsumerGroupsWithTypes() throws Exception { env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); env.kafkaClient().prepareResponseFrom( - expectListGroupsRequestWithFilters(singleton(GroupState.STABLE.toString()), Collections.emptySet()), + expectListGroupsRequestWithFilters(Set.of(GroupState.STABLE.toString()), Set.of()), new ListGroupsResponse(new ListGroupsResponseData() .setErrorCode(Errors.NONE.code()) - .setGroups(singletonList( + .setGroups(List.of( new ListGroupsResponseData.ListedGroup() .setGroupId("group-1") .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) @@ -3456,7 +3875,7 @@ public void testListConsumerGroupsWithTypes() throws Exception { .setGroupType(GroupType.CLASSIC.toString())))), env.cluster().nodeById(0)); - final ListConsumerGroupsOptions options = new ListConsumerGroupsOptions().inGroupStates(singleton(GroupState.STABLE)); + final ListConsumerGroupsOptions options = new ListConsumerGroupsOptions().inGroupStates(Set.of(GroupState.STABLE)); final ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(options); Collection listings = result.valid().get(); @@ -3470,10 +3889,10 @@ public void testListConsumerGroupsWithTypes() throws Exception { env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); env.kafkaClient().prepareResponseFrom( - expectListGroupsRequestWithFilters(Collections.emptySet(), singleton(GroupType.CONSUMER.toString())), + expectListGroupsRequestWithFilters(Set.of(), Set.of(GroupType.CONSUMER.toString())), new ListGroupsResponse(new ListGroupsResponseData() .setErrorCode(Errors.NONE.code()) - .setGroups(asList( + .setGroups(List.of( new ListGroupsResponseData.ListedGroup() .setGroupId("group-1") .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) @@ -3500,30 +3919,31 @@ public void testListConsumerGroupsWithTypes() throws Exception { @Test @SuppressWarnings("removal") - public void testListConsumerGroupsWithStatesOlderBrokerVersion() throws Exception { + public void testListConsumerGroupsDeprecatedWithStatesOlderBrokerVersion() throws Exception { ApiVersion listGroupV3 = new ApiVersion() - .setApiKey(ApiKeys.LIST_GROUPS.id) - .setMinVersion((short) 0) - .setMaxVersion((short) 3); + .setApiKey(ApiKeys.LIST_GROUPS.id) + .setMinVersion((short) 0) + .setMaxVersion((short) 3); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { - env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Collections.singletonList(listGroupV3))); + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(List.of(listGroupV3))); env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); // Check we can list groups with older broker if we don't specify states env.kafkaClient().prepareResponseFrom( - new ListGroupsResponse(new ListGroupsResponseData() - .setErrorCode(Errors.NONE.code()) - .setGroups(Collections.singletonList( - new ListGroupsResponseData.ListedGroup() - .setGroupId("group-1") - .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE)))), - env.cluster().nodeById(0)); + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE)))), + env.cluster().nodeById(0)); + ListConsumerGroupsOptions options = new ListConsumerGroupsOptions(); ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(options); Collection listing = result.all().get(); assertEquals(1, listing.size()); - List expected = Collections.singletonList(new ConsumerGroupListing("group-1", false)); + List expected = List.of(new ConsumerGroupListing("group-1", false)); assertEquals(expected, listing); // But we cannot set a state filter with older broker @@ -3531,7 +3951,7 @@ public void testListConsumerGroupsWithStatesOlderBrokerVersion() throws Exceptio env.kafkaClient().prepareUnsupportedVersionResponse( body -> body instanceof ListGroupsRequest); - options = new ListConsumerGroupsOptions().inGroupStates(singleton(GroupState.STABLE)); + options = new ListConsumerGroupsOptions().inGroupStates(Set.of(GroupState.STABLE)); result = env.adminClient().listConsumerGroups(options); TestUtils.assertFutureThrows(UnsupportedVersionException.class, result.all()); } @@ -3539,34 +3959,34 @@ public void testListConsumerGroupsWithStatesOlderBrokerVersion() throws Exceptio @Test @SuppressWarnings("removal") - public void testListConsumerGroupsWithTypesOlderBrokerVersion() throws Exception { + public void testListConsumerGroupsDeprecatedWithTypesOlderBrokerVersion() throws Exception { ApiVersion listGroupV4 = new ApiVersion() .setApiKey(ApiKeys.LIST_GROUPS.id) .setMinVersion((short) 0) .setMaxVersion((short) 4); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) { - env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(Collections.singletonList(listGroupV4))); + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create(List.of(listGroupV4))); env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); // Check if we can list groups with older broker if we specify states and don't specify types. env.kafkaClient().prepareResponseFrom( - expectListGroupsRequestWithFilters(singleton(GroupState.STABLE.toString()), Collections.emptySet()), + expectListGroupsRequestWithFilters(Set.of(GroupState.STABLE.toString()), Set.of()), new ListGroupsResponse(new ListGroupsResponseData() .setErrorCode(Errors.NONE.code()) - .setGroups(Collections.singletonList( + .setGroups(List.of( new ListGroupsResponseData.ListedGroup() .setGroupId("group-1") .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) .setGroupState(GroupState.STABLE.toString())))), env.cluster().nodeById(0)); - ListConsumerGroupsOptions options = new ListConsumerGroupsOptions().inGroupStates(singleton(GroupState.STABLE)); + ListConsumerGroupsOptions options = new ListConsumerGroupsOptions().inGroupStates(Set.of(GroupState.STABLE)); ListConsumerGroupsResult result = env.adminClient().listConsumerGroups(options); Collection listing = result.all().get(); assertEquals(1, listing.size()); - List expected = Collections.singletonList( + List expected = List.of( new ConsumerGroupListing("group-1", Optional.of(GroupState.STABLE), false) ); assertEquals(expected, listing); @@ -3577,9 +3997,31 @@ public void testListConsumerGroupsWithTypesOlderBrokerVersion() throws Exception request instanceof ListGroupsRequest && !((ListGroupsRequest) request).data().typesFilter().isEmpty() ); - options = new ListConsumerGroupsOptions().withTypes(singleton(GroupType.CLASSIC)); + options = new ListConsumerGroupsOptions().withTypes(Set.of(GroupType.SHARE)); result = env.adminClient().listConsumerGroups(options); TestUtils.assertFutureThrows(UnsupportedVersionException.class, result.all()); + + // But a type filter which is just classic groups is permitted with an older broker, because they + // only know about classic groups so the types filter can be omitted. + env.kafkaClient().prepareResponse(prepareMetadataResponse(env.cluster(), Errors.NONE)); + + env.kafkaClient().prepareResponseFrom( + expectListGroupsRequestWithFilters(Set.of(), Set.of()), + new ListGroupsResponse(new ListGroupsResponseData() + .setErrorCode(Errors.NONE.code()) + .setGroups(List.of( + new ListGroupsResponseData.ListedGroup() + .setGroupId("group-1") + .setProtocolType(ConsumerProtocol.PROTOCOL_TYPE) + .setGroupState(GroupState.STABLE.toString())))), + env.cluster().nodeById(0)); + + options = new ListConsumerGroupsOptions().withTypes(Set.of(GroupType.CLASSIC)); + result = env.adminClient().listConsumerGroups(options); + + listing = result.all().get(); + assertEquals(1, listing.size()); + assertEquals(expected, listing); } } @@ -4417,7 +4859,7 @@ public void testListConsumerGroupOffsetsNumRetries() throws Exception { env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR)); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID); @@ -4445,14 +4887,14 @@ public void testListConsumerGroupOffsetsRetryBackoff() throws Exception { mockClient.prepareResponse(body -> { firstAttemptTime.set(time.milliseconds()); return true; - }, offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + }, offsetFetchResponse(Errors.NOT_COORDINATOR)); mockClient.prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); mockClient.prepareResponse(body -> { secondAttemptTime.set(time.milliseconds()); return true; - }, offsetFetchResponse(Errors.NONE, Collections.emptyMap())); + }, offsetFetchResponse(Errors.NONE)); final KafkaFuture> future = env.adminClient().listConsumerGroupOffsets(GROUP_ID).partitionsToOffsetAndMetadata(); @@ -4481,7 +4923,7 @@ public void testListConsumerGroupOffsetsRetriableErrors() throws Exception { prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); env.kafkaClient().prepareResponse( - offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); + offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS)); /* * We need to return two responses here, one for NOT_COORDINATOR call when calling list consumer offsets @@ -4491,19 +4933,19 @@ public void testListConsumerGroupOffsetsRetriableErrors() throws Exception { * And the same reason for the following COORDINATOR_NOT_AVAILABLE error response */ env.kafkaClient().prepareResponse( - offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + offsetFetchResponse(Errors.NOT_COORDINATOR)); env.kafkaClient().prepareResponse( prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); env.kafkaClient().prepareResponse( - offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE)); env.kafkaClient().prepareResponse( prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); env.kafkaClient().prepareResponse( - offsetFetchResponse(Errors.NONE, Collections.emptyMap())); + offsetFetchResponse(Errors.NONE)); final ListConsumerGroupOffsetsResult errorResult1 = env.adminClient().listConsumerGroupOffsets(GROUP_ID); @@ -4525,7 +4967,7 @@ public void testListConsumerGroupOffsetsNonRetriableErrors() throws Exception { env.kafkaClient().prepareResponse( prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - env.kafkaClient().prepareResponse(offsetFetchResponse(error, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(error)); ListConsumerGroupOffsetsResult errorResult = env.adminClient().listConsumerGroupOffsets(GROUP_ID); @@ -4545,7 +4987,7 @@ public void testListConsumerGroupOffsets() throws Exception { env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); // Retriable errors should be retried - env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_LOAD_IN_PROGRESS)); /* * We need to return two responses here, one for NOT_COORDINATOR error when calling list consumer group offsets @@ -4554,10 +4996,10 @@ public void testListConsumerGroupOffsets() throws Exception { * * And the same reason for the following COORDINATOR_NOT_AVAILABLE error response */ - env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR)); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); - env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE)); env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller())); TopicPartition myTopicPartition0 = new TopicPartition("my_topic", 0); @@ -4565,16 +5007,31 @@ public void testListConsumerGroupOffsets() throws Exception { TopicPartition myTopicPartition2 = new TopicPartition("my_topic", 2); TopicPartition myTopicPartition3 = new TopicPartition("my_topic", 3); - final Map responseData = new HashMap<>(); - responseData.put(myTopicPartition0, new OffsetFetchResponse.PartitionData(10, - Optional.empty(), "", Errors.NONE)); - responseData.put(myTopicPartition1, new OffsetFetchResponse.PartitionData(0, - Optional.empty(), "", Errors.NONE)); - responseData.put(myTopicPartition2, new OffsetFetchResponse.PartitionData(20, - Optional.empty(), "", Errors.NONE)); - responseData.put(myTopicPartition3, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, - Optional.empty(), "", Errors.NONE)); - env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.NONE, responseData)); + final OffsetFetchResponseData response = new OffsetFetchResponseData() + .setGroups(List.of( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(GROUP_ID) + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName("my_topic") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(myTopicPartition0.partition()) + .setCommittedOffset(10), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(myTopicPartition1.partition()) + .setCommittedOffset(0), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(myTopicPartition2.partition()) + .setCommittedOffset(20), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(myTopicPartition3.partition()) + .setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET) + )) + )) + )); + + env.kafkaClient().prepareResponse(new OffsetFetchResponse(response, ApiKeys.OFFSET_FETCH.latestVersion())); final ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(GROUP_ID); final Map partitionToOffsetAndMetadata = result.partitionsToOffsetAndMetadata().get(); @@ -4704,7 +5161,7 @@ public void testBatchedListConsumerGroupOffsetsWithNoOffsetFetchBatching() throw env.kafkaClient().prepareResponse(prepareBatchedFindCoordinatorResponse(Errors.NONE, env.cluster().controller(), groupSpecs.keySet())); // Prepare a response to force client to attempt batched request creation that throws // NoBatchedOffsetFetchRequestException. This triggers creation of non-batched requests. - env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE)); ListConsumerGroupOffsetsResult result = env.adminClient().listConsumerGroupOffsets(groupSpecs); @@ -4737,7 +5194,7 @@ public void testBatchedListStreamsGroupOffsetsWithNoOffsetFetchBatching() throws env.kafkaClient().prepareResponse(prepareBatchedFindCoordinatorResponse(Errors.NONE, env.cluster().controller(), groupSpecs.keySet())); // Prepare a response to force client to attempt batched request creation that throws // NoBatchedOffsetFetchRequestException. This triggers creation of non-batched requests. - env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE, Collections.emptyMap())); + env.kafkaClient().prepareResponse(offsetFetchResponse(Errors.COORDINATOR_NOT_AVAILABLE)); ListStreamsGroupOffsetsResult result = env.adminClient().listStreamsGroupOffsets(groupSpecs); @@ -4792,21 +5249,29 @@ private void sendOffsetFetchResponse(MockClient mockClient, Map> results = new HashMap<>(); - Map errors = new HashMap<>(); - data.groups().forEach(group -> { - Map partitionResults = new HashMap<>(); - for (TopicPartition tp : groupSpecs.get(group.groupId()).topicPartitions()) { - partitionResults.put(tp, new PartitionData(10, Optional.empty(), "", Errors.NONE)); - } - results.put(group.groupId(), partitionResults); - errors.put(group.groupId(), error); - }); + if (!batched) { assertEquals(1, data.groups().size()); - mockClient.respond(new OffsetFetchResponse(THROTTLE, error, results.values().iterator().next())); - } else - mockClient.respond(new OffsetFetchResponse(THROTTLE, errors, results)); + } + + OffsetFetchResponseData response = new OffsetFetchResponseData() + .setGroups(data.groups().stream().map(group -> + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(group.groupId()) + .setErrorCode(error.code()) + .setTopics(groupSpecs.get(group.groupId()).topicPartitions().stream() + .collect(Collectors.groupingBy(TopicPartition::topic)).entrySet().stream().map(entry -> + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(entry.getKey()) + .setPartitions(entry.getValue().stream().map(partition -> + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(partition.partition()) + .setCommittedOffset(10) + ).collect(Collectors.toList())) + ).collect(Collectors.toList())) + ).collect(Collectors.toList())); + + mockClient.respond(new OffsetFetchResponse(response, ApiKeys.OFFSET_FETCH.latestVersion())); } private void sendStreamsOffsetFetchResponse(MockClient mockClient, Map groupSpecs, boolean batched, Errors error) throws Exception { @@ -4814,21 +5279,29 @@ private void sendStreamsOffsetFetchResponse(MockClient mockClient, Map> results = new HashMap<>(); - Map errors = new HashMap<>(); - data.groups().forEach(group -> { - Map partitionResults = new HashMap<>(); - for (TopicPartition tp : groupSpecs.get(group.groupId()).topicPartitions()) { - partitionResults.put(tp, new PartitionData(10, Optional.empty(), "", Errors.NONE)); - } - results.put(group.groupId(), partitionResults); - errors.put(group.groupId(), error); - }); + if (!batched) { assertEquals(1, data.groups().size()); - mockClient.respond(new OffsetFetchResponse(THROTTLE, error, results.values().iterator().next())); - } else - mockClient.respond(new OffsetFetchResponse(THROTTLE, errors, results)); + } + + OffsetFetchResponseData response = new OffsetFetchResponseData() + .setGroups(data.groups().stream().map(group -> + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(group.groupId()) + .setErrorCode(error.code()) + .setTopics(groupSpecs.get(group.groupId()).topicPartitions().stream() + .collect(Collectors.groupingBy(TopicPartition::topic)).entrySet().stream().map(entry -> + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(entry.getKey()) + .setPartitions(entry.getValue().stream().map(partition -> + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(partition.partition()) + .setCommittedOffset(10) + ).collect(Collectors.toList())) + ).collect(Collectors.toList())) + ).collect(Collectors.toList())); + + mockClient.respond(new OffsetFetchResponse(response, ApiKeys.OFFSET_FETCH.latestVersion())); } private void verifyListOffsetsForMultipleGroups(Map groupSpecs, @@ -6087,7 +6560,7 @@ public void testListStreamsGroups() throws Exception { .setGroups(Collections.emptyList())), env.cluster().nodeById(3)); - final ListGroupsResult result = env.adminClient().listGroups(new ListGroupsOptions().withTypes(Set.of(GroupType.STREAMS))); + final ListGroupsResult result = env.adminClient().listGroups(ListGroupsOptions.forStreamsGroups()); TestUtils.assertFutureThrows(UnknownServerException.class, result.all()); Collection listings = result.valid().get(); @@ -6122,7 +6595,7 @@ public void testListStreamsGroupsMetadataFailure() throws Exception { -1, Collections.emptyList())); - final ListGroupsResult result = env.adminClient().listGroups(new ListGroupsOptions().withTypes(Set.of(GroupType.STREAMS))); + final ListGroupsResult result = env.adminClient().listGroups(ListGroupsOptions.forStreamsGroups()); TestUtils.assertFutureThrows(KafkaException.class, result.all()); } } @@ -6150,7 +6623,7 @@ public void testListStreamsGroupsWithStates() throws Exception { .setGroupState("NotReady")))), env.cluster().nodeById(0)); - final ListGroupsResult result = env.adminClient().listGroups(new ListGroupsOptions().withTypes(Set.of(GroupType.STREAMS))); + final ListGroupsResult result = env.adminClient().listGroups(ListGroupsOptions.forStreamsGroups()); Collection listings = result.valid().get(); assertEquals(2, listings.size()); @@ -6181,7 +6654,7 @@ public void testListStreamsGroupsWithStatesOlderBrokerVersion() { new ListGroupsResponseData.ListedGroup() .setGroupId("streams-group-1")))), env.cluster().nodeById(0)); - ListGroupsResult result = env.adminClient().listGroups(new ListGroupsOptions().withTypes(Set.of(GroupType.STREAMS))); + ListGroupsResult result = env.adminClient().listGroups(ListGroupsOptions.forStreamsGroups()); TestUtils.assertFutureThrows(UnsupportedVersionException.class, result.all()); } } @@ -6497,7 +6970,7 @@ public void testListShareGroups() throws Exception { .setGroups(Collections.emptyList())), env.cluster().nodeById(3)); - final ListGroupsResult result = env.adminClient().listGroups(new ListGroupsOptions().withTypes(Set.of(GroupType.SHARE))); + final ListGroupsResult result = env.adminClient().listGroups(ListGroupsOptions.forShareGroups()); TestUtils.assertFutureThrows(UnknownServerException.class, result.all()); Collection listings = result.valid().get(); @@ -6532,7 +7005,7 @@ public void testListShareGroupsMetadataFailure() throws Exception { -1, Collections.emptyList())); - final ListGroupsResult result = env.adminClient().listGroups(new ListGroupsOptions().withTypes(Set.of(GroupType.SHARE))); + final ListGroupsResult result = env.adminClient().listGroups(ListGroupsOptions.forShareGroups()); TestUtils.assertFutureThrows(KafkaException.class, result.all()); } } @@ -6560,7 +7033,7 @@ public void testListShareGroupsWithStates() throws Exception { .setGroupState("Empty")))), env.cluster().nodeById(0)); - final ListGroupsResult result = env.adminClient().listGroups(new ListGroupsOptions().withTypes(Set.of(GroupType.SHARE))); + final ListGroupsResult result = env.adminClient().listGroups(ListGroupsOptions.forShareGroups()); Collection listings = result.valid().get(); assertEquals(2, listings.size()); @@ -6591,7 +7064,7 @@ public void testListShareGroupsWithStatesOlderBrokerVersion() { new ListGroupsResponseData.ListedGroup() .setGroupId("share-group-1")))), env.cluster().nodeById(0)); - ListGroupsResult result = env.adminClient().listGroups(new ListGroupsOptions().withTypes(Set.of(GroupType.SHARE))); + ListGroupsResult result = env.adminClient().listGroups(ListGroupsOptions.forShareGroups()); TestUtils.assertFutureThrows(UnsupportedVersionException.class, result.all()); } } @@ -8389,7 +8862,7 @@ public void testDescribeFeaturesFailure() { options.timeoutMs(10000); final KafkaFuture future = env.adminClient().describeFeatures(options).featureMetadata(); final ExecutionException e = assertThrows(ExecutionException.class, future::get); - assertEquals(e.getCause().getClass(), Errors.INVALID_REQUEST.exception().getClass()); + assertEquals(Errors.INVALID_REQUEST.exception().getClass(), e.getCause().getClass()); } } @@ -8978,15 +9451,15 @@ public void testDescribeClientQuotas() throws Exception { DescribeClientQuotasResult result = env.adminClient().describeClientQuotas(filter); Map> resultData = result.entities().get(); - assertEquals(resultData.size(), 2); + assertEquals(2, resultData.size()); assertTrue(resultData.containsKey(entity1)); Map config1 = resultData.get(entity1); - assertEquals(config1.size(), 1); - assertEquals(config1.get("consumer_byte_rate"), 10000.0, 1e-6); + assertEquals(1, config1.size()); + assertEquals(10000.0, config1.get("consumer_byte_rate"), 1e-6); assertTrue(resultData.containsKey(entity2)); Map config2 = resultData.get(entity2); - assertEquals(config2.size(), 1); - assertEquals(config2.get("producer_byte_rate"), 20000.0, 1e-6); + assertEquals(1, config2.size()); + assertEquals(20000.0, config2.get("producer_byte_rate"), 1e-6); } } @@ -10158,10 +10631,16 @@ private DescribeLogDirsResponse prepareDescribeLogDirsResponse(Errors error, Str .setLogDir(logDir)))); } - private OffsetFetchResponse offsetFetchResponse(Errors error, Map responseData) { - return new OffsetFetchResponse(THROTTLE, - Collections.singletonMap(GROUP_ID, error), - Collections.singletonMap(GROUP_ID, responseData)); + private static OffsetFetchResponse offsetFetchResponse(Errors error) { + return new OffsetFetchResponse( + new OffsetFetchResponseData() + .setGroups(List.of( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(GROUP_ID) + .setErrorCode(error.code()) + )), + ApiKeys.OFFSET_FETCH.latestVersion() + ); } private static MemberDescription convertToMemberDescriptions(DescribedGroupMember member, @@ -10577,12 +11056,24 @@ public void testListShareGroupOffsets() throws Exception { List.of( new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup().setGroupId(GROUP_ID).setTopics( List.of( - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(0).setStartOffset(10))), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(1).setStartOffset(11))), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(2).setStartOffset(40))), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(3).setStartOffset(50))), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_1").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(4).setStartOffset(100))), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_2").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(6).setStartOffset(500))) + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions( + List.of( + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(0).setStartOffset(10).setLeaderEpoch(0), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(1).setStartOffset(11).setLeaderEpoch(0), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(2).setStartOffset(40).setLeaderEpoch(0), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(3).setStartOffset(50).setLeaderEpoch(1) + ) + ), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_1").setPartitions( + List.of( + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(4).setStartOffset(100).setLeaderEpoch(2) + ) + ), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_2").setPartitions( + List.of( + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(6).setStartOffset(500).setLeaderEpoch(3) + ) + ) ) ) ) @@ -10590,15 +11081,15 @@ public void testListShareGroupOffsets() throws Exception { env.kafkaClient().prepareResponse(new DescribeShareGroupOffsetsResponse(data)); final ListShareGroupOffsetsResult result = env.adminClient().listShareGroupOffsets(groupSpecs); - final Map partitionToOffsetAndMetadata = result.partitionsToOffset(GROUP_ID).get(); + final Map partitionToOffsetAndMetadata = result.partitionsToOffsetAndMetadata(GROUP_ID).get(); assertEquals(6, partitionToOffsetAndMetadata.size()); - assertEquals(10, partitionToOffsetAndMetadata.get(myTopicPartition0)); - assertEquals(11, partitionToOffsetAndMetadata.get(myTopicPartition1)); - assertEquals(40, partitionToOffsetAndMetadata.get(myTopicPartition2)); - assertEquals(50, partitionToOffsetAndMetadata.get(myTopicPartition3)); - assertEquals(100, partitionToOffsetAndMetadata.get(myTopicPartition4)); - assertEquals(500, partitionToOffsetAndMetadata.get(myTopicPartition5)); + assertEquals(new OffsetAndMetadata(10, Optional.of(0), ""), partitionToOffsetAndMetadata.get(myTopicPartition0)); + assertEquals(new OffsetAndMetadata(11, Optional.of(0), ""), partitionToOffsetAndMetadata.get(myTopicPartition1)); + assertEquals(new OffsetAndMetadata(40, Optional.of(0), ""), partitionToOffsetAndMetadata.get(myTopicPartition2)); + assertEquals(new OffsetAndMetadata(50, Optional.of(1), ""), partitionToOffsetAndMetadata.get(myTopicPartition3)); + assertEquals(new OffsetAndMetadata(100, Optional.of(2), ""), partitionToOffsetAndMetadata.get(myTopicPartition4)); + assertEquals(new OffsetAndMetadata(500, Optional.of(3), ""), partitionToOffsetAndMetadata.get(myTopicPartition5)); } } @@ -10630,16 +11121,28 @@ public void testListShareGroupOffsetsMultipleGroups() throws Exception { List.of( new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup().setGroupId(GROUP_ID).setTopics( List.of( - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(0).setStartOffset(10))), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(1).setStartOffset(11))), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(2).setStartOffset(40))), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(3).setStartOffset(50))) + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions( + List.of( + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(0).setStartOffset(10).setLeaderEpoch(0), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(1).setStartOffset(11).setLeaderEpoch(0), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(2).setStartOffset(40).setLeaderEpoch(0), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(3).setStartOffset(50).setLeaderEpoch(1) + ) + ) ) ), new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup().setGroupId("group-1").setTopics( List.of( - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_1").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(4).setStartOffset(100))), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_2").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(6).setStartOffset(500))) + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_1").setPartitions( + List.of( + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(4).setStartOffset(100).setLeaderEpoch(2) + ) + ), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_2").setPartitions( + List.of( + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(6).setStartOffset(500).setLeaderEpoch(2) + ) + ) ) ) ) @@ -10649,17 +11152,17 @@ public void testListShareGroupOffsetsMultipleGroups() throws Exception { final ListShareGroupOffsetsResult result = env.adminClient().listShareGroupOffsets(groupSpecs); assertEquals(2, result.all().get().size()); - final Map partitionToOffsetAndMetadataGroup0 = result.partitionsToOffset(GROUP_ID).get(); + final Map partitionToOffsetAndMetadataGroup0 = result.partitionsToOffsetAndMetadata(GROUP_ID).get(); assertEquals(4, partitionToOffsetAndMetadataGroup0.size()); - assertEquals(10, partitionToOffsetAndMetadataGroup0.get(myTopicPartition0)); - assertEquals(11, partitionToOffsetAndMetadataGroup0.get(myTopicPartition1)); - assertEquals(40, partitionToOffsetAndMetadataGroup0.get(myTopicPartition2)); - assertEquals(50, partitionToOffsetAndMetadataGroup0.get(myTopicPartition3)); + assertEquals(new OffsetAndMetadata(10, Optional.of(0), ""), partitionToOffsetAndMetadataGroup0.get(myTopicPartition0)); + assertEquals(new OffsetAndMetadata(11, Optional.of(0), ""), partitionToOffsetAndMetadataGroup0.get(myTopicPartition1)); + assertEquals(new OffsetAndMetadata(40, Optional.of(0), ""), partitionToOffsetAndMetadataGroup0.get(myTopicPartition2)); + assertEquals(new OffsetAndMetadata(50, Optional.of(1), ""), partitionToOffsetAndMetadataGroup0.get(myTopicPartition3)); - final Map partitionToOffsetAndMetadataGroup1 = result.partitionsToOffset("group-1").get(); + final Map partitionToOffsetAndMetadataGroup1 = result.partitionsToOffsetAndMetadata("group-1").get(); assertEquals(2, partitionToOffsetAndMetadataGroup1.size()); - assertEquals(100, partitionToOffsetAndMetadataGroup1.get(myTopicPartition4)); - assertEquals(500, partitionToOffsetAndMetadataGroup1.get(myTopicPartition5)); + assertEquals(new OffsetAndMetadata(100, Optional.of(2), ""), partitionToOffsetAndMetadataGroup1.get(myTopicPartition4)); + assertEquals(new OffsetAndMetadata(500, Optional.of(2), ""), partitionToOffsetAndMetadataGroup1.get(myTopicPartition5)); } } @@ -10682,7 +11185,7 @@ public void testListShareGroupOffsetsEmpty() throws Exception { env.kafkaClient().prepareResponse(new DescribeShareGroupOffsetsResponse(data)); final ListShareGroupOffsetsResult result = env.adminClient().listShareGroupOffsets(groupSpecs); - final Map partitionToOffsetAndMetadata = result.partitionsToOffset(GROUP_ID).get(); + final Map partitionToOffsetAndMetadata = result.partitionsToOffsetAndMetadata(GROUP_ID).get(); assertEquals(0, partitionToOffsetAndMetadata.size()); } @@ -10711,12 +11214,22 @@ public void testListShareGroupOffsetsWithErrorInOnePartition() throws Exception List.of( new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup().setGroupId(GROUP_ID).setTopics( List.of( - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions(List.of( - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(0).setStartOffset(10), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(1).setStartOffset(11) - )), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_1").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(4).setErrorCode(Errors.NOT_COORDINATOR.code()).setErrorMessage("Not a Coordinator"))), - new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_2").setPartitions(List.of(new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(6).setStartOffset(500))) + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic").setPartitions( + List.of( + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(0).setStartOffset(10).setLeaderEpoch(0), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(1).setStartOffset(11).setLeaderEpoch(1) + ) + ), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_1").setPartitions( + List.of( + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(4).setErrorCode(Errors.NOT_COORDINATOR.code()).setErrorMessage("Not a Coordinator") + ) + ), + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic().setTopicName("my_topic_2").setPartitions( + List.of( + new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition().setPartitionIndex(6).setStartOffset(500).setLeaderEpoch(2) + ) + ) ) ) ) @@ -10724,13 +11237,13 @@ public void testListShareGroupOffsetsWithErrorInOnePartition() throws Exception env.kafkaClient().prepareResponse(new DescribeShareGroupOffsetsResponse(data)); final ListShareGroupOffsetsResult result = env.adminClient().listShareGroupOffsets(groupSpecs); - final Map partitionToOffsetAndMetadata = result.partitionsToOffset(GROUP_ID).get(); + final Map partitionToOffsetAndMetadata = result.partitionsToOffsetAndMetadata(GROUP_ID).get(); // For myTopicPartition2 we have set an error as the response. Thus, it should be skipped from the final result assertEquals(3, partitionToOffsetAndMetadata.size()); - assertEquals(10, partitionToOffsetAndMetadata.get(myTopicPartition0)); - assertEquals(11, partitionToOffsetAndMetadata.get(myTopicPartition1)); - assertEquals(500, partitionToOffsetAndMetadata.get(myTopicPartition3)); + assertEquals(new OffsetAndMetadata(10, Optional.of(0), ""), partitionToOffsetAndMetadata.get(myTopicPartition0)); + assertEquals(new OffsetAndMetadata(11, Optional.of(1), ""), partitionToOffsetAndMetadata.get(myTopicPartition1)); + assertEquals(new OffsetAndMetadata(500, Optional.of(2), ""), partitionToOffsetAndMetadata.get(myTopicPartition3)); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/ListGroupsOptionsTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/ListGroupsOptionsTest.java new file mode 100644 index 0000000000000..360da83b8da57 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/ListGroupsOptionsTest.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.admin; + +import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; +import org.apache.kafka.common.GroupState; +import org.apache.kafka.common.GroupType; + +import org.junit.jupiter.api.Test; + +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class ListGroupsOptionsTest { + @Test + public void testForConsumerGroups() { + ListGroupsOptions options = ListGroupsOptions.forConsumerGroups(); + assertTrue(options.groupStates().isEmpty()); + assertEquals(Set.of(GroupType.CONSUMER, GroupType.CLASSIC), options.types()); + assertEquals(Set.of("", ConsumerProtocol.PROTOCOL_TYPE), options.protocolTypes()); + + options.inGroupStates(Set.of(GroupState.STABLE)); + options.withTypes(Set.of(GroupType.CONSUMER)); + options.withProtocolTypes(Set.of(ConsumerProtocol.PROTOCOL_TYPE)); + assertEquals(Set.of(GroupState.STABLE), options.groupStates()); + assertEquals(Set.of(GroupType.CONSUMER), options.types()); + assertEquals(Set.of(ConsumerProtocol.PROTOCOL_TYPE), options.protocolTypes()); + } + + @Test + public void testForShareGroups() { + ListGroupsOptions options = ListGroupsOptions.forShareGroups(); + assertTrue(options.groupStates().isEmpty()); + assertEquals(Set.of(GroupType.SHARE), options.types()); + assertTrue(options.protocolTypes().isEmpty()); + + options.inGroupStates(Set.of(GroupState.STABLE)); + options.withTypes(Set.of(GroupType.CONSUMER)); + options.withProtocolTypes(Set.of(ConsumerProtocol.PROTOCOL_TYPE)); + assertEquals(Set.of(GroupState.STABLE), options.groupStates()); + assertEquals(Set.of(GroupType.CONSUMER), options.types()); + assertEquals(Set.of(ConsumerProtocol.PROTOCOL_TYPE), options.protocolTypes()); + } + + @Test + public void testForStreamsGroups() { + ListGroupsOptions options = ListGroupsOptions.forStreamsGroups(); + assertTrue(options.groupStates().isEmpty()); + assertEquals(Set.of(GroupType.STREAMS), options.types()); + assertTrue(options.protocolTypes().isEmpty()); + + options.inGroupStates(Set.of(GroupState.STABLE)); + options.withTypes(Set.of(GroupType.CONSUMER)); + options.withProtocolTypes(Set.of(ConsumerProtocol.PROTOCOL_TYPE)); + assertEquals(Set.of(GroupState.STABLE), options.groupStates()); + assertEquals(Set.of(GroupType.CONSUMER), options.types()); + assertEquals(Set.of(ConsumerProtocol.PROTOCOL_TYPE), options.protocolTypes()); + } + + @Test + public void testGroupStates() { + ListGroupsOptions options = new ListGroupsOptions(); + assertTrue(options.groupStates().isEmpty()); + + options.inGroupStates(Set.of(GroupState.DEAD)); + assertEquals(Set.of(GroupState.DEAD), options.groupStates()); + + Set groupStates = Set.of(GroupState.values()); + options = new ListGroupsOptions().inGroupStates(groupStates); + assertEquals(groupStates, options.groupStates()); + } + + @Test + public void testProtocolTypes() { + ListGroupsOptions options = new ListGroupsOptions(); + assertTrue(options.protocolTypes().isEmpty()); + + options.withProtocolTypes(Set.of(ConsumerProtocol.PROTOCOL_TYPE)); + assertEquals(Set.of(ConsumerProtocol.PROTOCOL_TYPE), options.protocolTypes()); + + Set protocolTypes = Set.of("", "consumer", "share"); + options = new ListGroupsOptions().withProtocolTypes(protocolTypes); + assertEquals(protocolTypes, options.protocolTypes()); + } + + @Test + public void testTypes() { + ListGroupsOptions options = new ListGroupsOptions(); + assertTrue(options.types().isEmpty()); + + options.withTypes(Set.of(GroupType.CLASSIC)); + assertEquals(Set.of(GroupType.CLASSIC), options.types()); + + Set groupTypes = Set.of(GroupType.values()); + options = new ListGroupsOptions().withTypes(groupTypes); + assertEquals(groupTypes, options.types()); + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index d6a7543fdc683..02a9e628b7e1e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -68,6 +68,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -460,14 +461,14 @@ public synchronized ListTopicsResult listTopics(ListTopicsOptions options) { @Override public synchronized DescribeTopicsResult describeTopics(TopicCollection topics, DescribeTopicsOptions options) { if (topics instanceof TopicIdCollection) - return DescribeTopicsResult.ofTopicIds(new HashMap<>(handleDescribeTopicsUsingIds(((TopicIdCollection) topics).topicIds(), options))); + return DescribeTopicsResult.ofTopicIds(new HashMap<>(handleDescribeTopicsUsingIds(((TopicIdCollection) topics).topicIds()))); else if (topics instanceof TopicNameCollection) - return DescribeTopicsResult.ofTopicNames(new HashMap<>(handleDescribeTopicsByNames(((TopicNameCollection) topics).topicNames(), options))); + return DescribeTopicsResult.ofTopicNames(new HashMap<>(handleDescribeTopicsByNames(((TopicNameCollection) topics).topicNames()))); else throw new IllegalArgumentException("The TopicCollection provided did not match any supported classes for describeTopics."); } - private Map> handleDescribeTopicsByNames(Collection topicNames, DescribeTopicsOptions options) { + private Map> handleDescribeTopicsByNames(Collection topicNames) { Map> topicDescriptions = new HashMap<>(); if (timeoutNextRequests > 0) { @@ -507,7 +508,7 @@ private Map> handleDescribeTopicsByNames(C return topicDescriptions; } - public synchronized Map> handleDescribeTopicsUsingIds(Collection topicIds, DescribeTopicsOptions options) { + public synchronized Map> handleDescribeTopicsUsingIds(Collection topicIds) { Map> topicDescriptions = new HashMap<>(); @@ -553,15 +554,15 @@ public synchronized Map> handleDescribeTopi public synchronized DeleteTopicsResult deleteTopics(TopicCollection topics, DeleteTopicsOptions options) { DeleteTopicsResult result; if (topics instanceof TopicIdCollection) - result = DeleteTopicsResult.ofTopicIds(new HashMap<>(handleDeleteTopicsUsingIds(((TopicIdCollection) topics).topicIds(), options))); + result = DeleteTopicsResult.ofTopicIds(new HashMap<>(handleDeleteTopicsUsingIds(((TopicIdCollection) topics).topicIds()))); else if (topics instanceof TopicNameCollection) - result = DeleteTopicsResult.ofTopicNames(new HashMap<>(handleDeleteTopicsUsingNames(((TopicNameCollection) topics).topicNames(), options))); + result = DeleteTopicsResult.ofTopicNames(new HashMap<>(handleDeleteTopicsUsingNames(((TopicNameCollection) topics).topicNames()))); else throw new IllegalArgumentException("The TopicCollection provided did not match any supported classes for deleteTopics."); return result; } - private Map> handleDeleteTopicsUsingNames(Collection topicNameCollection, DeleteTopicsOptions options) { + private Map> handleDeleteTopicsUsingNames(Collection topicNameCollection) { Map> deleteTopicsResult = new HashMap<>(); Collection topicNames = new ArrayList<>(topicNameCollection); @@ -590,7 +591,7 @@ private Map> handleDeleteTopicsUsingNames(Collection> handleDeleteTopicsUsingIds(Collection topicIdCollection, DeleteTopicsOptions options) { + private Map> handleDeleteTopicsUsingIds(Collection topicIdCollection) { Map> deleteTopicsResult = new HashMap<>(); Collection topicIds = new ArrayList<>(topicIdCollection); @@ -1118,11 +1119,7 @@ public synchronized DescribeReplicaLogDirsResult describeReplicaLogDirs( DescribeLogDirsResponse.INVALID_OFFSET_LAG)); } else { ReplicaLogDirInfo info = replicaMoves.get(replica); - if (info == null) { - future.complete(new ReplicaLogDirInfo(currentLogDir, 0, null, 0)); - } else { - future.complete(info); - } + future.complete(Objects.requireNonNullElseGet(info, () -> new ReplicaLogDirInfo(currentLogDir, 0, null, 0))); } } } @@ -1172,8 +1169,7 @@ public synchronized ListPartitionReassignmentsResult listPartitionReassignments( Optional> partitions, ListPartitionReassignmentsOptions options) { Map map = new HashMap<>(); - for (TopicPartition partition : partitions.isPresent() ? - partitions.get() : reassignments.keySet()) { + for (TopicPartition partition : partitions.orElseGet(reassignments::keySet)) { PartitionReassignment reassignment = findPartitionReassignment(partition); if (reassignment != null) { map.put(partition, reassignment); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/RemoveMembersFromConsumerGroupResultTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/RemoveMembersFromConsumerGroupResultTest.java index dfc12c578c8b1..ceb188a41759b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/RemoveMembersFromConsumerGroupResultTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/RemoveMembersFromConsumerGroupResultTest.java @@ -58,7 +58,7 @@ public void setUp() { } @Test - public void testTopLevelErrorConstructor() throws InterruptedException { + public void testTopLevelErrorConstructor() { memberFutures.completeExceptionally(Errors.GROUP_AUTHORIZATION_FAILED.exception()); RemoveMembersFromConsumerGroupResult topLevelErrorResult = new RemoveMembersFromConsumerGroupResult(memberFutures, membersToRemove); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java index e3bb56347a8ae..79e5974d4255b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListConsumerGroupOffsetsHandlerTest.java @@ -24,12 +24,14 @@ import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.GroupIdNotFoundException; import org.apache.kafka.common.errors.InvalidGroupIdException; +import org.apache.kafka.common.message.OffsetFetchRequestData; import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestGroup; -import org.apache.kafka.common.message.OffsetFetchRequestData.OffsetFetchRequestTopics; +import org.apache.kafka.common.message.OffsetFetchResponseData; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.requests.OffsetFetchRequest; import org.apache.kafka.common.requests.OffsetFetchResponse; -import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; import org.apache.kafka.common.utils.LogContext; import org.junit.jupiter.api.Test; @@ -41,7 +43,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -58,11 +59,11 @@ public class ListConsumerGroupOffsetsHandlerTest { private final LogContext logContext = new LogContext(); - private final int throttleMs = 10; - private final String groupZero = "group0"; - private final String groupOne = "group1"; - private final String groupTwo = "group2"; - private final List groups = Arrays.asList(groupZero, groupOne, groupTwo); + private final String group0 = "group0"; + private final String group1 = "group1"; + private final String group2 = "group2"; + private final String group3 = "group3"; + private final List groups = List.of(group0, group1, group2); private final TopicPartition t0p0 = new TopicPartition("t0", 0); private final TopicPartition t0p1 = new TopicPartition("t0", 1); private final TopicPartition t1p0 = new TopicPartition("t1", 0); @@ -70,84 +71,129 @@ public class ListConsumerGroupOffsetsHandlerTest { private final TopicPartition t2p0 = new TopicPartition("t2", 0); private final TopicPartition t2p1 = new TopicPartition("t2", 1); private final TopicPartition t2p2 = new TopicPartition("t2", 2); - private final Map singleRequestMap = Collections.singletonMap(groupZero, - new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t0p1, t1p0, t1p1))); - private final Map batchedRequestMap = - new HashMap() {{ - put(groupZero, new ListConsumerGroupOffsetsSpec().topicPartitions(singletonList(t0p0))); - put(groupOne, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1))); - put(groupTwo, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1, t2p0, t2p1, t2p2))); - }}; + private final TopicPartition t3p0 = new TopicPartition("t3", 0); + private final TopicPartition t3p1 = new TopicPartition("t3", 1); + + private final Map singleGroupSpec = Map.of( + group0, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t0p1, t1p0, t1p1)) + ); + private final Map multiGroupSpecs = Map.of( + group0, new ListConsumerGroupOffsetsSpec().topicPartitions(singletonList(t0p0)), + group1, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1)), + group2, new ListConsumerGroupOffsetsSpec().topicPartitions(Arrays.asList(t0p0, t1p0, t1p1, t2p0, t2p1, t2p2)) + ); @Test public void testBuildRequest() { - ListConsumerGroupOffsetsHandler handler = - new ListConsumerGroupOffsetsHandler(singleRequestMap, false, logContext); - OffsetFetchRequest request = handler.buildBatchedRequest(coordinatorKeys(groupZero)).build(); - assertEquals(groupZero, request.data().groups().get(0).groupId()); - assertEquals(2, request.data().groups().get(0).topics().size()); - assertEquals(2, request.data().groups().get(0).topics().get(0).partitionIndexes().size()); - assertEquals(2, request.data().groups().get(0).topics().get(1).partitionIndexes().size()); + var handler = new ListConsumerGroupOffsetsHandler( + singleGroupSpec, + false, + logContext + ); + + assertEquals( + new OffsetFetchRequestData() + .setGroups(List.of( + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId(group0) + .setTopics(List.of( + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName(t0p0.topic()) + .setPartitionIndexes(List.of(t0p0.partition(), t0p1.partition())), + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName(t1p0.topic()) + .setPartitionIndexes(List.of(t1p0.partition(), t1p1.partition())) + )) + )), + handler.buildBatchedRequest(coordinatorKeys(group0)).build().data() + ); } @Test public void testBuildRequestWithMultipleGroups() { - Map requestMap = new HashMap<>(this.batchedRequestMap); - String groupThree = "group3"; - requestMap.put(groupThree, new ListConsumerGroupOffsetsSpec() - .topicPartitions(Arrays.asList(new TopicPartition("t3", 0), new TopicPartition("t3", 1)))); - - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(requestMap, false, logContext); - OffsetFetchRequest request1 = handler.buildBatchedRequest(coordinatorKeys(groupZero, groupOne, groupTwo)).build(); - assertEquals(Set.of(groupZero, groupOne, groupTwo), requestGroups(request1)); - - OffsetFetchRequest request2 = handler.buildBatchedRequest(coordinatorKeys(groupThree)).build(); - assertEquals(Set.of(groupThree), requestGroups(request2)); - - Map builtRequests = new HashMap<>(); - request1.groupIdsToPartitions().forEach((group, partitions) -> - builtRequests.put(group, new ListConsumerGroupOffsetsSpec().topicPartitions(partitions))); - request2.groupIdsToPartitions().forEach((group, partitions) -> - builtRequests.put(group, new ListConsumerGroupOffsetsSpec().topicPartitions(partitions))); - - assertEquals(requestMap, builtRequests); - Map> groupIdsToTopics = request1.groupIdsToTopics(); - - assertEquals(3, groupIdsToTopics.size()); - assertEquals(1, groupIdsToTopics.get(groupZero).size()); - assertEquals(2, groupIdsToTopics.get(groupOne).size()); - assertEquals(3, groupIdsToTopics.get(groupTwo).size()); - - assertEquals(1, groupIdsToTopics.get(groupZero).get(0).partitionIndexes().size()); - assertEquals(1, groupIdsToTopics.get(groupOne).get(0).partitionIndexes().size()); - assertEquals(2, groupIdsToTopics.get(groupOne).get(1).partitionIndexes().size()); - assertEquals(1, groupIdsToTopics.get(groupTwo).get(0).partitionIndexes().size()); - assertEquals(2, groupIdsToTopics.get(groupTwo).get(1).partitionIndexes().size()); - assertEquals(3, groupIdsToTopics.get(groupTwo).get(2).partitionIndexes().size()); - - groupIdsToTopics = request2.groupIdsToTopics(); - assertEquals(1, groupIdsToTopics.size()); - assertEquals(1, groupIdsToTopics.get(groupThree).size()); - assertEquals(2, groupIdsToTopics.get(groupThree).get(0).partitionIndexes().size()); + var groupSpecs = new HashMap<>(multiGroupSpecs); + groupSpecs.put( + group3, + new ListConsumerGroupOffsetsSpec().topicPartitions(List.of(t3p0, t3p1)) + ); + + var handler = new ListConsumerGroupOffsetsHandler( + groupSpecs, + false, + logContext + ); + + var request1 = handler.buildBatchedRequest(coordinatorKeys(group0, group1, group2)).build(); + + assertEquals( + Set.of( + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId(group0) + .setTopics(List.of( + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName(t0p0.topic()) + .setPartitionIndexes(List.of(t0p0.partition())) + )), + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId(group1) + .setTopics(List.of( + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName(t0p0.topic()) + .setPartitionIndexes(List.of(t0p0.partition())), + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName(t1p0.topic()) + .setPartitionIndexes(List.of(t1p0.partition(), t1p1.partition())) + )), + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId(group2) + .setTopics(List.of( + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName(t0p0.topic()) + .setPartitionIndexes(List.of(t0p0.partition())), + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName(t1p0.topic()) + .setPartitionIndexes(List.of(t1p0.partition(), t1p1.partition())), + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName(t2p0.topic()) + .setPartitionIndexes(List.of(t2p0.partition(), t2p1.partition(), t2p2.partition())) + )) + ), + new HashSet(request1.data().groups()) + ); + + var request2 = handler.buildBatchedRequest(coordinatorKeys(group3)).build(); + + assertEquals( + Set.of( + new OffsetFetchRequestData.OffsetFetchRequestGroup() + .setGroupId(group3) + .setTopics(List.of( + new OffsetFetchRequestData.OffsetFetchRequestTopics() + .setName(t3p0.topic()) + .setPartitionIndexes(List.of(t3p0.partition(), t3p1.partition())) + )) + ), + new HashSet(request2.data().groups()) + ); } @Test public void testBuildRequestBatchGroups() { - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(batchedRequestMap, false, logContext); - Collection> requests = handler.buildRequest(1, coordinatorKeys(groupZero, groupOne, groupTwo)); + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(multiGroupSpecs, false, logContext); + Collection> requests = handler.buildRequest(1, coordinatorKeys(group0, group1, group2)); assertEquals(1, requests.size()); - assertEquals(Set.of(groupZero, groupOne, groupTwo), requestGroups((OffsetFetchRequest) requests.iterator().next().request.build())); + assertEquals(Set.of(group0, group1, group2), requestGroups((OffsetFetchRequest) requests.iterator().next().request.build())); } @Test public void testBuildRequestDoesNotBatchGroup() { - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(batchedRequestMap, false, logContext); + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(multiGroupSpecs, false, logContext); // Disable batching. ((CoordinatorStrategy) handler.lookupStrategy()).disableBatch(); - Collection> requests = handler.buildRequest(1, coordinatorKeys(groupZero, groupOne, groupTwo)); + Collection> requests = handler.buildRequest(1, coordinatorKeys(group0, group1, group2)); assertEquals(3, requests.size()); assertEquals( - Set.of(Set.of(groupZero), Set.of(groupOne), Set.of(groupTwo)), + Set.of(Set.of(group0), Set.of(group1), Set.of(group2)), requests.stream().map(requestAndKey -> requestGroups((OffsetFetchRequest) requestAndKey.request.build())).collect(Collectors.toSet()) ); } @@ -170,32 +216,31 @@ public void testSuccessfulHandleResponseWithOnePartitionError() { @Test public void testSuccessfulHandleResponseWithOnePartitionErrorWithMultipleGroups() { - Map offsetAndMetadataMapZero = - Collections.singletonMap(t0p0, new OffsetAndMetadata(10L)); - Map offsetAndMetadataMapOne = - Collections.singletonMap(t1p1, new OffsetAndMetadata(10L)); - Map offsetAndMetadataMapTwo = - Collections.singletonMap(t2p2, new OffsetAndMetadata(10L)); - Map> expectedResult = - new HashMap<>() {{ - put(groupZero, offsetAndMetadataMapZero); - put(groupOne, offsetAndMetadataMapOne); - put(groupTwo, offsetAndMetadataMapTwo); - }}; + var expectedResult = Map.of( + group0, Map.of(t0p0, new OffsetAndMetadata(10L)), + group1, Map.of(t1p1, new OffsetAndMetadata(10L)), + group2, Map.of(t2p2, new OffsetAndMetadata(10L)) + ); assertCompletedForMultipleGroups( - handleWithPartitionErrorMultipleGroups(Errors.UNKNOWN_TOPIC_OR_PARTITION), expectedResult); + handleWithPartitionErrorMultipleGroups(Errors.UNKNOWN_TOPIC_OR_PARTITION), + expectedResult + ); assertCompletedForMultipleGroups( - handleWithPartitionErrorMultipleGroups(Errors.TOPIC_AUTHORIZATION_FAILED), expectedResult); + handleWithPartitionErrorMultipleGroups(Errors.TOPIC_AUTHORIZATION_FAILED), + expectedResult + ); assertCompletedForMultipleGroups( - handleWithPartitionErrorMultipleGroups(Errors.UNSTABLE_OFFSET_COMMIT), expectedResult); + handleWithPartitionErrorMultipleGroups(Errors.UNSTABLE_OFFSET_COMMIT), + expectedResult + ); } @Test public void testSuccessfulHandleResponseWithMultipleGroups() { Map> expected = new HashMap<>(); Map errorMap = errorMap(groups, Errors.NONE); - assertCompletedForMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap), expected); + assertCompletedForMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, multiGroupSpecs), expected); } @Test @@ -206,11 +251,12 @@ public void testUnmappedHandleResponse() { @Test public void testUnmappedHandleResponseWithMultipleGroups() { - Map errorMap = new HashMap<>(); - errorMap.put(groupZero, Errors.NOT_COORDINATOR); - errorMap.put(groupOne, Errors.COORDINATOR_NOT_AVAILABLE); - errorMap.put(groupTwo, Errors.NOT_COORDINATOR); - assertUnmappedWithMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap)); + var errorMap = Map.of( + group0, Errors.NOT_COORDINATOR, + group1, Errors.COORDINATOR_NOT_AVAILABLE, + group2, Errors.NOT_COORDINATOR + ); + assertUnmappedWithMultipleGroups(handleWithErrorWithMultipleGroups(errorMap, multiGroupSpecs)); } @Test @@ -221,7 +267,7 @@ public void testRetriableHandleResponse() { @Test public void testRetriableHandleResponseWithMultipleGroups() { Map errorMap = errorMap(groups, Errors.COORDINATOR_LOAD_IN_PROGRESS); - assertRetriable(handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap)); + assertRetriable(handleWithErrorWithMultipleGroups(errorMap, multiGroupSpecs)); } @Test @@ -233,117 +279,226 @@ public void testFailedHandleResponse() { @Test public void testFailedHandleResponseWithMultipleGroups() { - Map errorMap = new HashMap<>(); - errorMap.put(groupZero, Errors.GROUP_AUTHORIZATION_FAILED); - errorMap.put(groupOne, Errors.GROUP_ID_NOT_FOUND); - errorMap.put(groupTwo, Errors.INVALID_GROUP_ID); - Map> groupToExceptionMap = new HashMap<>(); - groupToExceptionMap.put(groupZero, GroupAuthorizationException.class); - groupToExceptionMap.put(groupOne, GroupIdNotFoundException.class); - groupToExceptionMap.put(groupTwo, InvalidGroupIdException.class); - assertFailedForMultipleGroups(groupToExceptionMap, - handleWithErrorWithMultipleGroups(errorMap, batchedRequestMap)); + var errorMap = Map.of( + group0, Errors.GROUP_AUTHORIZATION_FAILED, + group1, Errors.GROUP_ID_NOT_FOUND, + group2, Errors.INVALID_GROUP_ID + ); + var groupToExceptionMap = Map.of( + group0, (Class) GroupAuthorizationException.class, + group1, (Class) GroupIdNotFoundException.class, + group2, (Class) InvalidGroupIdException.class + ); + assertFailedForMultipleGroups( + groupToExceptionMap, + handleWithErrorWithMultipleGroups(errorMap, multiGroupSpecs) + ); } private OffsetFetchResponse buildResponse(Errors error) { return new OffsetFetchResponse( - throttleMs, - Collections.singletonMap(groupZero, error), - Collections.singletonMap(groupZero, new HashMap<>())); - } - - private OffsetFetchResponse buildResponseWithMultipleGroups( - Map errorMap, - Map> responseData - ) { - return new OffsetFetchResponse(throttleMs, errorMap, responseData); + new OffsetFetchResponseData() + .setGroups(List.of( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(group0) + .setErrorCode(error.code()) + )), + ApiKeys.OFFSET_FETCH.latestVersion() + ); } private AdminApiHandler.ApiResult> handleWithErrorWithMultipleGroups( Map errorMap, Map groupSpecs ) { - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(groupSpecs, false, logContext); - Map> responseData = new HashMap<>(); - for (String group : errorMap.keySet()) { - responseData.put(group, new HashMap<>()); - } - OffsetFetchResponse response = buildResponseWithMultipleGroups(errorMap, responseData); + var handler = new ListConsumerGroupOffsetsHandler( + groupSpecs, + false, + logContext + ); + var response = new OffsetFetchResponse( + new OffsetFetchResponseData() + .setGroups(errorMap.entrySet().stream().map(entry -> + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(entry.getKey()) + .setErrorCode(entry.getValue().code()) + ).collect(Collectors.toList())), + ApiKeys.OFFSET_FETCH.latestVersion() + ); return handler.handleResponse(new Node(1, "host", 1234), - errorMap.keySet() - .stream() - .map(CoordinatorKey::byGroupId) - .collect(Collectors.toSet()), - response); + errorMap.keySet() + .stream() + .map(CoordinatorKey::byGroupId) + .collect(Collectors.toSet()), + response + ); } private OffsetFetchResponse buildResponseWithPartitionError(Errors error) { - - Map responseData = new HashMap<>(); - responseData.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); - responseData.put(t0p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); - - return new OffsetFetchResponse(Errors.NONE, responseData); + return new OffsetFetchResponse( + new OffsetFetchResponseData() + .setGroups(List.of( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(group0) + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(t0p0.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t0p0.partition()) + .setCommittedOffset(10), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t0p1.partition()) + .setCommittedOffset(10) + .setErrorCode(error.code()) + )) + )) + )), + ApiKeys.OFFSET_FETCH.latestVersion() + ); } private OffsetFetchResponse buildResponseWithPartitionErrorWithMultipleGroups(Errors error) { - Map responseDataZero = new HashMap<>(); - responseDataZero.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); - - Map responseDataOne = new HashMap<>(); - responseDataOne.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); - responseDataOne.put(t1p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); - responseDataOne.put(t1p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); - - Map responseDataTwo = new HashMap<>(); - responseDataTwo.put(t0p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); - responseDataTwo.put(t1p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); - responseDataTwo.put(t1p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); - responseDataTwo.put(t2p0, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); - responseDataTwo.put(t2p1, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", error)); - responseDataTwo.put(t2p2, new OffsetFetchResponse.PartitionData(10, Optional.empty(), "", Errors.NONE)); - - Map> responseData = - new HashMap<>() {{ - put(groupZero, responseDataZero); - put(groupOne, responseDataOne); - put(groupTwo, responseDataTwo); - }}; - - Map errorMap = errorMap(groups, Errors.NONE); - return new OffsetFetchResponse(0, errorMap, responseData); + var data = new OffsetFetchResponseData() + .setGroups(List.of( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(group0) + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(t0p0.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t0p0.partition()) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + .setErrorCode(Errors.NONE.code()) + )) + )), + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(group1) + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(t0p0.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t0p0.partition()) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + .setErrorCode(error.code()) + )), + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(t1p0.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t1p0.partition()) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + .setErrorCode(error.code()), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t1p1.partition()) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + .setErrorCode(Errors.NONE.code()) + )) + )), + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(group2) + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(t0p0.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t0p0.partition()) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + .setErrorCode(error.code()) + )), + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(t1p0.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t1p0.partition()) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + .setErrorCode(error.code()), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t1p1.partition()) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + .setErrorCode(error.code()) + )), + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(t2p0.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t2p0.partition()) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + .setErrorCode(error.code()), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t2p1.partition()) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + .setErrorCode(error.code()), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t2p2.partition()) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + .setErrorCode(Errors.NONE.code()) + )) + )) + )); + + return new OffsetFetchResponse(data, ApiKeys.OFFSET_FETCH.latestVersion()); } private AdminApiHandler.ApiResult> handleWithPartitionError( Errors error ) { - ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(singleRequestMap, - false, logContext); + ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler( + singleGroupSpec, + false, + logContext + ); OffsetFetchResponse response = buildResponseWithPartitionError(error); return handler.handleResponse(new Node(1, "host", 1234), - singleton(CoordinatorKey.byGroupId(groupZero)), response); + singleton(CoordinatorKey.byGroupId(group0)), response); } private AdminApiHandler.ApiResult> handleWithPartitionErrorMultipleGroups( Errors error ) { ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler( - batchedRequestMap, false, logContext); + multiGroupSpecs, + false, + logContext + ); OffsetFetchResponse response = buildResponseWithPartitionErrorWithMultipleGroups(error); return handler.handleResponse( new Node(1, "host", 1234), - coordinatorKeys(groupZero, groupOne, groupTwo), - response); + coordinatorKeys(group0, group1, group2), + response + ); } private AdminApiHandler.ApiResult> handleWithError( Errors error ) { ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler( - singleRequestMap, false, logContext); + singleGroupSpec, false, logContext); OffsetFetchResponse response = buildResponse(error); return handler.handleResponse(new Node(1, "host", 1234), - singleton(CoordinatorKey.byGroupId(groupZero)), + singleton(CoordinatorKey.byGroupId(group0)), response); } @@ -352,7 +507,7 @@ private void assertUnmapped( ) { assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptySet(), result.failedKeys.keySet()); - assertEquals(singletonList(CoordinatorKey.byGroupId(groupZero)), result.unmappedKeys); + assertEquals(singletonList(CoordinatorKey.byGroupId(group0)), result.unmappedKeys); } private void assertUnmappedWithMultipleGroups( @@ -360,7 +515,7 @@ private void assertUnmappedWithMultipleGroups( ) { assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptySet(), result.failedKeys.keySet()); - assertEquals(coordinatorKeys(groupZero, groupOne, groupTwo), new HashSet<>(result.unmappedKeys)); + assertEquals(coordinatorKeys(group0, group1, group2), new HashSet<>(result.unmappedKeys)); } private void assertRetriable( @@ -375,7 +530,7 @@ private void assertCompleted( AdminApiHandler.ApiResult> result, Map expected ) { - CoordinatorKey key = CoordinatorKey.byGroupId(groupZero); + CoordinatorKey key = CoordinatorKey.byGroupId(group0); assertEquals(emptySet(), result.failedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(key), result.completedKeys.keySet()); @@ -399,7 +554,7 @@ private void assertFailed( Class expectedExceptionType, AdminApiHandler.ApiResult> result ) { - CoordinatorKey key = CoordinatorKey.byGroupId(groupZero); + CoordinatorKey key = CoordinatorKey.byGroupId(group0); assertEquals(emptySet(), result.completedKeys.keySet()); assertEquals(emptyList(), result.unmappedKeys); assertEquals(singleton(key), result.failedKeys.keySet()); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandlerTest.java index 52073551e3d79..2c2529a0664c1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/ListTransactionsHandlerTest.java @@ -36,6 +36,7 @@ import java.util.List; import java.util.Map; import java.util.OptionalInt; +import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -43,6 +44,7 @@ import static java.util.Collections.singleton; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; public class ListTransactionsHandlerTest { @@ -86,6 +88,42 @@ public void testBuildRequestWithFilteredState() { assertEquals(Collections.emptyList(), request.data().producerIdFilters()); } + + @Test + public void testBuildRequestWithFilteredTransactionalIdPattern() { + int brokerId = 1; + BrokerKey brokerKey = new BrokerKey(OptionalInt.of(brokerId)); + String filteredTransactionalIdPattern = "^special-.*"; + ListTransactionsOptions options = new ListTransactionsOptions() + .filterOnTransactionalIdPattern(filteredTransactionalIdPattern); + ListTransactionsHandler handler = new ListTransactionsHandler(options, logContext); + ListTransactionsRequest request = handler.buildBatchedRequest(brokerId, Set.of(brokerKey)).build(); + assertEquals(filteredTransactionalIdPattern, request.data().transactionalIdPattern()); + assertEquals(List.of(), request.data().stateFilters()); + } + + @Test + public void testBuildRequestWithNullFilteredTransactionalIdPattern() { + int brokerId = 1; + BrokerKey brokerKey = new BrokerKey(OptionalInt.of(brokerId)); + ListTransactionsOptions options = new ListTransactionsOptions() + .filterOnTransactionalIdPattern(null); + ListTransactionsHandler handler = new ListTransactionsHandler(options, logContext); + ListTransactionsRequest request = handler.buildBatchedRequest(brokerId, Set.of(brokerKey)).build(); + assertNull(request.data().transactionalIdPattern()); + } + + @Test + public void testBuildRequestWithEmptyFilteredTransactionalIdPattern() { + int brokerId = 1; + BrokerKey brokerKey = new BrokerKey(OptionalInt.of(brokerId)); + ListTransactionsOptions options = new ListTransactionsOptions() + .filterOnTransactionalIdPattern(""); + ListTransactionsHandler handler = new ListTransactionsHandler(options, logContext); + ListTransactionsRequest request = handler.buildBatchedRequest(brokerId, Set.of(brokerKey)).build(); + assertNull(request.data().transactionalIdPattern()); + } + @Test public void testBuildRequestWithDurationFilter() { int brokerId = 1; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/CloseOptionsTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/CloseOptionsTest.java new file mode 100644 index 0000000000000..c33f73dc7a398 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/CloseOptionsTest.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer; + +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.Optional; + +import static org.apache.kafka.clients.consumer.CloseOptions.GroupMembershipOperation.DEFAULT; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class CloseOptionsTest { + + @Test + void operationShouldNotBeNull() { + assertThrows(NullPointerException.class, () -> CloseOptions.groupMembershipOperation(null)); + assertThrows(NullPointerException.class, () -> CloseOptions.timeout(Duration.ZERO).withGroupMembershipOperation(null)); + } + + @Test + void operationShouldHaveDefaultValue() { + assertEquals(DEFAULT, CloseOptions.timeout(Duration.ZERO).groupMembershipOperation()); + } + + @Test + void timeoutCouldBeNull() { + CloseOptions closeOptions = assertDoesNotThrow(() -> CloseOptions.timeout(null)); + assertEquals(Optional.empty(), closeOptions.timeout()); + } + + @Test + void timeoutShouldBeDefaultEmpty() { + assertEquals(Optional.empty(), CloseOptions.groupMembershipOperation(DEFAULT).timeout()); + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 37bd78f4bb0e2..deff7fff65660 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -60,6 +60,7 @@ import org.apache.kafka.common.message.ListOffsetsResponseData; import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; +import org.apache.kafka.common.message.OffsetFetchResponseData; import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.KafkaMetric; @@ -2870,16 +2871,26 @@ private SyncGroupResponse syncGroupResponse(List partitions, Err } private OffsetFetchResponse offsetResponse(Map offsets, Errors error) { - Map partitionData = new HashMap<>(); - for (Map.Entry entry : offsets.entrySet()) { - partitionData.put(entry.getKey(), new OffsetFetchResponse.PartitionData(entry.getValue(), - Optional.empty(), "", error)); - } - int throttleMs = 10; + var grouped = offsets.entrySet().stream().collect(Collectors.groupingBy(e -> e.getKey().topic())); + return new OffsetFetchResponse( - throttleMs, - Collections.singletonMap(groupId, Errors.NONE), - Collections.singletonMap(groupId, partitionData)); + new OffsetFetchResponseData() + .setGroups(List.of( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(groupId) + .setTopics(grouped.entrySet().stream().map(entry -> + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(entry.getKey()) + .setPartitions(entry.getValue().stream().map(partition -> + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(partition.getKey().partition()) + .setErrorCode(error.code()) + .setCommittedOffset(partition.getValue()) + ).collect(Collectors.toList())) + ).collect(Collectors.toList())) + )), + ApiKeys.OFFSET_FETCH.latestVersion() + ); } private ListOffsetsResponse listOffsetsResponse(Map offsets) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index 55d39db39a12c..df784da79379c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -1505,7 +1505,7 @@ public boolean matches(AbstractRequest body) { } @Test - public void testWakeupAfterSyncGroupReceived() throws Exception { + public void testWakeupAfterSyncGroupReceived() { setupCoordinator(RETRY_BACKOFF_MS, RETRY_BACKOFF_MAX_MS, REBALANCE_TIMEOUT_MS, Optional.empty(), Optional.of(() -> mock(BaseHeartbeatThread.class))); @@ -1537,7 +1537,7 @@ public void testWakeupAfterSyncGroupReceived() throws Exception { } @Test - public void testWakeupAfterSyncGroupReceivedExternalCompletion() throws Exception { + public void testWakeupAfterSyncGroupReceivedExternalCompletion() { setupCoordinator(RETRY_BACKOFF_MS, RETRY_BACKOFF_MAX_MS, REBALANCE_TIMEOUT_MS, Optional.empty(), Optional.of(() -> mock(BaseHeartbeatThread.class))); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 9a12ce75a6c97..bb9482aa636e8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -29,7 +29,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetCommitCallback; -import org.apache.kafka.clients.consumer.RetriableCommitFailedException; import org.apache.kafka.clients.consumer.SubscriptionPattern; import org.apache.kafka.clients.consumer.internals.events.ApplicationEvent; import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; @@ -68,7 +67,6 @@ import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.InvalidGroupIdException; -import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.errors.WakeupException; @@ -637,9 +635,7 @@ public void testEnsurePollExecutedCommitAsyncCallbacks() { consumer.assign(Collections.singleton(new TopicPartition("foo", 0))); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); markReconcileAndAutoCommitCompleteForPollEvent(); - assertMockCommitCallbackInvoked(() -> consumer.poll(Duration.ZERO), - callback, - null); + assertMockCommitCallbackInvoked(() -> consumer.poll(Duration.ZERO), callback); } @Test @@ -649,9 +645,7 @@ public void testEnsureShutdownExecutedCommitAsyncCallbacks() { MockCommitCallback callback = new MockCommitCallback(); completeCommitAsyncApplicationEventSuccessfully(); assertDoesNotThrow(() -> consumer.commitAsync(new HashMap<>(), callback)); - assertMockCommitCallbackInvoked(() -> consumer.close(), - callback, - null); + assertMockCommitCallbackInvoked(() -> consumer.close(), callback); } @Test @@ -782,15 +776,10 @@ public void testAutoCommitSyncDisabled() { verify(applicationEventHandler, never()).add(any(SyncCommitEvent.class)); } - private void assertMockCommitCallbackInvoked(final Executable task, - final MockCommitCallback callback, - final Errors errors) { + private void assertMockCommitCallbackInvoked(final Executable task, final MockCommitCallback callback) { assertDoesNotThrow(task); assertEquals(1, callback.invoked); - if (errors == null) - assertNull(callback.exception); - else if (errors.exception() instanceof RetriableException) - assertInstanceOf(RetriableCommitFailedException.class, callback.exception); + assertNull(callback.exception); } private static class MockCommitCallback implements OffsetCommitCallback { @@ -993,9 +982,8 @@ public void testBeginningOffsetsWithZeroTimeout() { TopicPartition tp = new TopicPartition("topic1", 0); Map result = assertDoesNotThrow(() -> consumer.beginningOffsets(Collections.singletonList(tp), Duration.ZERO)); - // The result should be {tp=null} - assertTrue(result.containsKey(tp)); - assertNull(result.get(tp)); + assertNotNull(result); + assertEquals(0, result.size()); verify(applicationEventHandler).add(ArgumentMatchers.isA(ListOffsetsEvent.class)); } @@ -1186,14 +1174,14 @@ public void testNoInterceptorCommitAsyncFailed() { @Test public void testRefreshCommittedOffsetsShouldNotResetIfFailedWithTimeout() { consumer = newConsumer(); - testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(true); + testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(); } @Test public void testRefreshCommittedOffsetsNotCalledIfNoGroupId() { // Create consumer without group id so committed offsets are not used for updating positions consumer = newConsumerWithoutGroupId(); - testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(false); + testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(); } @Test @@ -1648,7 +1636,7 @@ private Properties requiredConsumerConfigAndGroupId(final String groupId) { return props; } - private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout(boolean committedOffsetsEnabled) { + private void testUpdateFetchPositionsWithFetchCommittedOffsetsTimeout() { completeFetchedCommittedOffsetApplicationEventExceptionally(new TimeoutException()); doReturn(Fetch.empty()).when(fetchCollector).collectFetch(any(FetchBuffer.class)); when(applicationEventHandler.addAndGet(any(CheckAndUpdatePositionsEvent.class))).thenReturn(true); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index add8529f1f339..d4ceeedde56b1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -36,6 +36,7 @@ import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitResponseData; import org.apache.kafka.common.message.OffsetFetchRequestData; +import org.apache.kafka.common.message.OffsetFetchResponseData; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.ApiKeys; @@ -429,16 +430,6 @@ public void testCommitSyncRetriedAfterExpectedRetriableException(Errors error) { assertExceptionHandling(commitRequestManager, error, true); } - private static Stream commitSyncExpectedExceptions() { - return Stream.of( - Arguments.of(Errors.UNKNOWN_MEMBER_ID, CommitFailedException.class), - Arguments.of(Errors.OFFSET_METADATA_TOO_LARGE, Errors.OFFSET_METADATA_TOO_LARGE.exception().getClass()), - Arguments.of(Errors.INVALID_COMMIT_OFFSET_SIZE, Errors.INVALID_COMMIT_OFFSET_SIZE.exception().getClass()), - Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED, Errors.GROUP_AUTHORIZATION_FAILED.exception().getClass()), - Arguments.of(Errors.CORRUPT_MESSAGE, KafkaException.class), - Arguments.of(Errors.UNKNOWN_SERVER_ERROR, KafkaException.class)); - } - @Test public void testCommitSyncFailsWithCommitFailedExceptionIfUnknownMemberId() { CommitRequestManager commitRequestManager = create(false, 100); @@ -762,10 +753,10 @@ public void testSuccessfulOffsetFetch() { CommitRequestManager commitManager = create(false, 100); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); + TopicPartition tp = new TopicPartition("topic1", 0); long deadlineMs = time.milliseconds() + defaultApiTimeoutMs; CompletableFuture> fetchResult = - commitManager.fetchOffsets(Collections.singleton(new TopicPartition("test", 0)), - deadlineMs); + commitManager.fetchOffsets(Collections.singleton(tp), deadlineMs); // Send fetch request NetworkClientDelegate.PollResult result = commitManager.poll(time.milliseconds()); @@ -774,14 +765,21 @@ public void testSuccessfulOffsetFetch() { assertFalse(fetchResult.isDone()); // Complete request with a response - TopicPartition tp = new TopicPartition("topic1", 0); long expectedOffset = 100; NetworkClientDelegate.UnsentRequest req = result.unsentRequests.get(0); - Map topicPartitionData = - Collections.singletonMap( - tp, - new OffsetFetchResponse.PartitionData(expectedOffset, Optional.of(1), "", Errors.NONE)); - req.handler().onComplete(buildOffsetFetchClientResponse(req, topicPartitionData, Errors.NONE, false)); + OffsetFetchResponseData.OffsetFetchResponseGroup groupResponse = new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(DEFAULT_GROUP_ID) + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(tp.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(tp.partition()) + .setCommittedOffset(expectedOffset) + .setCommittedLeaderEpoch(1) + )) + )); + req.handler().onComplete(buildOffsetFetchClientResponse(req, groupResponse, false)); // Validate request future completes with the response received assertTrue(fetchResult.isDone()); @@ -1410,15 +1408,43 @@ public void testOffsetFetchRequestPartitionDataError(final Errors error, final b assertEquals(1, res.unsentRequests.size()); // Setting 1 partition with error - HashMap topicPartitionData = new HashMap<>(); - topicPartitionData.put(tp1, new OffsetFetchResponse.PartitionData(100L, Optional.of(1), "metadata", error)); - topicPartitionData.put(tp2, new OffsetFetchResponse.PartitionData(100L, Optional.of(1), "metadata", Errors.NONE)); - topicPartitionData.put(tp3, new OffsetFetchResponse.PartitionData(100L, Optional.of(1), "metadata", error)); + OffsetFetchResponseData.OffsetFetchResponseGroup groupResponse = new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(DEFAULT_GROUP_ID) + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(tp1.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(tp1.partition()) + .setCommittedOffset(100L) + .setCommittedLeaderEpoch(1) + .setMetadata("metadata") + .setErrorCode(error.code()) + )), + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(tp2.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(tp2.partition()) + .setCommittedOffset(100L) + .setCommittedLeaderEpoch(1) + .setMetadata("metadata") + )), + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(tp3.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(tp3.partition()) + .setCommittedOffset(100L) + .setCommittedLeaderEpoch(1) + .setMetadata("metadata") + .setErrorCode(error.code()) + )) + )); res.unsentRequests.get(0).handler().onComplete(buildOffsetFetchClientResponse( res.unsentRequests.get(0), - topicPartitionData, - Errors.NONE, + groupResponse, false)); if (isRetriable) testRetriable(commitRequestManager, Collections.singletonList(future), error); @@ -1559,18 +1585,26 @@ private ClientResponse buildOffsetFetchClientResponse( final NetworkClientDelegate.UnsentRequest request, final Set topicPartitions, final Errors error) { - HashMap topicPartitionData = new HashMap<>(); - topicPartitions.forEach(tp -> topicPartitionData.put(tp, new OffsetFetchResponse.PartitionData( - 100L, - Optional.of(1), - "metadata", - Errors.NONE))); - return buildOffsetFetchClientResponse(request, topicPartitionData, error, false); + OffsetFetchResponseData.OffsetFetchResponseGroup group = new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(DEFAULT_GROUP_ID) + .setErrorCode(error.code()) + .setTopics(topicPartitions.stream().collect(Collectors.groupingBy(TopicPartition::topic)).entrySet().stream().map(entry -> + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(entry.getKey()) + .setPartitions(entry.getValue().stream().map(partition -> + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(partition.partition()) + .setCommittedOffset(100L) + .setCommittedLeaderEpoch(1) + .setMetadata("metadata") + ).collect(Collectors.toList())) + ).collect(Collectors.toList())); + return buildOffsetFetchClientResponse(request, group, false); } private ClientResponse buildOffsetFetchClientResponseDisconnected( final NetworkClientDelegate.UnsentRequest request) { - return buildOffsetFetchClientResponse(request, Collections.emptyMap(), Errors.NONE, true); + return buildOffsetFetchClientResponse(request, new OffsetFetchResponseData.OffsetFetchResponseGroup(), true); } private ClientResponse buildOffsetCommitClientResponse(final OffsetCommitResponse commitResponse) { @@ -1686,14 +1720,12 @@ private ClientResponse mockOffsetCommitResponseDisconnected(String topic, int pa private ClientResponse buildOffsetFetchClientResponse( final NetworkClientDelegate.UnsentRequest request, - final Map topicPartitionData, - final Errors error, + final OffsetFetchResponseData.OffsetFetchResponseGroup groupResponse, final boolean disconnected) { AbstractRequest abstractRequest = request.requestBuilder().build(); assertInstanceOf(OffsetFetchRequest.class, abstractRequest); OffsetFetchRequest offsetFetchRequest = (OffsetFetchRequest) abstractRequest; - OffsetFetchResponse response = - new OffsetFetchResponse(error, topicPartitionData); + OffsetFetchResponse response = new OffsetFetchResponse.Builder(groupResponse).build(ApiKeys.OFFSET_FETCH.latestVersion()); return new ClientResponse( new RequestHeader(ApiKeys.OFFSET_FETCH, offsetFetchRequest.version(), "", 1), request.handler(), diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index a34f2f1633778..8d364b2ae30ce 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -52,6 +52,7 @@ import org.apache.kafka.common.message.LeaveGroupResponseData; import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitResponseData; +import org.apache.kafka.common.message.OffsetFetchResponseData; import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; @@ -73,7 +74,6 @@ import org.apache.kafka.common.requests.OffsetCommitRequest; import org.apache.kafka.common.requests.OffsetCommitResponse; import org.apache.kafka.common.requests.OffsetFetchResponse; -import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.requests.SyncGroupRequest; import org.apache.kafka.common.requests.SyncGroupResponse; @@ -163,7 +163,7 @@ public abstract class ConsumerCoordinatorTest { private final String consumerId2 = "consumer2"; private MockClient client; - private final MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(1, new HashMap() { + private final MetadataResponse metadataResponse = RequestTestUtils.metadataUpdateWith(1, new HashMap<>() { { put(topic1, 1); put(topic2, 1); @@ -367,7 +367,7 @@ private List validateCooperativeA List metadata = new ArrayList<>(); for (Map.Entry> subscriptionEntry : memberSubscriptions.entrySet()) { - ByteBuffer buf = null; + ByteBuffer buf; if (subscriptionEntry.getKey().equals(consumerId)) { buf = ConsumerProtocol.serializeSubscription(subscriptionConsumer1); } else { @@ -614,9 +614,7 @@ public void testEnsureCompletingAsyncCommitsWhenSyncCommitWithoutOffsets() { Map offsets = singletonMap(tp, new OffsetAndMetadata(123)); final AtomicBoolean committed = new AtomicBoolean(); - coordinator.commitOffsetsAsync(offsets, (committedOffsets, exception) -> { - committed.set(true); - }); + coordinator.commitOffsetsAsync(offsets, (committedOffsets, exception) -> committed.set(true)); assertFalse(coordinator.commitOffsetsSync(Collections.emptyMap(), time.timer(100L)), "expected sync commit to fail"); assertFalse(committed.get()); @@ -683,7 +681,7 @@ public void testCoordinatorUnknownInUnsentCallbacksAfterCoordinatorDead() { ); consumerClient.send(coordinator.checkAndGetCoordinator(), OffsetCommitRequest.Builder.forTopicNames(offsetCommitRequestData)) - .compose(new RequestFutureAdapter() { + .compose(new RequestFutureAdapter<>() { @Override public void onSuccess(ClientResponse value, RequestFuture future) {} @@ -1294,7 +1292,7 @@ public void testForceMetadataRefreshForPatternSubscriptionDuringRebalance() { public void testForceMetadataDeleteForPatternSubscriptionDuringRebalance() { try (ConsumerCoordinator coordinator = buildCoordinator(rebalanceConfig, new Metrics(), assignors, true, subscriptions)) { subscriptions.subscribe(Pattern.compile("test.*"), Optional.of(rebalanceListener)); - client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, new HashMap() { + client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, new HashMap<>() { { put(topic1, 1); put(topic2, 1); @@ -1306,11 +1304,7 @@ public void testForceMetadataDeleteForPatternSubscriptionDuringRebalance() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); - MetadataResponse deletedMetadataResponse = RequestTestUtils.metadataUpdateWith(1, new HashMap() { - { - put(topic1, 1); - } - }); + MetadataResponse deletedMetadataResponse = RequestTestUtils.metadataUpdateWith(1, Map.of(topic1, 1)); // Instrument the test so that metadata will contain only one topic after next refresh. client.prepareMetadataUpdate(deletedMetadataResponse); @@ -2697,12 +2691,7 @@ public void testAsyncCommitCallbacksInvokedPriorToSyncCommitCompletion() throws final OffsetAndMetadata firstOffset = new OffsetAndMetadata(0L); final OffsetAndMetadata secondOffset = new OffsetAndMetadata(1L); - coordinator.commitOffsetsAsync(singletonMap(t1p, firstOffset), new OffsetCommitCallback() { - @Override - public void onComplete(Map offsets, Exception exception) { - committedOffsets.add(firstOffset); - } - }); + coordinator.commitOffsetsAsync(singletonMap(t1p, firstOffset), (offsets, exception) -> committedOffsets.add(firstOffset)); // Do a synchronous commit in the background so that we can send both responses at the same time Thread thread = new Thread() { @@ -3122,10 +3111,19 @@ public void testFetchCommittedOffsets() { long offset = 500L; String metadata = "blahblah"; Optional leaderEpoch = Optional.of(15); - OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, leaderEpoch, - metadata, Errors.NONE); - client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); + client.prepareResponse(offsetFetchResponse(Errors.NONE, List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(t1p.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t1p.partition()) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get()) + .setMetadata(metadata) + )) + ))); + Map fetchedOffsets = coordinator.fetchCommittedOffsets(singleton(t1p), time.timer(Long.MAX_VALUE)); @@ -3138,10 +3136,17 @@ public void testTopicAuthorizationFailedInOffsetFetch() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); - OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(-1, Optional.empty(), - "", Errors.TOPIC_AUTHORIZATION_FAILED); + client.prepareResponse(offsetFetchResponse(Errors.NONE, List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(t1p.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t1p.partition()) + .setCommittedOffset(-1) + .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code()) + )) + ))); - client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); TopicAuthorizationException exception = assertThrows(TopicAuthorizationException.class, () -> coordinator.fetchCommittedOffsets(singleton(t1p), time.timer(Long.MAX_VALUE))); @@ -3154,7 +3159,7 @@ public void testRefreshOffsetsGroupNotAuthorized() { coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); subscriptions.assignFromUser(singleton(t1p)); - client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED, Collections.emptyMap())); + client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED, List.of())); try { coordinator.initWithCommittedOffsetsIfNeeded(time.timer(Long.MAX_VALUE)); fail("Expected group authorization error"); @@ -3203,7 +3208,7 @@ public void testRefreshOffsetRetriableErrorCoordinatorLookup(Errors error, boole coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE)); subscriptions.assignFromUser(singleton(t1p)); - client.prepareResponse(offsetFetchResponse(error, Collections.emptyMap())); + client.prepareResponse(offsetFetchResponse(error, List.of())); if (expectCoordinatorRelookup) { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE)); } @@ -3703,14 +3708,19 @@ private void supportStableFlag(final short upperVersion, final boolean expectThr long offset = 500L; String metadata = "blahblah"; Optional leaderEpoch = Optional.of(15); - OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, leaderEpoch, - metadata, Errors.NONE); - if (upperVersion < 8) { - client.prepareResponse(new OffsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); - } else { - client.prepareResponse(offsetFetchResponse(Errors.NONE, singletonMap(t1p, data))); - } + client.prepareResponse(offsetFetchResponse(Errors.NONE, List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(t1p.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(t1p.partition()) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get()) + .setMetadata(metadata) + )) + ))); + if (expectThrows) { assertThrows(UnsupportedVersionException.class, () -> coordinator.fetchCommittedOffsets(singleton(t1p), time.timer(Long.MAX_VALUE))); @@ -3975,10 +3985,20 @@ private OffsetCommitResponse offsetCommitResponse(Map re return new OffsetCommitResponse(responseData); } - private OffsetFetchResponse offsetFetchResponse(Errors error, Map responseData) { - return new OffsetFetchResponse(throttleMs, - singletonMap(groupId, error), - singletonMap(groupId, responseData)); + private OffsetFetchResponse offsetFetchResponse( + Errors errors, + List topics + ) { + return new OffsetFetchResponse( + new OffsetFetchResponseData() + .setGroups(List.of( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(groupId) + .setErrorCode(errors.code()) + .setTopics(topics) + )), + ApiKeys.OFFSET_FETCH.latestVersion() + ); } private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset) { @@ -3986,9 +4006,18 @@ private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partit } private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset, Optional epoch) { - OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, - epoch, metadata, partitionLevelError); - return offsetFetchResponse(Errors.NONE, singletonMap(tp, data)); + return offsetFetchResponse(Errors.NONE, List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(tp.topic()) + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(tp.partition()) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(epoch.orElse(-1)) + .setMetadata(metadata) + .setErrorCode(partitionLevelError.code()) + )) + )); } private OffsetCommitCallback callback(final AtomicBoolean success) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerHeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerHeartbeatRequestManagerTest.java index 2febd2085b8f9..2bdc8819aec64 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerHeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerHeartbeatRequestManagerTest.java @@ -647,17 +647,6 @@ public void testUnsupportedVersionFromClient(String errorMsg) { clearInvocations(backgroundEventHandler); } - private void mockErrorResponse(Errors error, String exceptionCustomMsg) { - time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); - NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); - assertEquals(1, result.unsentRequests.size()); - - when(subscriptions.hasAutoAssignedPartitions()).thenReturn(true); - ClientResponse response = createHeartbeatResponse( - result.unsentRequests.get(0), error, exceptionCustomMsg); - result.unsentRequests.get(0).handler().onComplete(response); - } - private void mockResponseWithException(UnsupportedVersionException exception, boolean isFromBroker) { time.sleep(DEFAULT_HEARTBEAT_INTERVAL_MS); NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java index 520279fc8d454..de0653b616f6f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThreadTest.java @@ -35,11 +35,8 @@ import org.junit.jupiter.params.provider.ValueSource; import java.time.Duration; -import java.util.ArrayList; -import java.util.Collections; import java.util.LinkedList; import java.util.List; -import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; @@ -117,10 +114,7 @@ public void testEnsureCloseStopsRunningThread() { @ParameterizedTest @ValueSource(longs = {ConsumerNetworkThread.MAX_POLL_TIMEOUT_MS - 1, ConsumerNetworkThread.MAX_POLL_TIMEOUT_MS, ConsumerNetworkThread.MAX_POLL_TIMEOUT_MS + 1}) public void testConsumerNetworkThreadPollTimeComputations(long exampleTime) { - List> list = new ArrayList<>(); - list.add(Optional.of(coordinatorRequestManager)); - list.add(Optional.of(heartbeatRequestManager)); - + List list = List.of(coordinatorRequestManager, heartbeatRequestManager); when(requestManagers.entries()).thenReturn(list); NetworkClientDelegate.PollResult pollResult = new NetworkClientDelegate.PollResult(exampleTime); @@ -158,16 +152,13 @@ public void testStartupAndTearDown() throws InterruptedException { @Test public void testRequestsTransferFromManagersToClientOnThreadRun() { - List> list = new ArrayList<>(); - list.add(Optional.of(coordinatorRequestManager)); - list.add(Optional.of(heartbeatRequestManager)); - list.add(Optional.of(offsetsRequestManager)); + List list = List.of(coordinatorRequestManager, heartbeatRequestManager, offsetsRequestManager); when(requestManagers.entries()).thenReturn(list); when(coordinatorRequestManager.poll(anyLong())).thenReturn(mock(NetworkClientDelegate.PollResult.class)); consumerNetworkThread.runOnce(); - requestManagers.entries().forEach(rmo -> rmo.ifPresent(rm -> verify(rm).poll(anyLong()))); - requestManagers.entries().forEach(rmo -> rmo.ifPresent(rm -> verify(rm).maximumTimeToWait(anyLong()))); + requestManagers.entries().forEach(rm -> verify(rm).poll(anyLong())); + requestManagers.entries().forEach(rm -> verify(rm).maximumTimeToWait(anyLong())); verify(networkClientDelegate).addAll(any(NetworkClientDelegate.PollResult.class)); verify(networkClientDelegate).poll(anyLong(), anyLong()); } @@ -178,7 +169,7 @@ public void testMaximumTimeToWait() { // Initial value before runOnce has been called assertEquals(ConsumerNetworkThread.MAX_POLL_TIMEOUT_MS, consumerNetworkThread.maximumTimeToWait()); - when(requestManagers.entries()).thenReturn(Collections.singletonList(Optional.of(heartbeatRequestManager))); + when(requestManagers.entries()).thenReturn(List.of(heartbeatRequestManager)); when(heartbeatRequestManager.maximumTimeToWait(time.milliseconds())).thenReturn((long) defaultHeartbeatIntervalMs); consumerNetworkThread.runOnce(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index e25bccc1892b2..7c8547ddd88e6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -382,7 +382,7 @@ public void testFetcherCloseClosesFetchSessionsInBroker() { // NOTE: by design the FetchRequestManager doesn't perform network I/O internally. That means that calling // the close() method with a Timer will NOT send out the close session requests on close. The network // I/O logic is handled inside ConsumerNetworkThread.runAtClose, so we need to run that logic here. - ConsumerNetworkThread.runAtClose(singletonList(Optional.of(fetcher)), networkClientDelegate, time.milliseconds()); + ConsumerNetworkThread.runAtClose(List.of(fetcher), networkClientDelegate, time.milliseconds()); // the network is polled during the last state of clean up. networkClientDelegate.poll(time.timer(1)); // validate that closing the fetcher has sent a request with final epoch. 2 requests are sent, one for the diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java index 64c729730fbc4..6880ad16b8d0f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java @@ -41,6 +41,7 @@ import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; @@ -50,6 +51,8 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import org.mockito.ArgumentMatchers; import org.mockito.InOrder; import org.mockito.Mockito; @@ -672,6 +675,32 @@ public void testEnsurePollEventSentOnConsumerPoll() { verify(applicationEventHandler).addAndGet(any(ShareAcknowledgeOnCloseEvent.class)); } + @ParameterizedTest + @EnumSource(value = Errors.class, names = {"TOPIC_AUTHORIZATION_FAILED", "GROUP_AUTHORIZATION_FAILED", "INVALID_TOPIC_EXCEPTION"}) + public void testCloseWithBackgroundQueueErrorsAfterUnsubscribe(Errors error) { + SubscriptionState subscriptions = new SubscriptionState(new LogContext(), AutoOffsetResetStrategy.NONE); + consumer = newConsumer(subscriptions); + + // Complete the acknowledge on close event successfully + completeShareAcknowledgeOnCloseApplicationEventSuccessfully(); + + // Complete the unsubscribe event successfully + completeShareUnsubscribeApplicationEventSuccessfully(subscriptions); + + // Mock the applicationEventHandler to add errors to the queue after unsubscribe + doAnswer(invocation -> { + // Add errors to the queue after unsubscribe event is processed + backgroundEventQueue.add(new ErrorEvent(error.exception())); + return null; + }).when(applicationEventHandler).add(any(StopFindCoordinatorOnCloseEvent.class)); + + // Close should complete successfully despite the errors in the background queue + assertDoesNotThrow(() -> consumer.close()); + + // Verify that the background queue was processed + assertTrue(backgroundEventQueue.isEmpty(), "Background queue should be empty after close"); + } + private Properties requiredConsumerPropertiesAndGroupId(final String groupId) { final Properties props = requiredConsumerProperties(); props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManagerTest.java index 7c4c5684bcce0..49bf42b1b4f18 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareMembershipManagerTest.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData.Assignment; -import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ShareGroupHeartbeatRequest; @@ -57,7 +56,6 @@ import java.util.stream.Stream; import static org.apache.kafka.clients.consumer.internals.AbstractMembershipManager.TOPIC_PARTITION_COMPARATOR; -import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_SHARE_METRIC_GROUP_PREFIX; import static org.apache.kafka.common.requests.ShareGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -1228,23 +1226,6 @@ private ShareMembershipManager mockStaleMember() { return membershipManager; } - private void mockPartitionOwnedAndNewPartitionAdded(String topicName, - int partitionOwned, - int partitionAdded, - CounterConsumerRebalanceListener listener, - ShareMembershipManager membershipManager) { - Uuid topicId = Uuid.randomUuid(); - TopicPartition owned = new TopicPartition(topicName, partitionOwned); - when(subscriptionState.assignedPartitions()).thenReturn(Collections.singleton(owned)); - membershipManager.updateAssignment(Collections.singletonMap(topicId, mkSortedSet(partitionOwned))); - when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); - when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); - when(subscriptionState.rebalanceListener()).thenReturn(Optional.ofNullable(listener)); - - // Receive assignment adding a new partition - receiveAssignment(topicId, Arrays.asList(partitionOwned, partitionAdded), membershipManager); - } - private SortedSet topicIdPartitionsSet(Uuid topicId, String topicName, int... partitions) { SortedSet topicIdPartitions = new TreeSet<>(new Utils.TopicIdPartitionComparator()); @@ -1573,17 +1554,6 @@ private void mockLeaveGroup() { doNothing().when(subscriptionState).markPendingRevocation(anySet()); } - private void mockPrepareLeaving(ShareMembershipManager membershipManager) { - String topicName = "topic1"; - TopicPartition ownedPartition = new TopicPartition(topicName, 0); - - // Start leaving group, blocked waiting for callback to complete. - when(subscriptionState.assignedPartitions()).thenReturn(Collections.singleton(ownedPartition)); - when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); - doNothing().when(subscriptionState).markPendingRevocation(anySet()); - membershipManager.leaveGroup(); - } - private void testStateUpdateOnFatalFailure(ShareMembershipManager membershipManager) { String memberId = membershipManager.memberId(); int lastEpoch = membershipManager.memberEpoch(); @@ -1641,10 +1611,6 @@ private ShareGroupHeartbeatResponseData.Assignment createAssignment(boolean mock )); } - private KafkaMetric getMetric(final String name) { - return metrics.metrics().get(metrics.metricName(name, CONSUMER_SHARE_METRIC_GROUP_PREFIX + "-coordinator-metrics")); - } - private ShareMembershipManager memberJoinWithAssignment() { Uuid topicId = Uuid.randomUuid(); ShareMembershipManager membershipManager = mockJoinAndReceiveAssignment(true); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandlerTest.java index 5a52b7bc35fe1..2549dcbf8ad4f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareSessionHandlerTest.java @@ -31,6 +31,8 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import java.util.ArrayList; import java.util.Collections; @@ -149,15 +151,15 @@ private static LinkedHashMap topicIds = new HashMap<>(); Map topicNames = new HashMap<>(); - Uuid fooId = addTopicId(topicIds, topicNames, "foo"); + Uuid fooId = addTopicId(topicNames, "foo"); TopicIdPartition foo0 = new TopicIdPartition(fooId, 0, "foo"); TopicIdPartition foo1 = new TopicIdPartition(fooId, 1, "foo"); handler.addPartitionToFetch(foo0, null); @@ -177,7 +179,7 @@ public void testShareSession() { handler.handleResponse(resp, ApiKeys.SHARE_FETCH.latestVersion(true)); // Test a fetch request which adds one partition - Uuid barId = addTopicId(topicIds, topicNames, "bar"); + Uuid barId = addTopicId(topicNames, "bar"); TopicIdPartition bar0 = new TopicIdPartition(barId, 0, "bar"); handler.addPartitionToFetch(foo0, null); handler.addPartitionToFetch(foo1, null); @@ -199,7 +201,7 @@ public void testShareSession() { handler.handleResponse(resp2, ApiKeys.SHARE_FETCH.latestVersion(true)); // A top-level error code will reset the session epoch - ShareFetchResponse resp3 = ShareFetchResponse.of(Errors.INVALID_SHARE_SESSION_EPOCH, 0, new LinkedHashMap<>(), List.of(), 0); + ShareFetchResponse resp3 = ShareFetchResponse.of(error, 0, new LinkedHashMap<>(), List.of(), 0); handler.handleResponse(resp3, ApiKeys.SHARE_FETCH.latestVersion(true)); ShareFetchRequestData requestData4 = handler.newShareFetchBuilder(groupId, fetchConfig).build().data(); @@ -222,10 +224,9 @@ public void testPartitionRemoval() { Uuid memberId = Uuid.randomUuid(); ShareSessionHandler handler = new ShareSessionHandler(LOG_CONTEXT, 1, memberId); - Map topicIds = new HashMap<>(); Map topicNames = new HashMap<>(); - Uuid fooId = addTopicId(topicIds, topicNames, "foo"); - Uuid barId = addTopicId(topicIds, topicNames, "bar"); + Uuid fooId = addTopicId(topicNames, "foo"); + Uuid barId = addTopicId(topicNames, "bar"); TopicIdPartition foo0 = new TopicIdPartition(fooId, 0, "foo"); TopicIdPartition foo1 = new TopicIdPartition(fooId, 1, "foo"); TopicIdPartition bar0 = new TopicIdPartition(barId, 0, "bar"); @@ -289,9 +290,8 @@ public void testTopicIdReplaced() { Uuid memberId = Uuid.randomUuid(); ShareSessionHandler handler = new ShareSessionHandler(LOG_CONTEXT, 1, memberId); - Map topicIds = new HashMap<>(); Map topicNames = new HashMap<>(); - Uuid topicId1 = addTopicId(topicIds, topicNames, "foo"); + Uuid topicId1 = addTopicId(topicNames, "foo"); TopicIdPartition tp = new TopicIdPartition(topicId1, 0, "foo"); handler.addPartitionToFetch(tp, null); ShareFetchRequestData requestData1 = handler.newShareFetchBuilder(groupId, fetchConfig).build().data(); @@ -309,7 +309,7 @@ public void testTopicIdReplaced() { handler.handleResponse(resp, ApiKeys.SHARE_FETCH.latestVersion(true)); // Try to add a new topic ID - Uuid topicId2 = addTopicId(topicIds, topicNames, "foo"); + Uuid topicId2 = addTopicId(topicNames, "foo"); TopicIdPartition tp2 = new TopicIdPartition(topicId2, 0, "foo"); // Use the same data besides the topic ID handler.addPartitionToFetch(tp2, null); @@ -334,9 +334,8 @@ public void testPartitionForgottenOnAcknowledgeOnly() { ShareSessionHandler handler = new ShareSessionHandler(LOG_CONTEXT, 1, memberId); // We want to test when all topics are removed from the session - Map topicIds = new HashMap<>(); Map topicNames = new HashMap<>(); - Uuid topicId = addTopicId(topicIds, topicNames, "foo"); + Uuid topicId = addTopicId(topicNames, "foo"); TopicIdPartition foo0 = new TopicIdPartition(topicId, 0, "foo"); handler.addPartitionToFetch(foo0, null); ShareFetchRequestData requestData1 = handler.newShareFetchBuilder(groupId, fetchConfig).build().data(); @@ -369,9 +368,8 @@ public void testForgottenPartitions() { ShareSessionHandler handler = new ShareSessionHandler(LOG_CONTEXT, 1, memberId); // We want to test when all topics are removed from the session - Map topicIds = new HashMap<>(); Map topicNames = new HashMap<>(); - Uuid topicId = addTopicId(topicIds, topicNames, "foo"); + Uuid topicId = addTopicId(topicNames, "foo"); TopicIdPartition foo0 = new TopicIdPartition(topicId, 0, "foo"); handler.addPartitionToFetch(foo0, null); ShareFetchRequestData requestData1 = handler.newShareFetchBuilder(groupId, fetchConfig).build().data(); @@ -402,9 +400,8 @@ public void testAddNewIdAfterTopicRemovedFromSession() { Uuid memberId = Uuid.randomUuid(); ShareSessionHandler handler = new ShareSessionHandler(LOG_CONTEXT, 1, memberId); - Map topicIds = new HashMap<>(); Map topicNames = new HashMap<>(); - Uuid topicId = addTopicId(topicIds, topicNames, "foo"); + Uuid topicId = addTopicId(topicNames, "foo"); handler.addPartitionToFetch(new TopicIdPartition(topicId, 0, "foo"), null); ShareFetchRequestData requestData1 = handler.newShareFetchBuilder(groupId, fetchConfig).build().data(); assertMapsEqual(reqMap(new TopicIdPartition(topicId, 0, "foo")), @@ -428,7 +425,7 @@ public void testAddNewIdAfterTopicRemovedFromSession() { handler.handleResponse(resp2, ApiKeys.SHARE_FETCH.latestVersion(true)); // After the topic is removed, add a recreated topic with a new ID - Uuid topicId2 = addTopicId(topicIds, topicNames, "foo"); + Uuid topicId2 = addTopicId(topicNames, "foo"); handler.addPartitionToFetch(new TopicIdPartition(topicId2, 0, "foo"), null); ShareFetchRequestData requestData3 = handler.newShareFetchBuilder(groupId, fetchConfig).build().data(); @@ -443,9 +440,8 @@ public void testNextAcknowledgementsClearedOnInvalidRequest() { Uuid memberId = Uuid.randomUuid(); ShareSessionHandler handler = new ShareSessionHandler(LOG_CONTEXT, 1, memberId); - Map topicIds = new HashMap<>(); Map topicNames = new HashMap<>(); - Uuid fooId = addTopicId(topicIds, topicNames, "foo"); + Uuid fooId = addTopicId(topicNames, "foo"); TopicIdPartition foo0 = new TopicIdPartition(fooId, 0, "foo"); Acknowledgements acknowledgements = Acknowledgements.empty(); @@ -470,9 +466,7 @@ public void testNextAcknowledgementsClearedOnInvalidRequest() { assertEquals(memberId.toString(), requestData.memberId()); } - private Uuid addTopicId(Map topicIds, Map topicNames, String name) { - // If the same topic name is added more than once, the latest mapping will be in the - // topicIds, but all mappings will be in topicNames. This is needed in the replace tests. + private Uuid addTopicId(Map topicNames, String name) { Uuid id = Uuid.randomUuid(); topicNames.put(id, name); return id; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSetTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSetTest.java new file mode 100644 index 0000000000000..f14e911578acc --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicIdPartitionSetTest.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TopicIdPartitionSetTest { + + private TopicIdPartitionSet topicIdPartitionSet; + + + @BeforeEach + public void setUp() { + topicIdPartitionSet = new TopicIdPartitionSet(); + } + + @Test + public void testIsEmpty() { + assertTrue(topicIdPartitionSet.isEmpty()); + + TopicIdPartition topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + topicIdPartitionSet.add(topicIdPartition); + + assertFalse(topicIdPartitionSet.isEmpty()); + } + + @Test + public void testRetrieveTopicPartitions() { + TopicPartition tp1 = new TopicPartition("foo", 0); + TopicPartition tp2 = new TopicPartition("foo", 1); + TopicPartition tp3 = new TopicPartition("bar", 0); + Uuid topicId1 = Uuid.randomUuid(); + Uuid topicId2 = Uuid.randomUuid(); + topicIdPartitionSet.add(new TopicIdPartition(topicId1, tp1)); + topicIdPartitionSet.add(new TopicIdPartition(topicId1, tp2)); + topicIdPartitionSet.add(new TopicIdPartition(topicId2, tp3)); + + Set topicPartitionSet = topicIdPartitionSet.topicPartitions(); + assertEquals(3, topicPartitionSet.size()); + assertTrue(topicPartitionSet.contains(tp1)); + assertTrue(topicPartitionSet.contains(tp2)); + assertTrue(topicPartitionSet.contains(tp3)); + } + + @Test + public void testRetrieveTopicIds() { + Uuid topicId1 = Uuid.randomUuid(); + Uuid topicId2 = Uuid.randomUuid(); + topicIdPartitionSet.add(new TopicIdPartition(topicId1, new TopicPartition("foo", 0))); + topicIdPartitionSet.add(new TopicIdPartition(topicId1, new TopicPartition("foo", 1))); + topicIdPartitionSet.add(new TopicIdPartition(topicId2, new TopicPartition("bar", 0))); + + Set topicIds = topicIdPartitionSet.topicIds(); + assertEquals(2, topicIds.size()); + assertTrue(topicIds.contains(topicId1)); + assertTrue(topicIds.contains(topicId2)); + } + + @Test + public void testRetrieveTopicNames() { + String topic1 = "foo"; + String topic2 = "bar"; + Uuid topicId1 = Uuid.randomUuid(); + Uuid topicId2 = Uuid.randomUuid(); + topicIdPartitionSet.add(new TopicIdPartition(topicId1, new TopicPartition(topic1, 0))); + topicIdPartitionSet.add(new TopicIdPartition(topicId1, new TopicPartition(topic1, 1))); + topicIdPartitionSet.add(new TopicIdPartition(topicId2, new TopicPartition(topic2, 0))); + + Set topicNames = topicIdPartitionSet.topicNames(); + assertEquals(2, topicNames.size()); + assertTrue(topicNames.contains(topic1)); + assertTrue(topicNames.contains(topic2)); + } + + @Test + public void testRetrievedTopicNamesAreSorted() { + LinkedHashSet expectedOrderedTopicPartitions = new LinkedHashSet<>(); + expectedOrderedTopicPartitions.add(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic-z", 1))); + expectedOrderedTopicPartitions.add(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic-z", 0))); + expectedOrderedTopicPartitions.add(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic-a", 0))); + expectedOrderedTopicPartitions.add(new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic-a", 1))); + + List reversed = new ArrayList<>(expectedOrderedTopicPartitions); + Collections.reverse(reversed); + reversed.forEach(tp -> topicIdPartitionSet.add(tp)); + + List topicPartitions = new ArrayList<>(topicIdPartitionSet.toTopicNamePartitionSet()); + + assertEquals(4, topicPartitions.size()); + assertEquals(new TopicPartition("topic-a", 0), topicPartitions.get(0)); + assertEquals(new TopicPartition("topic-a", 1), topicPartitions.get(1)); + assertEquals(new TopicPartition("topic-z", 0), topicPartitions.get(2)); + assertEquals(new TopicPartition("topic-z", 1), topicPartitions.get(3)); + } + + @Test + public void testToString() { + Uuid topicId1 = Uuid.randomUuid(); + TopicIdPartition tp1 = new TopicIdPartition(topicId1, new TopicPartition("topic-a", 0)); + TopicIdPartition tp2 = new TopicIdPartition(topicId1, new TopicPartition("topic-a", 1)); + TopicIdPartition tp3 = new TopicIdPartition(topicId1, new TopicPartition("topic-b", 0)); + topicIdPartitionSet.add(tp1); + topicIdPartitionSet.add(tp2); + topicIdPartitionSet.add(tp3); + + String toString = topicIdPartitionSet.toString(); + assertEquals(List.of(tp1, tp2, tp3).toString(), toString); + } + + @Test + public void testToStringSorted() { + Uuid topicId1 = Uuid.randomUuid(); + TopicIdPartition tp1 = new TopicIdPartition(topicId1, new TopicPartition("topic-a", 0)); + TopicIdPartition tpz1 = new TopicIdPartition(topicId1, new TopicPartition("topic-z", 0)); + TopicIdPartition tpz2 = new TopicIdPartition(topicId1, new TopicPartition("topic-z", 1)); + topicIdPartitionSet.add(tpz2); + topicIdPartitionSet.add(tpz1); + topicIdPartitionSet.add(tp1); + + String toString = topicIdPartitionSet.toString(); + assertEquals(List.of(tp1, tpz1, tpz2).toString(), toString); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java index 74f5db740628e..c81de1d74fc52 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java @@ -69,9 +69,10 @@ public void onAcknowledgement(RecordMetadata metadata, Exception exception) { onAckCount++; if (exception != null) { onErrorAckCount++; - // the length check is just to call topic() method and let it throw an exception - // if RecordMetadata.TopicPartition is null - if (metadata != null && metadata.topic().length() >= 0) { + if (metadata != null) { + if (metadata.topic() == null) { + throw new NullPointerException("Topic is null"); + } onErrorAckWithTopicSetCount++; if (metadata.partition() >= 0) onErrorAckWithTopicPartitionSetCount++; @@ -124,9 +125,10 @@ public void onAcknowledgement(RecordMetadata metadata, Exception exception, Head onAckCount++; if (exception != null) { onErrorAckCount++; - // the length check is just to call topic() method and let it throw an exception - // if RecordMetadata.TopicPartition is null - if (metadata != null && metadata.topic().length() >= 0) { + if (metadata != null) { + if (metadata.topic() == null) { + throw new NullPointerException("Topic is null"); + } onErrorAckWithTopicSetCount++; if (metadata.partition() >= 0) onErrorAckWithTopicPartitionSetCount++; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index cac3d2648b1e9..7c2d791ea5c58 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -203,7 +203,7 @@ public void testDrainBatches() throws Exception { } private void verifyTopicPartitionInBatches(Map> nodeBatches, TopicPartition... tp) { - int allTpBatchCount = (int) nodeBatches.values().stream().flatMap(Collection::stream).count(); + int allTpBatchCount = (int) nodeBatches.values().stream().mapToLong(Collection::size).sum(); assertEquals(tp.length, allTpBatchCount); List topicPartitionsInBatch = new ArrayList<>(); for (Map.Entry> entry : nodeBatches.entrySet()) { @@ -1610,22 +1610,6 @@ private int expectedNumAppends(int batchSize) { } } - /** - * Return the offset delta when there is no key. - */ - private int expectedNumAppendsNoKey(int batchSize) { - int size = 0; - int offsetDelta = 0; - while (true) { - int recordSize = DefaultRecord.sizeInBytes(offsetDelta, 0, 0, value.length, - Record.EMPTY_HEADERS); - if (size + recordSize > batchSize) - return offsetDelta; - offsetDelta += 1; - size += recordSize; - } - } - private RecordAccumulator createTestRecordAccumulator(int batchSize, long totalSize, Compression compression, int lingerMs) { int deliveryTimeoutMs = 3200; return createTestRecordAccumulator(deliveryTimeoutMs, batchSize, totalSize, compression, lingerMs); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 44d9bec5dde94..34392d58690ed 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -2413,7 +2413,7 @@ private void testSplitBatchAndSend(TransactionManager txnManager, // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1 MetadataResponse metadataUpdate1 = RequestTestUtils.metadataUpdateWithIds(2, Collections.singletonMap(topic, 2), TOPIC_IDS); client.prepareMetadataUpdate(metadataUpdate1); - metadataUpdate1.brokers().stream().forEach(node -> + metadataUpdate1.brokers().forEach(node -> apiVersions.update(node.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, ApiKeys.PRODUCE.oldestVersion(), ApiKeys.PRODUCE.latestVersion())) ); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index d53095c1538d9..e6ac3e56299bb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -3153,7 +3153,7 @@ public void testEpochBumpAfterLastInFlightBatchFailsIdempotentProducer(boolean t @ParameterizedTest @ValueSource(booleans = {true, false}) - public void testMaybeResolveSequencesTransactionalProducer(boolean transactionV2Enabled) throws Exception { + public void testMaybeResolveSequencesTransactionalProducer(boolean transactionV2Enabled) { initializeTransactionManager(Optional.of(transactionalId), transactionV2Enabled); // Initialize transaction with initial producer ID and epoch. @@ -4056,7 +4056,7 @@ private void prepareInitPidResponse( long producerId, short producerEpoch ) { - prepareInitPidResponse(error, shouldDisconnect, producerId, producerEpoch, false, false, (long) -1, (short) -1); + prepareInitPidResponse(error, shouldDisconnect, producerId, producerEpoch, false, false, -1, (short) -1); } private void prepareInitPidResponse( diff --git a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java index 90dd34bb07835..df5e1aea7f8df 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java +++ b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java @@ -183,11 +183,6 @@ public void verifyReauthenticationMetrics(int successfulReauthentications, final } } - public void verifyAuthenticationNoReauthMetric(int successfulAuthenticationNoReauths) throws InterruptedException { - waitForMetrics("successful-authentication-no-reauth", successfulAuthenticationNoReauths, - EnumSet.of(MetricType.TOTAL)); - } - public void waitForMetric(String name, final double expectedValue) throws InterruptedException { waitForMetrics(name, expectedValue, EnumSet.of(MetricType.TOTAL, MetricType.RATE)); } diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java index 9208171d1a926..d75028b1c80ca 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java @@ -1096,14 +1096,14 @@ false, securityProtocol, config, null, null, TIME, new LogContext(), CertStores invalidCertStores = certBuilder(true, "server", args.useInlinePem).addHostName("127.0.0.1").build(); Map invalidConfigs = args.getTrustingConfig(invalidCertStores, args.clientCertStores); - verifyInvalidReconfigure(reconfigurableBuilder, invalidConfigs, "keystore with different SubjectAltName"); + verifyInvalidReconfigure(reconfigurableBuilder, invalidConfigs); Map missingStoreConfigs = new HashMap<>(); missingStoreConfigs.put(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, "PKCS12"); missingStoreConfigs.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, "some.keystore.path"); missingStoreConfigs.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, new Password("some.keystore.password")); missingStoreConfigs.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, new Password("some.key.password")); - verifyInvalidReconfigure(reconfigurableBuilder, missingStoreConfigs, "keystore not found"); + verifyInvalidReconfigure(reconfigurableBuilder, missingStoreConfigs); // Verify that new connections continue to work with the server with previously configured keystore after failed reconfiguration newClientSelector.connect("3", addr, BUFFER_SIZE, BUFFER_SIZE); @@ -1167,7 +1167,7 @@ false, securityProtocol, config, null, null, TIME, new LogContext(), for (String propName : CertStores.KEYSTORE_PROPS) { invalidKeystoreConfigs.put(propName, invalidConfig.get(propName)); } - verifyInvalidReconfigure(reconfigurableBuilder, invalidKeystoreConfigs, "keystore without existing SubjectAltName"); + verifyInvalidReconfigure(reconfigurableBuilder, invalidKeystoreConfigs); String node3 = "3"; selector.connect(node3, addr, BUFFER_SIZE, BUFFER_SIZE); NetworkTestUtils.checkClientConnection(selector, node3, 100, 10); @@ -1223,13 +1223,13 @@ false, securityProtocol, config, null, null, TIME, new LogContext(), Map invalidConfigs = new HashMap<>(newTruststoreConfigs); invalidConfigs.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, "INVALID_TYPE"); - verifyInvalidReconfigure(reconfigurableBuilder, invalidConfigs, "invalid truststore type"); + verifyInvalidReconfigure(reconfigurableBuilder, invalidConfigs); Map missingStoreConfigs = new HashMap<>(); missingStoreConfigs.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, "PKCS12"); missingStoreConfigs.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "some.truststore.path"); missingStoreConfigs.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, new Password("some.truststore.password")); - verifyInvalidReconfigure(reconfigurableBuilder, missingStoreConfigs, "truststore not found"); + verifyInvalidReconfigure(reconfigurableBuilder, missingStoreConfigs); // Verify that new connections continue to work with the server with previously configured keystore after failed reconfiguration newClientSelector.connect("3", addr, BUFFER_SIZE, BUFFER_SIZE); @@ -1280,7 +1280,7 @@ public void testInvalidSslEngineFactory(Args args) { } private void verifyInvalidReconfigure(ListenerReconfigurable reconfigurable, - Map invalidConfigs, String errorMessage) { + Map invalidConfigs) { assertThrows(KafkaException.class, () -> reconfigurable.validateReconfiguration(invalidConfigs)); assertThrows(KafkaException.class, () -> reconfigurable.reconfigure(invalidConfigs)); } diff --git a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java index a2e89d3f4c669..75babcf95b017 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java @@ -142,7 +142,7 @@ public void testSliceSizeLimitWithConcurrentWrite() throws Exception { Future readerCompletion = executor.submit(() -> { while (log.sizeInBytes() < maxSizeInBytes) { int currentSize = log.sizeInBytes(); - FileRecords slice = log.slice(0, currentSize); + Records slice = log.slice(0, currentSize); assertEquals(currentSize, slice.sizeInBytes()); } return null; @@ -198,9 +198,9 @@ public void testIterationDoesntChangePosition() throws IOException { */ @Test public void testRead() throws IOException { - FileRecords read = fileRecords.slice(0, fileRecords.sizeInBytes()); + Records read = fileRecords.slice(0, fileRecords.sizeInBytes()); assertEquals(fileRecords.sizeInBytes(), read.sizeInBytes()); - TestUtils.checkEquals(fileRecords.batches(), read.batches()); + TestUtils.checkEquals(fileRecords.batches(), ((FileRecords) read).batches()); List items = batches(read); RecordBatch first = items.get(0); @@ -283,9 +283,9 @@ public void testIteratorWithLimits() throws IOException { RecordBatch batch = batches(fileRecords).get(1); int start = fileRecords.searchForOffsetFromPosition(1, 0).position; int size = batch.sizeInBytes(); - FileRecords slice = fileRecords.slice(start, size); + Records slice = fileRecords.slice(start, size); assertEquals(Collections.singletonList(batch), batches(slice)); - FileRecords slice2 = fileRecords.slice(start, size - 1); + Records slice2 = fileRecords.slice(start, size - 1); assertEquals(Collections.emptyList(), batches(slice2)); } @@ -429,24 +429,22 @@ public void testSliceForAlreadySlicedFileRecords() throws IOException { "abcd".getBytes(), "efgh".getBytes(), "ijkl".getBytes(), - "mnop".getBytes(), - "qrst".getBytes() + "mnopqr".getBytes(), + "stuv".getBytes() }; try (FileRecords fileRecords = createFileRecords(values)) { List items = batches(fileRecords.slice(0, fileRecords.sizeInBytes())); // Slice from fourth message until the end. int position = IntStream.range(0, 3).map(i -> items.get(i).sizeInBytes()).sum(); - FileRecords sliced = fileRecords.slice(position, fileRecords.sizeInBytes() - position); + Records sliced = fileRecords.slice(position, fileRecords.sizeInBytes() - position); assertEquals(fileRecords.sizeInBytes() - position, sliced.sizeInBytes()); assertEquals(items.subList(3, items.size()), batches(sliced), "Read starting from the fourth message"); // Further slice the already sliced file records, from fifth message until the end. Now the - // bytes available in the sliced file records are less than the start position. However, the - // position to slice is relative hence reset position to second message in the sliced file - // records i.e. reset with the size of the fourth message from the original file records. - position = items.get(4).sizeInBytes(); - FileRecords finalSliced = sliced.slice(position, sliced.sizeInBytes() - position); + // bytes available in the sliced records are less than the moved position from original records. + position = items.get(3).sizeInBytes(); + Records finalSliced = sliced.slice(position, sliced.sizeInBytes() - position); assertEquals(sliced.sizeInBytes() - position, finalSliced.sizeInBytes()); assertEquals(items.subList(4, items.size()), batches(finalSliced), "Read starting from the fifth message"); } diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java index 84d917fa39d28..f102bd5fd9534 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.ByteBufferOutputStream; import org.apache.kafka.common.utils.CloseableIterator; -import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; @@ -50,7 +49,6 @@ import static org.apache.kafka.common.record.RecordBatch.MAGIC_VALUE_V2; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -91,8 +89,6 @@ public Stream provideArguments(ExtensionContext context) { } } - private final Time time = Time.SYSTEM; - @Test public void testUnsupportedCompress() { BiFunction builderBiFunction = (magic, compression) -> @@ -638,31 +634,6 @@ public Stream provideArguments(ExtensionContext context) { } } - private void verifyRecordsProcessingStats(Compression compression, RecordValidationStats processingStats, - int numRecords, int numRecordsConverted, long finalBytes, - long preConvertedBytes) { - assertNotNull(processingStats, "Records processing info is null"); - assertEquals(numRecordsConverted, processingStats.numRecordsConverted()); - // Since nanoTime accuracy on build machines may not be sufficient to measure small conversion times, - // only check if the value >= 0. Default is -1, so this checks if time has been recorded. - assertTrue(processingStats.conversionTimeNanos() >= 0, "Processing time not recorded: " + processingStats); - long tempBytes = processingStats.temporaryMemoryBytes(); - if (compression.type() == CompressionType.NONE) { - if (numRecordsConverted == 0) - assertEquals(finalBytes, tempBytes); - else if (numRecordsConverted == numRecords) - assertEquals(preConvertedBytes + finalBytes, tempBytes); - else { - assertTrue(tempBytes > finalBytes && tempBytes < finalBytes + preConvertedBytes, - String.format("Unexpected temp bytes %d final %d pre %d", tempBytes, finalBytes, preConvertedBytes)); - } - } else { - long compressedBytes = finalBytes - Records.LOG_OVERHEAD - LegacyRecord.RECORD_OVERHEAD_V0; - assertTrue(tempBytes > compressedBytes, - String.format("Uncompressed size expected temp=%d, compressed=%d", tempBytes, compressedBytes)); - } - } - private ByteBuffer allocateBuffer(int size, Args args) { ByteBuffer buffer = ByteBuffer.allocate(size); buffer.position(args.bufferOffset); diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index 3818976e423fd..6f98da21eed86 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -35,14 +35,19 @@ import org.junit.jupiter.params.provider.ArgumentsProvider; import org.junit.jupiter.params.provider.ArgumentsSource; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.OptionalLong; import java.util.function.BiFunction; import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import java.util.stream.Stream; import static java.util.Arrays.asList; @@ -1068,6 +1073,146 @@ public void testUnsupportedCompress() { }); } + @ParameterizedTest + @ArgumentsSource(MemoryRecordsArgumentsProvider.class) + public void testSlice(Args args) throws IOException { + // Create a MemoryRecords instance with multiple batches. Prior RecordBatch.MAGIC_VALUE_V2, + // every append in a batch is a new batch. After RecordBatch.MAGIC_VALUE_V2, we can have multiple + // batches in a single MemoryRecords instance. Though with compression, we can have multiple + // appends resulting in a single batch prior RecordBatch.MAGIC_VALUE_V2 as well. + LinkedHashMap recordsPerOffset = new LinkedHashMap<>(); + recordsPerOffset.put(args.firstOffset, 3); + recordsPerOffset.put(args.firstOffset + 6L, 8); + recordsPerOffset.put(args.firstOffset + 15L, 4); + MemoryRecords records = createMemoryRecords(args, recordsPerOffset); + + // Test slicing from start + Records sliced = records.slice(0, records.sizeInBytes()); + assertEquals(records.sizeInBytes(), sliced.sizeInBytes()); + assertEquals(records.validBytes(), ((MemoryRecords) sliced).validBytes()); + TestUtils.checkEquals(records.batches(), ((MemoryRecords) sliced).batches()); + + List items = batches(records); + // Test slicing first message. + RecordBatch first = items.get(0); + sliced = records.slice(first.sizeInBytes(), records.sizeInBytes() - first.sizeInBytes()); + assertEquals(records.sizeInBytes() - first.sizeInBytes(), sliced.sizeInBytes()); + assertEquals(items.subList(1, items.size()), batches(sliced), "Read starting from the second message"); + assertTrue(((MemoryRecords) sliced).validBytes() <= sliced.sizeInBytes()); + + // Read from second message and size is past the end of the file. + sliced = records.slice(first.sizeInBytes(), records.sizeInBytes()); + assertEquals(records.sizeInBytes() - first.sizeInBytes(), sliced.sizeInBytes()); + assertEquals(items.subList(1, items.size()), batches(sliced), "Read starting from the second message"); + assertTrue(((MemoryRecords) sliced).validBytes() <= sliced.sizeInBytes()); + + // Read from second message and position + size overflows. + sliced = records.slice(first.sizeInBytes(), Integer.MAX_VALUE); + assertEquals(records.sizeInBytes() - first.sizeInBytes(), sliced.sizeInBytes()); + assertEquals(items.subList(1, items.size()), batches(sliced), "Read starting from the second message"); + assertTrue(((MemoryRecords) sliced).validBytes() <= sliced.sizeInBytes()); + + // Read a single message starting from second message. + RecordBatch second = items.get(1); + sliced = records.slice(first.sizeInBytes(), second.sizeInBytes()); + assertEquals(second.sizeInBytes(), sliced.sizeInBytes()); + assertEquals(Collections.singletonList(second), batches(sliced), "Read a single message starting from the second message"); + + // Read from already sliced view. + List remainingItems = IntStream.range(0, items.size()).filter(i -> i != 0 && i != 1).mapToObj(items::get).collect(Collectors.toList()); + int remainingSize = remainingItems.stream().mapToInt(RecordBatch::sizeInBytes).sum(); + sliced = records.slice(first.sizeInBytes(), records.sizeInBytes() - first.sizeInBytes()) + .slice(second.sizeInBytes(), records.sizeInBytes() - first.sizeInBytes() - second.sizeInBytes()); + assertEquals(remainingSize, sliced.sizeInBytes()); + assertEquals(remainingItems, batches(sliced), "Read starting from the third message"); + + // Read from second message and size is past the end of the file on the already sliced view. + sliced = records.slice(1, records.sizeInBytes() - 1) + .slice(first.sizeInBytes() - 1, records.sizeInBytes()); + assertEquals(records.sizeInBytes() - first.sizeInBytes(), sliced.sizeInBytes()); + assertEquals(items.subList(1, items.size()), batches(sliced), "Read starting from the second message"); + assertTrue(((MemoryRecords) sliced).validBytes() <= sliced.sizeInBytes()); + + // Read from second message and position + size overflows on the already sliced view. + sliced = records.slice(1, records.sizeInBytes() - 1) + .slice(first.sizeInBytes() - 1, Integer.MAX_VALUE); + assertEquals(records.sizeInBytes() - first.sizeInBytes(), sliced.sizeInBytes()); + assertEquals(items.subList(1, items.size()), batches(sliced), "Read starting from the second message"); + assertTrue(((MemoryRecords) sliced).validBytes() <= sliced.sizeInBytes()); + } + + @ParameterizedTest + @ArgumentsSource(MemoryRecordsArgumentsProvider.class) + public void testSliceInvalidPosition(Args args) { + MemoryRecords records = createMemoryRecords(args, Map.of(args.firstOffset, 1)); + assertThrows(IllegalArgumentException.class, () -> records.slice(-1, records.sizeInBytes())); + assertThrows(IllegalArgumentException.class, () -> records.slice(records.sizeInBytes() + 1, records.sizeInBytes())); + } + + @ParameterizedTest + @ArgumentsSource(MemoryRecordsArgumentsProvider.class) + public void testSliceInvalidSize(Args args) { + MemoryRecords records = createMemoryRecords(args, Map.of(args.firstOffset, 1)); + assertThrows(IllegalArgumentException.class, () -> records.slice(0, -1)); + } + + @Test + public void testSliceEmptyRecords() { + MemoryRecords empty = MemoryRecords.EMPTY; + Records sliced = empty.slice(0, 0); + assertEquals(0, sliced.sizeInBytes()); + assertEquals(0, batches(sliced).size()); + } + + /** + * Test slice when already sliced memory records have start position greater than available bytes + * in the memory records. + */ + @ParameterizedTest + @ArgumentsSource(MemoryRecordsArgumentsProvider.class) + public void testSliceForAlreadySlicedMemoryRecords(Args args) throws IOException { + LinkedHashMap recordsPerOffset = new LinkedHashMap<>(); + recordsPerOffset.put(args.firstOffset, 5); + recordsPerOffset.put(args.firstOffset + 5L, 10); + recordsPerOffset.put(args.firstOffset + 15L, 12); + recordsPerOffset.put(args.firstOffset + 27L, 4); + + MemoryRecords records = createMemoryRecords(args, recordsPerOffset); + List items = batches(records.slice(0, records.sizeInBytes())); + + // Slice from third message until the end. + int position = IntStream.range(0, 2).map(i -> items.get(i).sizeInBytes()).sum(); + Records sliced = records.slice(position, records.sizeInBytes() - position); + assertEquals(records.sizeInBytes() - position, sliced.sizeInBytes()); + assertEquals(items.subList(2, items.size()), batches(sliced), "Read starting from the third message"); + + // Further slice the already sliced memory records, from fourth message until the end. Now the + // bytes available in the sliced records are less than the moved position from original records. + position = items.get(2).sizeInBytes(); + Records finalSliced = sliced.slice(position, sliced.sizeInBytes() - position); + assertEquals(sliced.sizeInBytes() - position, finalSliced.sizeInBytes()); + assertEquals(items.subList(3, items.size()), batches(finalSliced), "Read starting from the fourth message"); + } + + private MemoryRecords createMemoryRecords(Args args, Map recordsPerOffset) { + ByteBuffer buffer = ByteBuffer.allocate(1024); + recordsPerOffset.forEach((offset, numOfRecords) -> { + MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, args.magic, args.compression, + TimestampType.CREATE_TIME, offset); + for (int i = 0; i < numOfRecords; i++) { + builder.appendWithOffset(offset + i, 0L, TestUtils.randomString(10).getBytes(), TestUtils.randomString(10).getBytes()); + } + builder.close(); + }); + buffer.flip(); + + return MemoryRecords.readableRecords(buffer); + } + + private static List batches(Records buffer) { + return TestUtils.toList(buffer.batches()); + } + private static class RetainNonNullKeysFilter extends MemoryRecords.RecordFilter { public RetainNonNullKeysFilter() { super(0, 0); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/CreateAclsRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/CreateAclsRequestTest.java index 4e6aa4f87e3a1..85f35e683ec8a 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/CreateAclsRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/CreateAclsRequestTest.java @@ -44,9 +44,6 @@ public class CreateAclsRequestTest { private static final AclBinding LITERAL_ACL1 = new AclBinding(new ResourcePattern(ResourceType.TOPIC, "foo", PatternType.LITERAL), new AccessControlEntry("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY)); - private static final AclBinding LITERAL_ACL2 = new AclBinding(new ResourcePattern(ResourceType.GROUP, "group", PatternType.LITERAL), - new AccessControlEntry("User:*", "127.0.0.1", AclOperation.WRITE, AclPermissionType.ALLOW)); - private static final AclBinding PREFIXED_ACL1 = new AclBinding(new ResourcePattern(ResourceType.GROUP, "prefix", PatternType.PREFIXED), new AccessControlEntry("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW)); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsRequestTest.java index e09c1eee72155..5062677e12300 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsRequestTest.java @@ -33,9 +33,6 @@ public class DescribeAclsRequestTest { private static final short V1 = 1; - private static final AclBindingFilter LITERAL_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.TOPIC, "foo", PatternType.LITERAL), - new AccessControlEntryFilter("User:ANONYMOUS", "127.0.0.1", AclOperation.READ, AclPermissionType.DENY)); - private static final AclBindingFilter PREFIXED_FILTER = new AclBindingFilter(new ResourcePatternFilter(ResourceType.GROUP, "prefix", PatternType.PREFIXED), new AccessControlEntryFilter("User:*", "127.0.0.1", AclOperation.CREATE, AclPermissionType.ALLOW)); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java index f2925b45828f5..c4ec20385e102 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java @@ -74,12 +74,6 @@ public class DescribeAclsResponseTest { PatternType.LITERAL, Collections.singletonList(ALLOW_CREATE_ACL)); - private static final DescribeAclsResource LITERAL_ACL2 = buildResource( - "group", - ResourceType.GROUP, - PatternType.LITERAL, - Collections.singletonList(DENY_READ_ACL)); - @Test public void shouldThrowIfUnknown() { assertThrows(IllegalArgumentException.class, diff --git a/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java index 2d8c6ef43f94a..23b5258a2352e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java @@ -16,426 +16,231 @@ */ package org.apache.kafka.common.requests; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.OffsetFetchResponseData; -import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseGroup; -import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponsePartition; -import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponsePartitions; -import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopic; -import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopics; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.record.RecordBatch; -import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; -import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; +import java.util.List; -import static org.apache.kafka.common.requests.AbstractResponse.DEFAULT_THROTTLE_TIME; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertThrows; public class OffsetFetchResponseTest { - private final int throttleTimeMs = 10; - private final int offset = 100; - private final String metadata = "metadata"; - - private final String groupOne = "group1"; - private final String groupTwo = "group2"; - private final String groupThree = "group3"; - private final String topicOne = "topic1"; - private final int partitionOne = 1; - private final Optional leaderEpochOne = Optional.of(1); - private final String topicTwo = "topic2"; - private final int partitionTwo = 2; - private final Optional leaderEpochTwo = Optional.of(2); - private final String topicThree = "topic3"; - private final int partitionThree = 3; - private final Optional leaderEpochThree = Optional.of(3); - - - private Map partitionDataMap; - - @BeforeEach - public void setUp() { - partitionDataMap = new HashMap<>(); - partitionDataMap.put(new TopicPartition(topicOne, partitionOne), new PartitionData( - offset, - leaderEpochOne, - metadata, - Errors.TOPIC_AUTHORIZATION_FAILED - )); - partitionDataMap.put(new TopicPartition(topicTwo, partitionTwo), new PartitionData( - offset, - leaderEpochTwo, - metadata, - Errors.UNKNOWN_TOPIC_OR_PARTITION - )); - } - - @Test - public void testConstructor() { - for (short version : ApiKeys.OFFSET_FETCH.allVersions()) { - if (version < 8) { - OffsetFetchResponse response = new OffsetFetchResponse(throttleTimeMs, Errors.NOT_COORDINATOR, partitionDataMap); - assertEquals(Errors.NOT_COORDINATOR, response.error()); - assertEquals(3, response.errorCounts().size()); - assertEquals(Utils.mkMap(Utils.mkEntry(Errors.NOT_COORDINATOR, 1), - Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1), - Utils.mkEntry(Errors.UNKNOWN_TOPIC_OR_PARTITION, 1)), - response.errorCounts()); - - assertEquals(throttleTimeMs, response.throttleTimeMs()); - - Map responseData = response.responseDataV0ToV7(); - assertEquals(partitionDataMap, responseData); - responseData.forEach((tp, data) -> assertTrue(data.hasError())); - } else { - OffsetFetchResponse response = new OffsetFetchResponse( - throttleTimeMs, - Collections.singletonMap(groupOne, Errors.NOT_COORDINATOR), - Collections.singletonMap(groupOne, partitionDataMap)); - assertEquals(Errors.NOT_COORDINATOR, response.groupLevelError(groupOne)); - assertEquals(3, response.errorCounts().size()); - assertEquals(Utils.mkMap(Utils.mkEntry(Errors.NOT_COORDINATOR, 1), - Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1), - Utils.mkEntry(Errors.UNKNOWN_TOPIC_OR_PARTITION, 1)), - response.errorCounts()); - - assertEquals(throttleTimeMs, response.throttleTimeMs()); - - Map responseData = response.partitionDataMap(groupOne); - assertEquals(partitionDataMap, responseData); - responseData.forEach((tp, data) -> assertTrue(data.hasError())); - } - } - } - - @Test - public void testConstructorWithMultipleGroups() { - Map> responseData = new HashMap<>(); - Map errorMap = new HashMap<>(); - Map pd1 = new HashMap<>(); - Map pd2 = new HashMap<>(); - Map pd3 = new HashMap<>(); - pd1.put(new TopicPartition(topicOne, partitionOne), new PartitionData( - offset, - leaderEpochOne, - metadata, - Errors.TOPIC_AUTHORIZATION_FAILED)); - pd2.put(new TopicPartition(topicTwo, partitionTwo), new PartitionData( - offset, - leaderEpochTwo, - metadata, - Errors.UNKNOWN_TOPIC_OR_PARTITION)); - pd3.put(new TopicPartition(topicThree, partitionThree), new PartitionData( - offset, - leaderEpochThree, - metadata, - Errors.NONE)); - responseData.put(groupOne, pd1); - responseData.put(groupTwo, pd2); - responseData.put(groupThree, pd3); - errorMap.put(groupOne, Errors.NOT_COORDINATOR); - errorMap.put(groupTwo, Errors.COORDINATOR_LOAD_IN_PROGRESS); - errorMap.put(groupThree, Errors.NONE); - for (short version : ApiKeys.OFFSET_FETCH.allVersions()) { - if (version >= 8) { - OffsetFetchResponse response = new OffsetFetchResponse( - throttleTimeMs, errorMap, responseData); - - assertEquals(Errors.NOT_COORDINATOR, response.groupLevelError(groupOne)); - assertEquals(Errors.COORDINATOR_LOAD_IN_PROGRESS, response.groupLevelError(groupTwo)); - assertEquals(Errors.NONE, response.groupLevelError(groupThree)); - assertTrue(response.groupHasError(groupOne)); - assertTrue(response.groupHasError(groupTwo)); - assertFalse(response.groupHasError(groupThree)); - assertEquals(5, response.errorCounts().size()); - assertEquals(Utils.mkMap(Utils.mkEntry(Errors.NOT_COORDINATOR, 1), - Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1), - Utils.mkEntry(Errors.UNKNOWN_TOPIC_OR_PARTITION, 1), - Utils.mkEntry(Errors.COORDINATOR_LOAD_IN_PROGRESS, 1), - Utils.mkEntry(Errors.NONE, 2)), - response.errorCounts()); - - assertEquals(throttleTimeMs, response.throttleTimeMs()); - - Map responseData1 = response.partitionDataMap(groupOne); - assertEquals(pd1, responseData1); - responseData1.forEach((tp, data) -> assertTrue(data.hasError())); - Map responseData2 = response.partitionDataMap(groupTwo); - assertEquals(pd2, responseData2); - responseData2.forEach((tp, data) -> assertTrue(data.hasError())); - Map responseData3 = response.partitionDataMap(groupThree); - assertEquals(pd3, responseData3); - responseData3.forEach((tp, data) -> assertFalse(data.hasError())); - } + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_FETCH) + public void testBuilderWithSingleGroup(short version) { + var group = new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId("group") + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName("foo") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(5) + .setMetadata("metadata") + )) + )); + + if (version < 8) { + assertEquals( + new OffsetFetchResponseData() + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopic() + .setName("foo") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartition() + .setPartitionIndex(0) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(5) + .setMetadata("metadata") + )) + )), + new OffsetFetchResponse.Builder(group).build(version).data() + ); + } else { + assertEquals( + new OffsetFetchResponseData() + .setGroups(List.of(group)), + new OffsetFetchResponse.Builder(group).build(version).data() + ); } } - /** - * Test behavior changes over the versions. Refer to resources.common.messages.OffsetFetchResponse.json - */ - @Test - public void testStructBuild() { - for (short version : ApiKeys.OFFSET_FETCH.allVersions()) { - if (version < 8) { - partitionDataMap.put(new TopicPartition(topicTwo, partitionTwo), new PartitionData( - offset, - leaderEpochTwo, - metadata, - Errors.GROUP_AUTHORIZATION_FAILED - )); - - OffsetFetchResponse latestResponse = new OffsetFetchResponse(throttleTimeMs, Errors.NONE, partitionDataMap); - OffsetFetchResponseData data = new OffsetFetchResponseData( - latestResponse.serialize(version), version); - - OffsetFetchResponse oldResponse = new OffsetFetchResponse(data, version); - - if (version <= 1) { - assertEquals(Errors.NONE.code(), data.errorCode()); - - // Partition level error populated in older versions. - assertEquals(Errors.GROUP_AUTHORIZATION_FAILED, oldResponse.error()); - assertEquals(Utils.mkMap(Utils.mkEntry(Errors.GROUP_AUTHORIZATION_FAILED, 2), - Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1)), - oldResponse.errorCounts()); - } else { - assertEquals(Errors.NONE.code(), data.errorCode()); - - assertEquals(Errors.NONE, oldResponse.error()); - assertEquals(Utils.mkMap( - Utils.mkEntry(Errors.NONE, 1), - Utils.mkEntry(Errors.GROUP_AUTHORIZATION_FAILED, 1), - Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1)), - oldResponse.errorCounts()); - } - - if (version <= 2) { - assertEquals(DEFAULT_THROTTLE_TIME, oldResponse.throttleTimeMs()); - } else { - assertEquals(throttleTimeMs, oldResponse.throttleTimeMs()); - } - - Map expectedDataMap = new HashMap<>(); - for (Map.Entry entry : partitionDataMap.entrySet()) { - PartitionData partitionData = entry.getValue(); - expectedDataMap.put(entry.getKey(), new PartitionData( - partitionData.offset, - version <= 4 ? Optional.empty() : partitionData.leaderEpoch, - partitionData.metadata, - partitionData.error - )); - } - - Map responseData = oldResponse.responseDataV0ToV7(); - assertEquals(expectedDataMap, responseData); - - responseData.forEach((tp, rdata) -> assertTrue(rdata.hasError())); - } else { - partitionDataMap.put(new TopicPartition(topicTwo, partitionTwo), new PartitionData( - offset, - leaderEpochTwo, - metadata, - Errors.GROUP_AUTHORIZATION_FAILED)); - OffsetFetchResponse latestResponse = new OffsetFetchResponse( - throttleTimeMs, - Collections.singletonMap(groupOne, Errors.NONE), - Collections.singletonMap(groupOne, partitionDataMap)); - OffsetFetchResponseData data = new OffsetFetchResponseData( - latestResponse.serialize(version), version); - OffsetFetchResponse oldResponse = new OffsetFetchResponse(data, version); - assertEquals(Errors.NONE.code(), data.groups().get(0).errorCode()); - - assertEquals(Errors.NONE, oldResponse.groupLevelError(groupOne)); - assertEquals(Utils.mkMap( - Utils.mkEntry(Errors.NONE, 1), - Utils.mkEntry(Errors.GROUP_AUTHORIZATION_FAILED, 1), - Utils.mkEntry(Errors.TOPIC_AUTHORIZATION_FAILED, 1)), - oldResponse.errorCounts()); - assertEquals(throttleTimeMs, oldResponse.throttleTimeMs()); - - Map expectedDataMap = new HashMap<>(); - for (Map.Entry entry : partitionDataMap.entrySet()) { - PartitionData partitionData = entry.getValue(); - expectedDataMap.put(entry.getKey(), new PartitionData( - partitionData.offset, - partitionData.leaderEpoch, - partitionData.metadata, - partitionData.error - )); - } - - Map responseData = oldResponse.partitionDataMap(groupOne); - assertEquals(expectedDataMap, responseData); + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_FETCH) + public void testBuilderWithMultipleGroups(short version) { + var groups = List.of( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId("group1") + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName("foo") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(5) + .setMetadata("metadata") + )) + )), + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId("group2") + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName("bar") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(5) + .setMetadata("metadata") + )) + )) + ); - responseData.forEach((tp, rdata) -> assertTrue(rdata.hasError())); - } + if (version < 8) { + assertThrows(UnsupportedVersionException.class, + () -> new OffsetFetchResponse.Builder(groups).build(version)); + } else { + assertEquals( + new OffsetFetchResponseData() + .setGroups(groups), + new OffsetFetchResponse.Builder(groups).build(version).data() + ); } } - @Test - public void testShouldThrottle() { - for (short version : ApiKeys.OFFSET_FETCH.allVersions()) { - if (version < 8) { - OffsetFetchResponse response = new OffsetFetchResponse(throttleTimeMs, Errors.NONE, partitionDataMap); - if (version >= 4) { - assertTrue(response.shouldClientThrottle(version)); - } else { - assertFalse(response.shouldClientThrottle(version)); - } - } else { - OffsetFetchResponse response = new OffsetFetchResponse( - throttleTimeMs, - Collections.singletonMap(groupOne, Errors.NOT_COORDINATOR), - Collections.singletonMap(groupOne, partitionDataMap)); - assertTrue(response.shouldClientThrottle(version)); - } + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_FETCH) + public void testGroupWithSingleGroup(short version) { + var data = new OffsetFetchResponseData(); + + if (version < 8) { + data.setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopic() + .setName("foo") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartition() + .setPartitionIndex(0) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(5) + .setMetadata("metadata") + )) + )); + } else { + data.setGroups(List.of( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId("foo") + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName("foo") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(5) + .setMetadata("metadata") + )) + )) + )); } - } - - @Test - public void testNullableMetadataV0ToV7() { - PartitionData pd = new PartitionData( - offset, - leaderEpochOne, - null, - Errors.UNKNOWN_TOPIC_OR_PARTITION); - // test PartitionData.equals with null metadata - assertEquals(pd, pd); - partitionDataMap.clear(); - partitionDataMap.put(new TopicPartition(topicOne, partitionOne), pd); - OffsetFetchResponse response = new OffsetFetchResponse(throttleTimeMs, Errors.GROUP_AUTHORIZATION_FAILED, partitionDataMap); - OffsetFetchResponseData expectedData = - new OffsetFetchResponseData() - .setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code()) - .setThrottleTimeMs(throttleTimeMs) - .setTopics(Collections.singletonList( - new OffsetFetchResponseTopic() - .setName(topicOne) - .setPartitions(Collections.singletonList( - new OffsetFetchResponsePartition() - .setPartitionIndex(partitionOne) - .setCommittedOffset(offset) - .setCommittedLeaderEpoch(leaderEpochOne.orElse(-1)) - .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) - .setMetadata(null)) + assertEquals( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId("foo") + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName("foo") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(5) + .setMetadata("metadata") )) - ); - assertEquals(expectedData, response.data()); - } - - @Test - public void testNullableMetadataV8AndAbove() { - PartitionData pd = new PartitionData( - offset, - leaderEpochOne, - null, - Errors.UNKNOWN_TOPIC_OR_PARTITION); - // test PartitionData.equals with null metadata - assertEquals(pd, pd); - partitionDataMap.clear(); - partitionDataMap.put(new TopicPartition(topicOne, partitionOne), pd); - - OffsetFetchResponse response = new OffsetFetchResponse( - throttleTimeMs, - Collections.singletonMap(groupOne, Errors.GROUP_AUTHORIZATION_FAILED), - Collections.singletonMap(groupOne, partitionDataMap)); - OffsetFetchResponseData expectedData = - new OffsetFetchResponseData() - .setGroups(Collections.singletonList( - new OffsetFetchResponseGroup() - .setGroupId(groupOne) - .setTopics(Collections.singletonList( - new OffsetFetchResponseTopics() - .setName(topicOne) - .setPartitions(Collections.singletonList( - new OffsetFetchResponsePartitions() - .setPartitionIndex(partitionOne) - .setCommittedOffset(offset) - .setCommittedLeaderEpoch(leaderEpochOne.orElse(-1)) - .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) - .setMetadata(null))))) - .setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code()))) - .setThrottleTimeMs(throttleTimeMs); - assertEquals(expectedData, response.data()); + )), + new OffsetFetchResponse(data, version).group("foo") + ); } - @Test - public void testUseDefaultLeaderEpochV0ToV7() { - final Optional emptyLeaderEpoch = Optional.empty(); - partitionDataMap.clear(); + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_FETCH, fromVersion = 8) + public void testGroupWithMultipleGroups(short version) { + var groups = List.of( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId("group1") + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName("foo") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(5) + .setMetadata("metadata") + )) + )), + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId("group2") + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName("bar") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(10) + .setCommittedLeaderEpoch(5) + .setMetadata("metadata") + )) + )) + ); - partitionDataMap.put(new TopicPartition(topicOne, partitionOne), - new PartitionData( - offset, - emptyLeaderEpoch, - metadata, - Errors.UNKNOWN_TOPIC_OR_PARTITION) + var response = new OffsetFetchResponse( + new OffsetFetchResponseData().setGroups(groups), + version ); - OffsetFetchResponse response = new OffsetFetchResponse(throttleTimeMs, Errors.NOT_COORDINATOR, partitionDataMap); - OffsetFetchResponseData expectedData = - new OffsetFetchResponseData() - .setErrorCode(Errors.NOT_COORDINATOR.code()) - .setThrottleTimeMs(throttleTimeMs) - .setTopics(Collections.singletonList( - new OffsetFetchResponseTopic() - .setName(topicOne) - .setPartitions(Collections.singletonList( - new OffsetFetchResponsePartition() - .setPartitionIndex(partitionOne) - .setCommittedOffset(offset) - .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) - .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) - .setMetadata(metadata)) - )) - ); - assertEquals(expectedData, response.data()); + groups.forEach(group -> + assertEquals(group, response.group(group.groupId())) + ); } - @Test - public void testUseDefaultLeaderEpochV8() { - final Optional emptyLeaderEpoch = Optional.empty(); - partitionDataMap.clear(); + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_FETCH) + public void testGroupWithSingleGroupWithTopLevelError(short version) { + var data = new OffsetFetchResponseData(); + + if (version < 2) { + data.setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopic() + .setName("foo") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartition() + .setPartitionIndex(0) + .setErrorCode(Errors.INVALID_GROUP_ID.code()) + )) + )); + } else if (version < 8) { + data.setErrorCode(Errors.INVALID_GROUP_ID.code()); + } else { + data.setGroups(List.of( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId("foo") + .setErrorCode(Errors.INVALID_GROUP_ID.code()) + )); + } - partitionDataMap.put(new TopicPartition(topicOne, partitionOne), - new PartitionData( - offset, - emptyLeaderEpoch, - metadata, - Errors.UNKNOWN_TOPIC_OR_PARTITION) + assertEquals( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId("foo") + .setErrorCode(Errors.INVALID_GROUP_ID.code()), + new OffsetFetchResponse(data, version).group("foo") ); - OffsetFetchResponse response = new OffsetFetchResponse( - throttleTimeMs, - Collections.singletonMap(groupOne, Errors.NOT_COORDINATOR), - Collections.singletonMap(groupOne, partitionDataMap)); - OffsetFetchResponseData expectedData = - new OffsetFetchResponseData() - .setGroups(Collections.singletonList( - new OffsetFetchResponseGroup() - .setGroupId(groupOne) - .setTopics(Collections.singletonList( - new OffsetFetchResponseTopics() - .setName(topicOne) - .setPartitions(Collections.singletonList( - new OffsetFetchResponsePartitions() - .setPartitionIndex(partitionOne) - .setCommittedOffset(offset) - .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) - .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) - .setMetadata(metadata))))) - .setErrorCode(Errors.NOT_COORDINATOR.code()))) - .setThrottleTimeMs(throttleTimeMs); - assertEquals(expectedData, response.data()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 6bbaa335771ed..609e191675a41 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -200,6 +200,7 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; import org.apache.kafka.common.message.OffsetFetchRequestData; +import org.apache.kafka.common.message.OffsetFetchResponseData; import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition; import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopic; import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderTopicCollection; @@ -1124,7 +1125,7 @@ private AbstractResponse getResponse(ApiKeys apikey, short version) { case ALTER_CLIENT_QUOTAS: return createAlterClientQuotasResponse(); case DESCRIBE_USER_SCRAM_CREDENTIALS: return createDescribeUserScramCredentialsResponse(); case ALTER_USER_SCRAM_CREDENTIALS: return createAlterUserScramCredentialsResponse(); - case VOTE: return createVoteResponse(version); + case VOTE: return createVoteResponse(); case BEGIN_QUORUM_EPOCH: return createBeginQuorumEpochResponse(); case END_QUORUM_EPOCH: return createEndQuorumEpochResponse(); case DESCRIBE_QUORUM: return createDescribeQuorumResponse(); @@ -1187,7 +1188,7 @@ private ConsumerGroupDescribeResponse createConsumerGroupDescribeResponse() { .setGroupEpoch(0) .setAssignmentEpoch(0) .setAssignorName("range") - .setMembers(new ArrayList(0)) + .setMembers(new ArrayList<>(0)) )) .setThrottleTimeMs(1000); return new ConsumerGroupDescribeResponse(data); @@ -1660,7 +1661,7 @@ private VoteRequest createVoteRequest(short version) { return new VoteRequest.Builder(data).build(version); } - private VoteResponse createVoteResponse(short version) { + private VoteResponse createVoteResponse() { VoteResponseData.PartitionData partitionData = new VoteResponseData.PartitionData() .setErrorCode(Errors.NONE.code()) .setLeaderEpoch(0) @@ -2483,17 +2484,23 @@ private OffsetFetchRequest createOffsetFetchRequestForAllPartition(short version } private OffsetFetchResponse createOffsetFetchResponse(short version) { - Map responseData = new HashMap<>(); - responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData( - 100L, Optional.empty(), "", Errors.NONE)); - responseData.put(new TopicPartition("test", 1), new OffsetFetchResponse.PartitionData( - 100L, Optional.of(10), null, Errors.NONE)); - if (version < 8) { - return new OffsetFetchResponse(Errors.NONE, responseData); - } - int throttleMs = 10; - return new OffsetFetchResponse(throttleMs, Collections.singletonMap("group1", Errors.NONE), - Collections.singletonMap("group1", responseData)); + var group = new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId("group1") + .setTopics(List.of( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName("test") + .setPartitions(List.of( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(100), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(1) + .setCommittedOffset(100) + .setCommittedLeaderEpoch(10) + .setMetadata(null) + )) + )); + return new OffsetFetchResponse.Builder(group).build(version); } private ProduceRequest createProduceRequest(short version) { diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java index f9734dabb7a8c..fd15fe6ff67d6 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java @@ -128,7 +128,7 @@ public void testUnexpectedRequestTypeWithValidRequestHeader() throws IOException return headerBuffer.remaining(); }); - assertThrows(InvalidRequestException.class, () -> authenticator.authenticate()); + assertThrows(InvalidRequestException.class, authenticator::authenticate); verify(transportLayer, times(2)).read(any(ByteBuffer.class)); } @@ -156,7 +156,7 @@ public void testInvalidRequestHeader() throws IOException { return headerBuffer.remaining(); }); - assertThrows(InvalidRequestException.class, () -> authenticator.authenticate()); + assertThrows(InvalidRequestException.class, authenticator::authenticate); verify(transportLayer, times(2)).read(any(ByteBuffer.class)); } diff --git a/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java index 6770e4702ff1f..1c82dc62ff0d4 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/ssl/SslFactoryTest.java @@ -359,7 +359,6 @@ public void testPemReconfiguration() throws Exception { sslConfig = new TestSecurityConfig(props); sslFactory.reconfigure(sslConfig.values()); assertNotSame(sslEngineFactory, sslFactory.sslEngineFactory(), "SslEngineFactory not recreated"); - sslEngineFactory = sslFactory.sslEngineFactory(); } @Test @@ -400,15 +399,15 @@ public void testUntrustedKeyStoreValidationFails() throws Exception { @Test public void testKeystoreVerifiableUsingTruststore() throws Exception { - verifyKeystoreVerifiableUsingTruststore(false, tlsProtocol); + verifyKeystoreVerifiableUsingTruststore(false); } @Test public void testPemKeystoreVerifiableUsingTruststore() throws Exception { - verifyKeystoreVerifiableUsingTruststore(true, tlsProtocol); + verifyKeystoreVerifiableUsingTruststore(true); } - private void verifyKeystoreVerifiableUsingTruststore(boolean usePem, String tlsProtocol) throws Exception { + private void verifyKeystoreVerifiableUsingTruststore(boolean usePem) throws Exception { File trustStoreFile1 = usePem ? null : TestUtils.tempFile("truststore1", ".jks"); Map sslConfig1 = sslConfigsBuilder(ConnectionMode.SERVER) .createNewTrustStore(trustStoreFile1) @@ -436,15 +435,15 @@ private void verifyKeystoreVerifiableUsingTruststore(boolean usePem, String tlsP @Test public void testCertificateEntriesValidation() throws Exception { - verifyCertificateEntriesValidation(false, tlsProtocol); + verifyCertificateEntriesValidation(false); } @Test public void testPemCertificateEntriesValidation() throws Exception { - verifyCertificateEntriesValidation(true, tlsProtocol); + verifyCertificateEntriesValidation(true); } - private void verifyCertificateEntriesValidation(boolean usePem, String tlsProtocol) throws Exception { + private void verifyCertificateEntriesValidation(boolean usePem) throws Exception { File trustStoreFile = usePem ? null : TestUtils.tempFile("truststore", ".jks"); Map serverSslConfig = sslConfigsBuilder(ConnectionMode.SERVER) .createNewTrustStore(trustStoreFile) diff --git a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java index 72a1ccfe65106..b89aade368617 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java @@ -372,7 +372,7 @@ static String pem(Certificate cert) throws IOException { try (PemWriter pemWriter = new PemWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))) { pemWriter.writeObject(new JcaMiscPEMGenerator(cert)); } - return new String(out.toByteArray(), StandardCharsets.UTF_8); + return out.toString(StandardCharsets.UTF_8); } static String pem(PrivateKey privateKey, Password password) throws IOException { @@ -390,7 +390,7 @@ static String pem(PrivateKey privateKey, Password password) throws IOException { } } } - return new String(out.toByteArray(), StandardCharsets.UTF_8); + return out.toString(StandardCharsets.UTF_8); } public static class CertificateBuilder { diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java index 37f68a97ecb1c..0f93d953be51e 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -505,7 +505,7 @@ public static void isValidClusterId(String clusterId) { assertNotNull(clusterId); // Base 64 encoded value is 22 characters - assertEquals(clusterId.length(), 22); + assertEquals(22, clusterId.length()); Pattern clusterIdPattern = Pattern.compile("[a-zA-Z0-9_\\-]+"); Matcher matcher = clusterIdPattern.matcher(clusterId); @@ -516,7 +516,7 @@ public static void isValidClusterId(String clusterId) { byte[] decodedUuid = Base64.getDecoder().decode(originalClusterId); // We expect 16 bytes, same as the input UUID. - assertEquals(decodedUuid.length, 16); + assertEquals(16, decodedUuid.length); //Check if it can be converted back to a UUID. try { @@ -570,17 +570,6 @@ public static ByteBuffer toBuffer(UnalignedRecords records) { return toBuffer(records.toSend()); } - public static Set generateRandomTopicPartitions(int numTopic, int numPartitionPerTopic) { - Set tps = new HashSet<>(); - for (int i = 0; i < numTopic; i++) { - String topic = randomString(32); - for (int j = 0; j < numPartitionPerTopic; j++) { - tps.add(new TopicPartition(topic, j)); - } - } - return tps; - } - /** * Assert that a future raises an expected exception cause type. * This method will wait for the future to complete or timeout(15000 milliseconds). diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java index 1c08cbaf72ef9..b55673810a427 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/clients/admin/FakeLocalMetadataStore.java @@ -66,7 +66,7 @@ public static void updatePartitionCount(String topic, int newPartitionCount) { */ public static void updateTopicConfig(String topic, Config newConfig) { ConcurrentHashMap topicConfigs = FakeLocalMetadataStore.ALL_TOPICS.getOrDefault(topic, new ConcurrentHashMap<>()); - newConfig.entries().stream().forEach(configEntry -> { + newConfig.entries().forEach(configEntry -> { if (configEntry.name() != null) { if (configEntry.value() != null) { log.debug("Topic '{}' update config '{}' to '{}'", topic, configEntry.name(), configEntry.value()); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java index 2f6de675219a9..ad8328deabe11 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java @@ -43,7 +43,6 @@ import org.apache.kafka.connect.util.SinkUtils; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.connect.util.clusters.WorkerHandle; -import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; @@ -78,6 +77,7 @@ import static org.apache.kafka.clients.CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG; import static org.apache.kafka.common.config.AbstractConfig.CONFIG_PROVIDERS_CONFIG; import static org.apache.kafka.common.config.TopicConfig.DELETE_RETENTION_MS_CONFIG; +import static org.apache.kafka.common.config.TopicConfig.SEGMENT_MS_CONFIG; import static org.apache.kafka.connect.integration.BlockingConnectorTest.TASK_STOP; import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; @@ -1136,7 +1136,7 @@ public void testCompactedDeletedOlderConnectorConfig() throws Exception { final String configTopic = "kafka-16838-configs"; final int offsetCommitIntervalMs = 100; workerProps.put(CONFIG_TOPIC_CONFIG, configTopic); - workerProps.put(CONFIG_STORAGE_PREFIX + LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "100"); + workerProps.put(CONFIG_STORAGE_PREFIX + SEGMENT_MS_CONFIG, "100"); workerProps.put(CONFIG_STORAGE_PREFIX + DELETE_RETENTION_MS_CONFIG, "1"); workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, Integer.toString(offsetCommitIntervalMs)); diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java index 418d58376ccd8..b82829e1d627d 100644 --- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java +++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java @@ -2815,9 +2815,8 @@ public void testWriteEventWriteTimeoutTaskIsCancelledWhenHighWatermarkIsUpdated( assertTrue(write1.isDone()); assertTrue(write2.isDone()); - // All timer tasks have been cancelled. TimerTask entries are not removed in MockTimer. - assertEquals(2, timer.size()); - timer.taskQueue().forEach(taskEntry -> assertTrue(taskEntry.cancelled())); + // All timer tasks have been cancelled. Hence,they have been removed in MockTimer. + assertEquals(0, timer.size()); } @Test @@ -2885,9 +2884,8 @@ public void testCoordinatorCompleteTransactionEventWriteTimeoutTaskIsCancelledWh assertEquals(1, runtime.contextOrThrow(TP).coordinator.lastCommittedOffset()); assertTrue(write1.isDone()); - // All timer tasks have been cancelled. TimerTask entries are not removed in MockTimer. - assertEquals(1, timer.size()); - timer.taskQueue().forEach(taskEntry -> assertTrue(taskEntry.cancelled())); + // All timer tasks have been cancelled. Hence, they have been removed in MockTimer. + assertEquals(0, timer.size()); } @Test diff --git a/core/src/main/java/kafka/server/TierStateMachine.java b/core/src/main/java/kafka/server/TierStateMachine.java index 902bc62cbb953..1e851f2f4fb8d 100644 --- a/core/src/main/java/kafka/server/TierStateMachine.java +++ b/core/src/main/java/kafka/server/TierStateMachine.java @@ -96,7 +96,7 @@ PartitionFetchState start(TopicPartition topicPartition, PartitionFetchState currentFetchState, PartitionData fetchPartitionData) throws Exception { OffsetAndEpoch epochAndLeaderLocalStartOffset = leader.fetchEarliestLocalOffset(topicPartition, currentFetchState.currentLeaderEpoch()); - int epoch = epochAndLeaderLocalStartOffset.leaderEpoch(); + int epoch = epochAndLeaderLocalStartOffset.epoch(); long leaderLocalStartOffset = epochAndLeaderLocalStartOffset.offset(); long offsetToFetch; diff --git a/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java b/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java index c898f448a1a04..ecbb6c8b1543a 100644 --- a/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java +++ b/core/src/main/java/kafka/server/builders/KafkaApisBuilder.java @@ -69,7 +69,7 @@ public class KafkaApisBuilder { private DelegationTokenManager tokenManager = null; private ApiVersionManager apiVersionManager = null; private ClientMetricsManager clientMetricsManager = null; - private Optional shareCoordinator = Optional.empty(); + private ShareCoordinator shareCoordinator = null; private GroupConfigManager groupConfigManager = null; public KafkaApisBuilder setRequestChannel(RequestChannel requestChannel) { @@ -97,7 +97,7 @@ public KafkaApisBuilder setTxnCoordinator(TransactionCoordinator txnCoordinator) return this; } - public KafkaApisBuilder setShareCoordinator(Optional shareCoordinator) { + public KafkaApisBuilder setShareCoordinator(ShareCoordinator shareCoordinator) { this.shareCoordinator = shareCoordinator; return this; } @@ -194,8 +194,8 @@ public KafkaApis build() { if (replicaManager == null) throw new RuntimeException("You must set replicaManager"); if (groupCoordinator == null) throw new RuntimeException("You must set groupCoordinator"); if (txnCoordinator == null) throw new RuntimeException("You must set txnCoordinator"); - if (autoTopicCreationManager == null) - throw new RuntimeException("You must set autoTopicCreationManager"); + if (shareCoordinator == null) throw new RuntimeException("You must set shareCoordinator"); + if (autoTopicCreationManager == null) throw new RuntimeException("You must set autoTopicCreationManager"); if (config == null) config = new KafkaConfig(Map.of()); if (configRepository == null) throw new RuntimeException("You must set configRepository"); if (metadataCache == null) throw new RuntimeException("You must set metadataCache"); @@ -213,7 +213,7 @@ public KafkaApis build() { replicaManager, groupCoordinator, txnCoordinator, - OptionConverters.toScala(shareCoordinator), + shareCoordinator, autoTopicCreationManager, brokerId, config, diff --git a/core/src/main/java/kafka/server/share/DelayedShareFetch.java b/core/src/main/java/kafka/server/share/DelayedShareFetch.java index 5bd5f4ea6aa55..addfc691f1a08 100644 --- a/core/src/main/java/kafka/server/share/DelayedShareFetch.java +++ b/core/src/main/java/kafka/server/share/DelayedShareFetch.java @@ -22,6 +22,7 @@ import kafka.server.ReplicaManager; import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.errors.NotLeaderOrFollowerException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; @@ -72,6 +73,8 @@ import scala.jdk.javaapi.CollectionConverters; import scala.runtime.BoxedUnit; +import static kafka.server.share.PendingRemoteFetches.RemoteFetch; + /** * A delayed share fetch operation has been introduced in case there is a share fetch request which cannot be completed instantaneously. */ @@ -94,11 +97,15 @@ public class DelayedShareFetch extends DelayedOperation { * Metric for the rate of expired delayed fetch requests. */ private final Meter expiredRequestMeter; + /** + * fetchId serves as a token while acquiring/releasing share partition's fetch lock. + */ + private final Uuid fetchId; // Tracks the start time to acquire any share partition for a fetch request. private long acquireStartTimeMs; private LinkedHashMap partitionsAcquired; private LinkedHashMap localPartitionsAlreadyFetched; - private Optional remoteFetchOpt; + private Optional pendingRemoteFetchesOpt; private Optional remoteStorageFetchException; /** @@ -127,7 +134,8 @@ public DelayedShareFetch( PartitionMaxBytesStrategy.type(PartitionMaxBytesStrategy.StrategyType.UNIFORM), shareGroupMetrics, time, - Optional.empty() + Optional.empty(), + Uuid.randomUuid() ); } @@ -142,7 +150,7 @@ public DelayedShareFetch( * @param partitionMaxBytesStrategy The strategy to identify the max bytes for topic partitions in the share fetch request. * @param shareGroupMetrics The share group metrics to record the metrics. * @param time The system time. - * @param remoteFetchOpt Optional containing an in-flight remote fetch object or an empty optional. + * @param pendingRemoteFetchesOpt Optional containing an in-flight remote fetch object or an empty optional. */ DelayedShareFetch( ShareFetch shareFetch, @@ -152,7 +160,8 @@ public DelayedShareFetch( PartitionMaxBytesStrategy partitionMaxBytesStrategy, ShareGroupMetrics shareGroupMetrics, Time time, - Optional remoteFetchOpt + Optional pendingRemoteFetchesOpt, + Uuid fetchId ) { super(shareFetch.fetchParams().maxWaitMs, Optional.empty()); this.shareFetch = shareFetch; @@ -165,8 +174,9 @@ public DelayedShareFetch( this.shareGroupMetrics = shareGroupMetrics; this.time = time; this.acquireStartTimeMs = time.hiResClockMs(); - this.remoteFetchOpt = remoteFetchOpt; + this.pendingRemoteFetchesOpt = pendingRemoteFetchesOpt; this.remoteStorageFetchException = Optional.empty(); + this.fetchId = fetchId; // Register metrics for DelayedShareFetch. KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "DelayedShareFetchMetrics"); this.expiredRequestMeter = metricsGroup.newMeter(EXPIRES_PER_SEC, "requests", TimeUnit.SECONDS); @@ -194,7 +204,7 @@ public void onComplete() { try { if (remoteStorageFetchException.isPresent()) { completeErroneousRemoteShareFetchRequest(); - } else if (remoteFetchOpt.isPresent()) { + } else if (pendingRemoteFetchesOpt.isPresent()) { completeRemoteStorageShareFetchRequest(); } else { completeLocalLogShareFetchRequest(); @@ -248,14 +258,18 @@ private void processAcquiredTopicPartitionsForLocalLogFetch(LinkedHashMap shareFetchPartitionDataList = new ArrayList<>(); - responseData.forEach((topicIdPartition, logReadResult) -> - shareFetchPartitionDataList.add(new ShareFetchPartitionData( - topicIdPartition, - topicPartitionData.get(topicIdPartition), - logReadResult.toFetchPartitionData(false) - )) - ); + responseData.forEach((topicIdPartition, logReadResult) -> { + if (logReadResult.info().delayedRemoteStorageFetch.isEmpty()) { + shareFetchPartitionDataList.add(new ShareFetchPartitionData( + topicIdPartition, + topicPartitionData.get(topicIdPartition), + logReadResult.toFetchPartitionData(false) + )); + } + }); shareFetch.maybeComplete(ShareFetchUtils.processFetchResponse( shareFetch, @@ -272,13 +286,39 @@ private void processAcquiredTopicPartitionsForLocalLogFetch(LinkedHashMap topicPartitionData, + LinkedHashMap replicaManagerReadResponse + ) { + replicaManagerReadResponse.forEach((topicIdPartition, logReadResult) -> { + if (logReadResult.info().delayedRemoteStorageFetch.isPresent()) { + SharePartition sharePartition = sharePartitions.get(topicIdPartition); + sharePartition.updateFetchOffsetMetadata( + topicPartitionData.get(topicIdPartition), + null + ); + } + }); + } + /** * Try to complete the fetch operation if we can acquire records for any partition in the share fetch request. */ @Override public boolean tryComplete() { // Check to see if the remote fetch is in flight. If there is an in flight remote fetch we want to resolve it first. - if (remoteFetchOpt.isPresent()) { + if (pendingRemoteFetchesOpt.isPresent()) { return maybeCompletePendingRemoteFetch(); } @@ -291,11 +331,11 @@ public boolean tryComplete() { // replicaManager.readFromLog to populate the offset metadata and update the fetch offset metadata for // those topic partitions. LinkedHashMap replicaManagerReadResponse = maybeReadFromLog(topicPartitionData); - // Store the remote fetch info and the topic partition for which we need to perform remote fetch. - Optional topicPartitionRemoteFetchInfoOpt = maybePrepareRemoteStorageFetchInfo(topicPartitionData, replicaManagerReadResponse); + // Store the remote fetch info for the topic partitions for which we need to perform remote fetch. + LinkedHashMap remoteStorageFetchInfoMap = maybePrepareRemoteStorageFetchInfo(topicPartitionData, replicaManagerReadResponse); - if (topicPartitionRemoteFetchInfoOpt.isPresent()) { - return maybeProcessRemoteFetch(topicPartitionData, topicPartitionRemoteFetchInfoOpt.get()); + if (!remoteStorageFetchInfoMap.isEmpty()) { + return maybeProcessRemoteFetch(topicPartitionData, remoteStorageFetchInfoMap); } maybeUpdateFetchOffsetMetadata(topicPartitionData, replicaManagerReadResponse); if (anyPartitionHasLogReadError(replicaManagerReadResponse) || isMinBytesSatisfied(topicPartitionData, partitionMaxBytesStrategy.maxBytes(shareFetch.fetchParams().maxBytes, topicPartitionData.keySet(), topicPartitionData.size()))) { @@ -344,20 +384,22 @@ LinkedHashMap acquirablePartitions( sharePartitionsForAcquire.forEach((topicIdPartition, sharePartition) -> { // Add the share partition to the list of partitions to be fetched only if we can // acquire the fetch lock on it. - if (sharePartition.maybeAcquireFetchLock()) { + if (sharePartition.maybeAcquireFetchLock(fetchId)) { try { + log.trace("Fetch lock for share partition {}-{} has been acquired by {}", shareFetch.groupId(), topicIdPartition, fetchId); // If the share partition is already at capacity, we should not attempt to fetch. if (sharePartition.canAcquireRecords()) { topicPartitionData.put(topicIdPartition, sharePartition.nextFetchOffset()); } else { - sharePartition.releaseFetchLock(); - log.trace("Record lock partition limit exceeded for SharePartition {}, " + - "cannot acquire more records", sharePartition); + sharePartition.releaseFetchLock(fetchId); + log.trace("Record lock partition limit exceeded for SharePartition {}-{}, " + + "cannot acquire more records. Releasing the fetch lock by {}", shareFetch.groupId(), topicIdPartition, fetchId); } } catch (Exception e) { - log.error("Error checking condition for SharePartition: {}", sharePartition, e); + log.error("Error checking condition for SharePartition: {}-{}", shareFetch.groupId(), topicIdPartition, e); // Release the lock, if error occurred. - sharePartition.releaseFetchLock(); + sharePartition.releaseFetchLock(fetchId); + log.trace("Fetch lock for share partition {}-{} is being released by {}", shareFetch.groupId(), topicIdPartition, fetchId); } } }); @@ -569,7 +611,8 @@ LinkedHashMap combineLogReadResponse(LinkedHash void releasePartitionLocks(Set topicIdPartitions) { topicIdPartitions.forEach(tp -> { SharePartition sharePartition = sharePartitions.get(tp); - sharePartition.releaseFetchLock(); + sharePartition.releaseFetchLock(fetchId); + log.trace("Fetch lock for share partition {}-{} is being released by {}", shareFetch.groupId(), tp, fetchId); }); } @@ -579,8 +622,8 @@ Lock lock() { } // Visible for testing. - RemoteFetch remoteFetch() { - return remoteFetchOpt.orElse(null); + PendingRemoteFetches pendingRemoteFetches() { + return pendingRemoteFetchesOpt.orElse(null); } // Visible for testing. @@ -588,101 +631,111 @@ Meter expiredRequestMeter() { return expiredRequestMeter; } - private Optional maybePrepareRemoteStorageFetchInfo( + private LinkedHashMap maybePrepareRemoteStorageFetchInfo( LinkedHashMap topicPartitionData, LinkedHashMap replicaManagerReadResponse ) { - Optional topicPartitionRemoteFetchInfoOpt = Optional.empty(); + LinkedHashMap remoteStorageFetchInfoMap = new LinkedHashMap<>(); for (Map.Entry entry : replicaManagerReadResponse.entrySet()) { TopicIdPartition topicIdPartition = entry.getKey(); LogReadResult logReadResult = entry.getValue(); if (logReadResult.info().delayedRemoteStorageFetch.isPresent()) { - // TODO: There is a limitation in remote storage fetch for consumer groups that we can only perform remote fetch for - // a single topic partition in a fetch request. Since, the logic of fetch is largely based on how consumer groups work, - // we are following the same logic. However, this problem should be addressed as part of KAFKA-19133 which should help us perform - // fetch for multiple remote fetch topic partition in a single share fetch request - topicPartitionRemoteFetchInfoOpt = Optional.of(new TopicPartitionRemoteFetchInfo(topicIdPartition, logReadResult)); + remoteStorageFetchInfoMap.put(topicIdPartition, logReadResult); partitionsAcquired.put(topicIdPartition, topicPartitionData.get(topicIdPartition)); - break; } } - return topicPartitionRemoteFetchInfoOpt; + return remoteStorageFetchInfoMap; } private boolean maybeProcessRemoteFetch( LinkedHashMap topicPartitionData, - TopicPartitionRemoteFetchInfo topicPartitionRemoteFetchInfo + LinkedHashMap remoteStorageFetchInfoMap ) { Set nonRemoteFetchTopicPartitions = new LinkedHashSet<>(); topicPartitionData.keySet().forEach(topicIdPartition -> { - // topic partitions for which fetch would not be happening in this share fetch request. - if (!topicPartitionRemoteFetchInfo.topicIdPartition().equals(topicIdPartition)) { + // non-remote storage fetch topic partitions for which fetch would not be happening in this share fetch request. + if (!remoteStorageFetchInfoMap.containsKey(topicIdPartition)) { nonRemoteFetchTopicPartitions.add(topicIdPartition); } }); // Release fetch lock for the topic partitions that were acquired but were not a part of remote fetch and add // them to the delayed actions queue. releasePartitionLocksAndAddToActionQueue(nonRemoteFetchTopicPartitions); - processRemoteFetchOrException(topicPartitionRemoteFetchInfo); + processRemoteFetchOrException(remoteStorageFetchInfoMap); // Check if remote fetch can be completed. return maybeCompletePendingRemoteFetch(); } /** - * Throws an exception if a task for remote storage fetch could not be scheduled successfully else updates remoteFetchOpt. - * @param topicPartitionRemoteFetchInfo - The remote storage fetch information. + * Throws an exception if a task for remote storage fetch could not be scheduled successfully else updates pendingRemoteFetchesOpt. + * @param remoteStorageFetchInfoMap - The remote storage fetch information. */ private void processRemoteFetchOrException( - TopicPartitionRemoteFetchInfo topicPartitionRemoteFetchInfo + LinkedHashMap remoteStorageFetchInfoMap ) { - TopicIdPartition remoteFetchTopicIdPartition = topicPartitionRemoteFetchInfo.topicIdPartition(); - RemoteStorageFetchInfo remoteStorageFetchInfo = topicPartitionRemoteFetchInfo.logReadResult().info().delayedRemoteStorageFetch.get(); + LinkedHashMap fetchOffsetMetadataMap = new LinkedHashMap<>(); + remoteStorageFetchInfoMap.forEach((topicIdPartition, logReadResult) -> fetchOffsetMetadataMap.put( + topicIdPartition, + logReadResult.info().fetchOffsetMetadata + )); - Future remoteFetchTask; - CompletableFuture remoteFetchResult = new CompletableFuture<>(); - try { - remoteFetchTask = replicaManager.remoteLogManager().get().asyncRead( - remoteStorageFetchInfo, - result -> { - remoteFetchResult.complete(result); - replicaManager.completeDelayedShareFetchRequest(new DelayedShareFetchGroupKey(shareFetch.groupId(), remoteFetchTopicIdPartition.topicId(), remoteFetchTopicIdPartition.partition())); - } - ); - } catch (Exception e) { - // Throw the error if any in scheduling the remote fetch task. - remoteStorageFetchException = Optional.of(e); - throw e; + List remoteFetches = new ArrayList<>(); + for (Map.Entry entry : remoteStorageFetchInfoMap.entrySet()) { + TopicIdPartition remoteFetchTopicIdPartition = entry.getKey(); + RemoteStorageFetchInfo remoteStorageFetchInfo = entry.getValue().info().delayedRemoteStorageFetch.get(); + + Future remoteFetchTask; + CompletableFuture remoteFetchResult = new CompletableFuture<>(); + try { + remoteFetchTask = replicaManager.remoteLogManager().get().asyncRead( + remoteStorageFetchInfo, + result -> { + remoteFetchResult.complete(result); + replicaManager.completeDelayedShareFetchRequest(new DelayedShareFetchGroupKey(shareFetch.groupId(), remoteFetchTopicIdPartition.topicId(), remoteFetchTopicIdPartition.partition())); + } + ); + } catch (Exception e) { + // Cancel the already created remote fetch tasks in case an exception occurs. + remoteFetches.forEach(this::cancelRemoteFetchTask); + // Throw the error if any in scheduling the remote fetch task. + remoteStorageFetchException = Optional.of(e); + throw e; + } + remoteFetches.add(new RemoteFetch(remoteFetchTopicIdPartition, entry.getValue(), remoteFetchTask, remoteFetchResult, remoteStorageFetchInfo)); } - remoteFetchOpt = Optional.of(new RemoteFetch(remoteFetchTopicIdPartition, topicPartitionRemoteFetchInfo.logReadResult(), remoteFetchTask, remoteFetchResult, remoteStorageFetchInfo)); + pendingRemoteFetchesOpt = Optional.of(new PendingRemoteFetches(remoteFetches, fetchOffsetMetadataMap)); } /** - * This function checks if the remote fetch can be completed or not. It should always be called once you confirm remoteFetchOpt.isPresent(). + * This function checks if the remote fetch can be completed or not. It should always be called once you confirm pendingRemoteFetchesOpt.isPresent(). * The operation can be completed if: * Case a: The partition is in an offline log directory on this broker * Case b: This broker does not know the partition it tries to fetch * Case c: This broker is no longer the leader of the partition it tries to fetch - * Case d: The remote storage read request completed (succeeded or failed) + * Case d: All remote storage read requests completed * @return boolean representing whether the remote fetch is completed or not. */ private boolean maybeCompletePendingRemoteFetch() { boolean canComplete = false; - TopicIdPartition topicIdPartition = remoteFetchOpt.get().topicIdPartition(); - try { - replicaManager.getPartitionOrException(topicIdPartition.topicPartition()); - } catch (KafkaStorageException e) { // Case a - log.debug("TopicPartition {} is in an offline log directory, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); - canComplete = true; - } catch (UnknownTopicOrPartitionException e) { // Case b - log.debug("Broker no longer knows of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); - canComplete = true; - } catch (NotLeaderOrFollowerException e) { // Case c - log.debug("Broker is no longer the leader or follower of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); - canComplete = true; + for (TopicIdPartition topicIdPartition : pendingRemoteFetchesOpt.get().fetchOffsetMetadataMap().keySet()) { + try { + replicaManager.getPartitionOrException(topicIdPartition.topicPartition()); + } catch (KafkaStorageException e) { // Case a + log.debug("TopicPartition {} is in an offline log directory, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); + canComplete = true; + } catch (UnknownTopicOrPartitionException e) { // Case b + log.debug("Broker no longer knows of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); + canComplete = true; + } catch (NotLeaderOrFollowerException e) { // Case c + log.debug("Broker is no longer the leader or follower of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); + canComplete = true; + } + if (canComplete) + break; } - if (canComplete || remoteFetchOpt.get().remoteFetchResult().isDone()) { // Case d + if (canComplete || pendingRemoteFetchesOpt.get().isDone()) { // Case d return forceCompleteRequest(); } else return false; @@ -725,44 +778,45 @@ private void completeRemoteStorageShareFetchRequest() { try { List shareFetchPartitionData = new ArrayList<>(); int readableBytes = 0; - if (remoteFetchOpt.get().remoteFetchResult().isDone()) { - RemoteFetch remoteFetch = remoteFetchOpt.get(); - RemoteLogReadResult remoteLogReadResult = remoteFetch.remoteFetchResult().get(); - if (remoteLogReadResult.error.isPresent()) { - Throwable error = remoteLogReadResult.error.get(); - // If there is any error for the remote fetch topic partition, we populate the error accordingly. - shareFetchPartitionData.add( - new ShareFetchPartitionData( - remoteFetch.topicIdPartition(), - partitionsAcquired.get(remoteFetch.topicIdPartition()), - ReplicaManager.createLogReadResult(error).toFetchPartitionData(false) - ) - ); - } else { - FetchDataInfo info = remoteLogReadResult.fetchDataInfo.get(); - TopicIdPartition topicIdPartition = remoteFetch.topicIdPartition(); - LogReadResult logReadResult = remoteFetch.logReadResult(); - shareFetchPartitionData.add( - new ShareFetchPartitionData( - topicIdPartition, - partitionsAcquired.get(remoteFetch.topicIdPartition()), - new FetchPartitionData( - logReadResult.error(), - logReadResult.highWatermark(), - logReadResult.leaderLogStartOffset(), - info.records, - Optional.empty(), - logReadResult.lastStableOffset().isDefined() ? OptionalLong.of((Long) logReadResult.lastStableOffset().get()) : OptionalLong.empty(), - info.abortedTransactions, - logReadResult.preferredReadReplica().isDefined() ? OptionalInt.of((Integer) logReadResult.preferredReadReplica().get()) : OptionalInt.empty(), - false + for (RemoteFetch remoteFetch : pendingRemoteFetchesOpt.get().remoteFetches()) { + if (remoteFetch.remoteFetchResult().isDone()) { + RemoteLogReadResult remoteLogReadResult = remoteFetch.remoteFetchResult().get(); + if (remoteLogReadResult.error.isPresent()) { + Throwable error = remoteLogReadResult.error.get(); + // If there is any error for the remote fetch topic partition, we populate the error accordingly. + shareFetchPartitionData.add( + new ShareFetchPartitionData( + remoteFetch.topicIdPartition(), + partitionsAcquired.get(remoteFetch.topicIdPartition()), + ReplicaManager.createLogReadResult(error).toFetchPartitionData(false) + ) + ); + } else { + FetchDataInfo info = remoteLogReadResult.fetchDataInfo.get(); + TopicIdPartition topicIdPartition = remoteFetch.topicIdPartition(); + LogReadResult logReadResult = remoteFetch.logReadResult(); + shareFetchPartitionData.add( + new ShareFetchPartitionData( + topicIdPartition, + partitionsAcquired.get(remoteFetch.topicIdPartition()), + new FetchPartitionData( + logReadResult.error(), + logReadResult.highWatermark(), + logReadResult.leaderLogStartOffset(), + info.records, + Optional.empty(), + logReadResult.lastStableOffset().isDefined() ? OptionalLong.of((Long) logReadResult.lastStableOffset().get()) : OptionalLong.empty(), + info.abortedTransactions, + logReadResult.preferredReadReplica().isDefined() ? OptionalInt.of((Integer) logReadResult.preferredReadReplica().get()) : OptionalInt.empty(), + false + ) ) - ) - ); - readableBytes += info.records.sizeInBytes(); + ); + readableBytes += info.records.sizeInBytes(); + } + } else { + cancelRemoteFetchTask(remoteFetch); } - } else { - cancelRemoteFetchTask(); } // If remote fetch bytes < shareFetch.fetchParams().maxBytes, then we will try for a local read. @@ -782,6 +836,7 @@ private void completeRemoteStorageShareFetchRequest() { LinkedHashMap responseData = readFromLog( acquiredNonRemoteFetchTopicPartitionData, partitionMaxBytesStrategy.maxBytes(shareFetch.fetchParams().maxBytes - readableBytes, acquiredNonRemoteFetchTopicPartitionData.keySet(), acquiredNonRemoteFetchTopicPartitionData.size())); + resetFetchOffsetMetadataForRemoteFetchPartitions(acquiredNonRemoteFetchTopicPartitionData, responseData); for (Map.Entry entry : responseData.entrySet()) { if (entry.getValue().info().delayedRemoteStorageFetch.isEmpty()) { shareFetchPartitionData.add( @@ -806,7 +861,7 @@ private void completeRemoteStorageShareFetchRequest() { shareFetch.maybeComplete(remoteFetchResponse); log.trace("Remote share fetch request completed successfully, response: {}", remoteFetchResponse); } catch (InterruptedException | ExecutionException e) { - log.error("Exception occurred in completing remote fetch {} for delayed share fetch request {}", remoteFetchOpt.get(), e); + log.error("Exception occurred in completing remote fetch {} for delayed share fetch request {}", pendingRemoteFetchesOpt.get(), e); handleExceptionInCompletingRemoteStorageShareFetchRequest(acquiredNonRemoteFetchTopicPartitionData.keySet(), e); } catch (Exception e) { log.error("Unexpected error in processing delayed share fetch request", e); @@ -832,11 +887,11 @@ private void handleExceptionInCompletingRemoteStorageShareFetchRequest( * already running as it may force closing opened/cached resources as transaction index. * Note - This function should only be called when we know that there is remote fetch. */ - private void cancelRemoteFetchTask() { - boolean cancelled = remoteFetchOpt.get().remoteFetchTask().cancel(false); + private void cancelRemoteFetchTask(RemoteFetch remoteFetch) { + boolean cancelled = remoteFetch.remoteFetchTask().cancel(false); if (!cancelled) { log.debug("Remote fetch task for RemoteStorageFetchInfo: {} could not be cancelled and its isDone value is {}", - remoteFetchOpt.get().remoteFetchInfo(), remoteFetchOpt.get().remoteFetchTask().isDone()); + remoteFetch.remoteFetchInfo(), remoteFetch.remoteFetchTask().isDone()); } } @@ -849,36 +904,4 @@ private boolean forceCompleteRequest() { } return completedByMe; } - - public record RemoteFetch( - TopicIdPartition topicIdPartition, - LogReadResult logReadResult, - Future remoteFetchTask, - CompletableFuture remoteFetchResult, - RemoteStorageFetchInfo remoteFetchInfo - ) { - @Override - public String toString() { - return "RemoteFetch(" + - "topicIdPartition=" + topicIdPartition + - ", logReadResult=" + logReadResult + - ", remoteFetchTask=" + remoteFetchTask + - ", remoteFetchResult=" + remoteFetchResult + - ", remoteFetchInfo=" + remoteFetchInfo + - ")"; - } - } - - public record TopicPartitionRemoteFetchInfo( - TopicIdPartition topicIdPartition, - LogReadResult logReadResult - ) { - @Override - public String toString() { - return "TopicPartitionRemoteFetchInfo(" + - "topicIdPartition=" + topicIdPartition + - ", logReadResult=" + logReadResult + - ")"; - } - } } diff --git a/core/src/main/java/kafka/server/share/PendingRemoteFetches.java b/core/src/main/java/kafka/server/share/PendingRemoteFetches.java new file mode 100644 index 0000000000000..2eb92672dc525 --- /dev/null +++ b/core/src/main/java/kafka/server/share/PendingRemoteFetches.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server.share; + +import kafka.server.LogReadResult; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.RemoteLogReadResult; +import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; + +/** + * This class is used to store the remote storage fetch information for topic partitions in a share fetch request. + */ +public class PendingRemoteFetches { + private final List remoteFetches; + private final LinkedHashMap fetchOffsetMetadataMap; + + PendingRemoteFetches(List remoteFetches, LinkedHashMap fetchOffsetMetadataMap) { + this.remoteFetches = remoteFetches; + this.fetchOffsetMetadataMap = fetchOffsetMetadataMap; + } + + public boolean isDone() { + for (RemoteFetch remoteFetch : remoteFetches) { + if (!remoteFetch.remoteFetchResult.isDone()) + return false; + } + return true; + } + + public List remoteFetches() { + return remoteFetches; + } + + public LinkedHashMap fetchOffsetMetadataMap() { + return fetchOffsetMetadataMap; + } + + @Override + public String toString() { + return "PendingRemoteFetches(" + + "remoteFetches=" + remoteFetches + + ", fetchOffsetMetadataMap=" + fetchOffsetMetadataMap + + ")"; + } + + public record RemoteFetch( + TopicIdPartition topicIdPartition, + LogReadResult logReadResult, + Future remoteFetchTask, + CompletableFuture remoteFetchResult, + RemoteStorageFetchInfo remoteFetchInfo + ) { + @Override + public String toString() { + return "RemoteFetch(" + + "topicIdPartition=" + topicIdPartition + + ", logReadResult=" + logReadResult + + ", remoteFetchTask=" + remoteFetchTask + + ", remoteFetchResult=" + remoteFetchResult + + ", remoteFetchInfo=" + remoteFetchInfo + + ")"; + } + } +} diff --git a/core/src/main/java/kafka/server/share/ShareFetchUtils.java b/core/src/main/java/kafka/server/share/ShareFetchUtils.java index 603ae8e048b23..ba9e5368bcfd8 100644 --- a/core/src/main/java/kafka/server/share/ShareFetchUtils.java +++ b/core/src/main/java/kafka/server/share/ShareFetchUtils.java @@ -27,9 +27,9 @@ import org.apache.kafka.common.message.ShareFetchResponseData; import org.apache.kafka.common.message.ShareFetchResponseData.AcquiredRecords; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.record.FileLogInputStream.FileChannelRecordBatch; import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.coordinator.group.GroupConfigManager; @@ -205,20 +205,17 @@ static Partition partition(ReplicaManager replicaManager, TopicPartition tp) { * * @param records The records to be sliced. * @param shareAcquiredRecords The share acquired records containing the non-empty acquired records. - * @return The sliced records, if the records are of type FileRecords and the acquired records are a subset - * of the fetched records. Otherwise, the original records are returned. + * @return The sliced records, if the acquired records are a subset of the fetched records. Otherwise, + * the original records are returned. */ static Records maybeSliceFetchRecords(Records records, ShareAcquiredRecords shareAcquiredRecords) { - if (!(records instanceof FileRecords fileRecords)) { - return records; - } // The acquired records should be non-empty, do not check as the method is called only when the // acquired records are non-empty. List acquiredRecords = shareAcquiredRecords.acquiredRecords(); try { - final Iterator iterator = fileRecords.batchIterator(); + final Iterator iterator = records.batchIterator(); // Track the first overlapping batch with the first acquired offset. - FileChannelRecordBatch firstOverlapBatch = iterator.next(); + RecordBatch firstOverlapBatch = iterator.next(); // If there exists single fetch batch, then return the original records. if (!iterator.hasNext()) { return records; @@ -230,7 +227,7 @@ static Records maybeSliceFetchRecords(Records records, ShareAcquiredRecords shar int size = 0; // Start iterating from the second batch. while (iterator.hasNext()) { - FileChannelRecordBatch batch = iterator.next(); + RecordBatch batch = iterator.next(); // Iterate until finds the first overlap batch with the first acquired offset. All the // batches before this first overlap batch should be sliced hence increment the start // position. @@ -249,10 +246,10 @@ static Records maybeSliceFetchRecords(Records records, ShareAcquiredRecords shar // acquired offset. size += firstOverlapBatch.sizeInBytes(); // Check if we do not need slicing i.e. neither start position nor size changed. - if (startPosition == 0 && size == fileRecords.sizeInBytes()) { + if (startPosition == 0 && size == records.sizeInBytes()) { return records; } - return fileRecords.slice(startPosition, size); + return records.slice(startPosition, size); } catch (Exception e) { log.error("Error while checking batches for acquired records: {}, skipping slicing.", acquiredRecords, e); // If there is an exception while slicing, return the original records so that the fetch diff --git a/core/src/main/java/kafka/server/share/SharePartition.java b/core/src/main/java/kafka/server/share/SharePartition.java index db2ff4fc8cfeb..91a11d488f49f 100644 --- a/core/src/main/java/kafka/server/share/SharePartition.java +++ b/core/src/main/java/kafka/server/share/SharePartition.java @@ -81,6 +81,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -197,6 +198,16 @@ public byte id() { } } + /** + * The DeliveryCountOps is used to specify the behavior on the delivery count: increase, decrease, + * or do nothing. + */ + private enum DeliveryCountOps { + INCREASE, + DECREASE, + NO_OP + } + /** * The group id of the share partition belongs to. */ @@ -234,9 +245,10 @@ public byte id() { /** * The lock to ensure that the same share partition does not enter a fetch queue - * while another one is being fetched within the queue. + * while another one is being fetched within the queue. The caller's id that acquires the fetch + * lock is utilized for ensuring the above. */ - private final AtomicBoolean fetchLock; + private final AtomicReference fetchLock; /** * The max in-flight messages is used to limit the number of records that can be in-flight at any @@ -318,6 +330,12 @@ public byte id() { */ private final OffsetMetadata fetchOffsetMetadata; + /** + * The delayed share fetch key is used to track the delayed share fetch requests that are waiting + * for the respective share partition. + */ + private final DelayedShareFetchKey delayedShareFetchKey; + /** * The state epoch is used to track the version of the state of the share partition. */ @@ -394,7 +412,7 @@ public byte id() { this.cachedState = new ConcurrentSkipListMap<>(); this.lock = new ReentrantReadWriteLock(); this.findNextFetchOffset = new AtomicBoolean(false); - this.fetchLock = new AtomicBoolean(false); + this.fetchLock = new AtomicReference<>(null); this.defaultRecordLockDurationMs = defaultRecordLockDurationMs; this.timer = timer; this.time = time; @@ -404,6 +422,7 @@ public byte id() { this.replicaManager = replicaManager; this.groupConfigManager = groupConfigManager; this.fetchOffsetMetadata = new OffsetMetadata(); + this.delayedShareFetchKey = new DelayedShareFetchGroupKey(groupId, topicIdPartition); this.listener = listener; this.sharePartitionMetrics = sharePartitionMetrics; this.registerGaugeMetrics(); @@ -541,6 +560,9 @@ public CompletableFuture maybeInitialize() { } // Release the lock. lock.writeLock().unlock(); + // Avoid triggering the listener for waiting share fetch requests in purgatory as the + // share partition manager keeps track of same and will trigger the listener for the + // respective share partition. // Complete the future. if (isFailed) { future.completeExceptionally(throwable); @@ -832,7 +854,7 @@ public ShareAcquiredRecords acquire( continue; } - InFlightState updateResult = inFlightBatch.tryUpdateBatchState(RecordState.ACQUIRED, true, maxDeliveryCount, memberId); + InFlightState updateResult = inFlightBatch.tryUpdateBatchState(RecordState.ACQUIRED, DeliveryCountOps.INCREASE, maxDeliveryCount, memberId); if (updateResult == null) { log.info("Unable to acquire records for the batch: {} in share partition: {}-{}", inFlightBatch, groupId, topicIdPartition); @@ -1024,9 +1046,10 @@ private Optional releaseAcquiredRecordsForPerOffsetBatch(String membe return Optional.empty(); } if (offsetState.getValue().state == RecordState.ACQUIRED) { + // These records were fetched but they were not actually delivered to the client. InFlightState updateResult = offsetState.getValue().startStateTransition( offsetState.getKey() < startOffset ? RecordState.ARCHIVED : recordState, - false, + DeliveryCountOps.DECREASE, this.maxDeliveryCount, EMPTY_MEMBER_ID ); @@ -1072,7 +1095,7 @@ private Optional releaseAcquiredRecordsForCompleteBatch(String member if (inFlightBatch.batchState() == RecordState.ACQUIRED) { InFlightState updateResult = inFlightBatch.startBatchStateTransition( inFlightBatch.lastOffset() < startOffset ? RecordState.ARCHIVED : recordState, - false, + DeliveryCountOps.DECREASE, this.maxDeliveryCount, EMPTY_MEMBER_ID ); @@ -1327,13 +1350,14 @@ boolean canAcquireRecords() { * share partition is not fetched concurrently by multiple clients. The fetch lock is released once * the records are fetched and acquired. * + * @param fetchId - the caller's id that is trying to acquire the fetch lock. * @return A boolean which indicates whether the fetch lock is acquired. */ - public boolean maybeAcquireFetchLock() { + public boolean maybeAcquireFetchLock(Uuid fetchId) { if (stateNotActive()) { return false; } - boolean acquired = fetchLock.compareAndSet(false, true); + boolean acquired = fetchLock.compareAndSet(null, Objects.requireNonNull(fetchId)); if (acquired) { long currentTime = time.hiResClockMs(); fetchLockAcquiredTimeMs = currentTime; @@ -1343,21 +1367,29 @@ public boolean maybeAcquireFetchLock() { } /** - * Release the fetch lock once the records are fetched from the leader. + * Release the fetch lock once the records are fetched from the leader. It is imperative that the caller + * that acquired the fetch lock should be the one releasing it. + * @param fetchId - The caller's id that is trying to release the fetch lock. */ - void releaseFetchLock() { + void releaseFetchLock(Uuid fetchId) { // Register the metric for the duration the fetch lock was held. Do not register the metric // if the fetch lock was not acquired. - if (fetchLock.get()) { - long currentTime = time.hiResClockMs(); - long acquiredDurationMs = currentTime - fetchLockAcquiredTimeMs; - // Update the metric for the fetch lock time. - sharePartitionMetrics.recordFetchLockTimeMs(acquiredDurationMs); - // Update fetch lock ratio metric. - recordFetchLockRatioMetric(acquiredDurationMs); - fetchLockReleasedTimeMs = currentTime; - } - fetchLock.set(false); + long currentTime = time.hiResClockMs(); + if (!fetchLock.compareAndSet(Objects.requireNonNull(fetchId), null)) { + // This code should not be reached unless we are in error-prone scenarios. Since we are releasing the fetch + // lock for multiple share partitions at different places in DelayedShareFetch (due to tackling remote + // storage fetch and local log fetch from a single purgatory), in order to safeguard ourselves from bad code, + // we are logging when an instance that does not hold the fetch lock tries to release it. + Uuid fetchLockAcquiredBy = fetchLock.getAndSet(null); + log.info("Instance {} does not hold the fetch lock, yet trying to release it for share partition {}-{}. The lock was held by {}", + fetchId, groupId, topicIdPartition, fetchLockAcquiredBy); + } + long acquiredDurationMs = currentTime - fetchLockAcquiredTimeMs; + // Update the metric for the fetch lock time. + sharePartitionMetrics.recordFetchLockTimeMs(acquiredDurationMs); + // Update fetch lock ratio metric. + recordFetchLockRatioMetric(acquiredDurationMs); + fetchLockReleasedTimeMs = currentTime; } /** @@ -1624,8 +1656,8 @@ private int acquireSubsetBatchRecords( continue; } - InFlightState updateResult = offsetState.getValue().tryUpdateState(RecordState.ACQUIRED, true, maxDeliveryCount, - memberId); + InFlightState updateResult = offsetState.getValue().tryUpdateState(RecordState.ACQUIRED, DeliveryCountOps.INCREASE, + maxDeliveryCount, memberId); if (updateResult == null) { log.trace("Unable to acquire records for the offset: {} in batch: {}" + " for the share partition: {}-{}", offsetState.getKey(), inFlightBatch, @@ -1897,7 +1929,7 @@ private Optional acknowledgePerOffsetBatchRecords( recordStateDefault; InFlightState updateResult = offsetState.getValue().startStateTransition( recordState, - false, + DeliveryCountOps.NO_OP, this.maxDeliveryCount, EMPTY_MEMBER_ID ); @@ -1950,7 +1982,7 @@ private Optional acknowledgeCompleteBatch( // is only important when the batch is acquired. InFlightState updateResult = inFlightBatch.startBatchStateTransition( recordState, - false, + DeliveryCountOps.NO_OP, this.maxDeliveryCount, EMPTY_MEMBER_ID ); @@ -2037,6 +2069,10 @@ void rollbackOrProcessStateUpdates( } writeShareGroupState(stateBatches).whenComplete((result, exception) -> { + // There can be a pending delayed share fetch requests for the share partition which are waiting + // on the startOffset to move ahead, hence track if the state is updated in the cache. If + // yes, then notify the delayed share fetch purgatory to complete the pending requests. + boolean cacheStateUpdated = false; lock.writeLock().lock(); try { if (exception != null) { @@ -2055,27 +2091,31 @@ void rollbackOrProcessStateUpdates( state.cancelAndClearAcquisitionLockTimeoutTask(); }); // Update the cached state and start and end offsets after acknowledging/releasing the acquired records. - maybeUpdateCachedStateAndOffsets(); + cacheStateUpdated = maybeUpdateCachedStateAndOffsets(); future.complete(null); } finally { lock.writeLock().unlock(); + // Maybe complete the delayed share fetch request if the state has been changed in cache + // which might have moved start offset ahead. Hence, the pending delayed share fetch + // request can be completed. The call should be made outside the lock to avoid deadlock. + maybeCompleteDelayedShareFetchRequest(cacheStateUpdated); } }); } - private void maybeUpdateCachedStateAndOffsets() { + private boolean maybeUpdateCachedStateAndOffsets() { lock.writeLock().lock(); try { if (!canMoveStartOffset()) { - return; + return false; } // This will help to find the next position for the startOffset. // The new position of startOffset will be lastOffsetAcknowledged + 1 long lastOffsetAcknowledged = findLastOffsetAcknowledged(); - // If lastOffsetAcknowledged is -1, this means we cannot move out startOffset ahead + // If lastOffsetAcknowledged is -1, this means we cannot move startOffset ahead if (lastOffsetAcknowledged == -1) { - return; + return false; } // This is true if all records in the cachedState have been acknowledged (either Accept or Reject). @@ -2086,7 +2126,7 @@ private void maybeUpdateCachedStateAndOffsets() { endOffset = lastCachedOffset + 1; cachedState.clear(); // Nothing further to do. - return; + return true; } /* @@ -2133,6 +2173,7 @@ be removed once all the messages (0-99) are acknowledged (ACCEPT or REJECT). if (lastKeyToRemove != -1) { cachedState.subMap(firstKeyToRemove, true, lastKeyToRemove, true).clear(); } + return true; } finally { lock.writeLock().unlock(); } @@ -2394,13 +2435,10 @@ && checkForStartOffsetWithinBatch(inFlightBatch.firstOffset(), inFlightBatch.las lock.writeLock().unlock(); } + // If we have an acquisition lock timeout for a share-partition, then we should check if + // there is a pending share fetch request for the share-partition and complete it. // Skip null check for stateBatches, it should always be initialized if reached here. - if (!stateBatches.isEmpty()) { - // If we have an acquisition lock timeout for a share-partition, then we should check if - // there is a pending share fetch request for the share-partition and complete it. - DelayedShareFetchKey delayedShareFetchKey = new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()); - replicaManager.completeDelayedShareFetchRequest(delayedShareFetchKey); - } + maybeCompleteDelayedShareFetchRequest(!stateBatches.isEmpty()); } private void releaseAcquisitionLockOnTimeoutForCompleteBatch(InFlightBatch inFlightBatch, @@ -2409,7 +2447,7 @@ private void releaseAcquisitionLockOnTimeoutForCompleteBatch(InFlightBatch inFli if (inFlightBatch.batchState() == RecordState.ACQUIRED) { InFlightState updateResult = inFlightBatch.tryUpdateBatchState( inFlightBatch.lastOffset() < startOffset ? RecordState.ARCHIVED : RecordState.AVAILABLE, - false, + DeliveryCountOps.NO_OP, maxDeliveryCount, EMPTY_MEMBER_ID); if (updateResult == null) { @@ -2455,7 +2493,7 @@ private void releaseAcquisitionLockOnTimeoutForPerOffsetBatch(InFlightBatch inFl } InFlightState updateResult = offsetState.getValue().tryUpdateState( offsetState.getKey() < startOffset ? RecordState.ARCHIVED : RecordState.AVAILABLE, - false, + DeliveryCountOps.NO_OP, maxDeliveryCount, EMPTY_MEMBER_ID); if (updateResult == null) { @@ -2475,6 +2513,12 @@ private void releaseAcquisitionLockOnTimeoutForPerOffsetBatch(InFlightBatch inFl } } + private void maybeCompleteDelayedShareFetchRequest(boolean shouldComplete) { + if (shouldComplete) { + replicaManager.completeDelayedShareFetchRequest(delayedShareFetchKey); + } + } + private long startOffsetDuringInitialization(long partitionDataStartOffset) { // Set the state epoch and end offset from the persisted state. if (partitionDataStartOffset != PartitionFactory.UNINITIALIZED_START_OFFSET) { @@ -2733,6 +2777,11 @@ InitialReadGapOffset initialReadGapOffset() { return initialReadGapOffset; } + // Visible for testing. + Uuid fetchLock() { + return fetchLock.get(); + } + /** * The InitialReadGapOffset class is used to record the gap start and end offset of the probable gaps * of available records which are neither known to Persister nor to SharePartition. Share Partition @@ -2875,19 +2924,19 @@ private void archiveBatch(String newMemberId) { inFlightState().archive(newMemberId); } - private InFlightState tryUpdateBatchState(RecordState newState, boolean incrementDeliveryCount, int maxDeliveryCount, String newMemberId) { + private InFlightState tryUpdateBatchState(RecordState newState, DeliveryCountOps ops, int maxDeliveryCount, String newMemberId) { if (batchState == null) { throw new IllegalStateException("The batch state update is not available as the offset state is maintained"); } - return batchState.tryUpdateState(newState, incrementDeliveryCount, maxDeliveryCount, newMemberId); + return batchState.tryUpdateState(newState, ops, maxDeliveryCount, newMemberId); } - private InFlightState startBatchStateTransition(RecordState newState, boolean incrementDeliveryCount, int maxDeliveryCount, + private InFlightState startBatchStateTransition(RecordState newState, DeliveryCountOps ops, int maxDeliveryCount, String newMemberId) { if (batchState == null) { throw new IllegalStateException("The batch state update is not available as the offset state is maintained"); } - return batchState.startStateTransition(newState, incrementDeliveryCount, maxDeliveryCount, newMemberId); + return batchState.startStateTransition(newState, ops, maxDeliveryCount, newMemberId); } private void maybeInitializeOffsetStateUpdate() { @@ -2999,23 +3048,23 @@ private boolean hasOngoingStateTransition() { /** * Try to update the state of the records. The state of the records can only be updated if the - * new state is allowed to be transitioned from old state. The delivery count is not incremented + * new state is allowed to be transitioned from old state. The delivery count is not changed * if the state update is unsuccessful. * * @param newState The new state of the records. - * @param incrementDeliveryCount Whether to increment the delivery count. + * @param ops The behavior on the delivery count. * * @return {@code InFlightState} if update succeeds, null otherwise. Returning state * helps update chaining. */ - private InFlightState tryUpdateState(RecordState newState, boolean incrementDeliveryCount, int maxDeliveryCount, String newMemberId) { + private InFlightState tryUpdateState(RecordState newState, DeliveryCountOps ops, int maxDeliveryCount, String newMemberId) { try { - if (newState == RecordState.AVAILABLE && deliveryCount >= maxDeliveryCount) { + if (newState == RecordState.AVAILABLE && ops != DeliveryCountOps.DECREASE && deliveryCount >= maxDeliveryCount) { newState = RecordState.ARCHIVED; } state = state.validateTransition(newState); - if (incrementDeliveryCount && newState != RecordState.ARCHIVED) { - deliveryCount++; + if (newState != RecordState.ARCHIVED) { + deliveryCount = updatedDeliveryCount(ops); } memberId = newMemberId; return this; @@ -3025,14 +3074,23 @@ private InFlightState tryUpdateState(RecordState newState, boolean incrementDeli } } + private int updatedDeliveryCount(DeliveryCountOps ops) { + return switch (ops) { + case INCREASE -> deliveryCount + 1; + case DECREASE -> deliveryCount - 1; + // do nothing + case NO_OP -> deliveryCount; + }; + } + private void archive(String newMemberId) { state = RecordState.ARCHIVED; memberId = newMemberId; } - private InFlightState startStateTransition(RecordState newState, boolean incrementDeliveryCount, int maxDeliveryCount, String newMemberId) { + private InFlightState startStateTransition(RecordState newState, DeliveryCountOps ops, int maxDeliveryCount, String newMemberId) { rollbackState = new InFlightState(state, deliveryCount, memberId, acquisitionLockTimeoutTask); - return tryUpdateState(newState, incrementDeliveryCount, maxDeliveryCount, newMemberId); + return tryUpdateState(newState, ops, maxDeliveryCount, newMemberId); } private void completeStateTransition(boolean commit) { diff --git a/core/src/main/java/kafka/server/share/SharePartitionManager.java b/core/src/main/java/kafka/server/share/SharePartitionManager.java index a53f846a01c24..44af40ec8f8d6 100644 --- a/core/src/main/java/kafka/server/share/SharePartitionManager.java +++ b/core/src/main/java/kafka/server/share/SharePartitionManager.java @@ -420,11 +420,18 @@ private CompletableFuture shareFetchData, - List toForget, ShareRequestMetadata reqMetadata, Boolean isAcknowledgeDataPresent) { + public ShareFetchContext newContext( + String groupId, + List shareFetchData, + List toForget, + ShareRequestMetadata reqMetadata, + Boolean isAcknowledgeDataPresent, + String clientConnectionId + ) { ShareFetchContext context; // If the request's epoch is FINAL_EPOCH or INITIAL_EPOCH, we should remove the existing sessions. Also, start a // new session in case it is INITIAL_EPOCH. Hence, we need to treat them as special cases. @@ -448,7 +455,8 @@ public ShareFetchContext newContext(String groupId, List share ImplicitLinkedHashCollection<>(shareFetchData.size()); shareFetchData.forEach(topicIdPartition -> cachedSharePartitions.mustAdd(new CachedSharePartition(topicIdPartition, false))); - ShareSessionKey responseShareSessionKey = cache.maybeCreateSession(groupId, reqMetadata.memberId(), cachedSharePartitions); + ShareSessionKey responseShareSessionKey = cache.maybeCreateSession(groupId, reqMetadata.memberId(), + cachedSharePartitions, clientConnectionId); if (responseShareSessionKey == null) { log.error("Could not create a share session for group {} member {}", groupId, reqMetadata.memberId()); throw Errors.SHARE_SESSION_NOT_FOUND.exception(); diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 0f6b71d8092bd..ee1fbbca2da8f 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -21,7 +21,7 @@ import joptsimple._ import kafka.server.DynamicConfig import kafka.utils.Implicits._ import kafka.utils.Logging -import org.apache.kafka.clients.admin.{Admin, AlterClientQuotasOptions, AlterConfigOp, AlterConfigsOptions, ConfigEntry, DescribeClusterOptions, DescribeConfigsOptions, ListGroupsOptions, ListTopicsOptions, ScramCredentialInfo, UserScramCredentialDeletion, UserScramCredentialUpsertion, ScramMechanism => PublicScramMechanism} +import org.apache.kafka.clients.admin.{Admin, AlterClientQuotasOptions, AlterConfigOp, AlterConfigsOptions, ConfigEntry, DescribeClusterOptions, DescribeConfigsOptions, ListTopicsOptions, ScramCredentialInfo, UserScramCredentialDeletion, UserScramCredentialUpsertion, ScramMechanism => PublicScramMechanism} import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors.{InvalidConfigurationException, UnsupportedVersionException} import org.apache.kafka.common.internals.Topic @@ -352,7 +352,7 @@ object ConfigCommand extends Logging { case ClientMetricsType => adminClient.listClientMetricsResources().all().get().asScala.map(_.name).toSeq case GroupType => - adminClient.listGroups(ListGroupsOptions.forConsumerGroups()).all.get.asScala.map(_.groupId).toSeq + adminClient.listGroups().all.get.asScala.map(_.groupId).toSeq case entityType => throw new IllegalArgumentException(s"Invalid entity type: $entityType") }) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 1e19a37cb2a4f..930c08d607d4a 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -1730,7 +1730,7 @@ class Partition(val topicPartition: TopicPartition, case Some(epochAndOffset) => new EpochEndOffset() .setPartition(partitionId) .setErrorCode(Errors.NONE.code) - .setLeaderEpoch(epochAndOffset.leaderEpoch) + .setLeaderEpoch(epochAndOffset.epoch()) .setEndOffset(epochAndOffset.offset) case None => new EpochEndOffset() .setPartition(partitionId) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index 7130d39136e9f..a200cd3d6aec4 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.{AddPartitionsToTxnResponse, TransactionResult} import org.apache.kafka.common.utils.{LogContext, ProducerIdAndEpoch, Time} -import org.apache.kafka.coordinator.transaction.{ProducerIdManager, TransactionLogConfig, TransactionStateManagerConfig} +import org.apache.kafka.coordinator.transaction.{ProducerIdManager, TransactionLogConfig, TransactionState, TransactionStateManagerConfig} import org.apache.kafka.metadata.MetadataCache import org.apache.kafka.server.common.{RequestLocal, TransactionVersion} import org.apache.kafka.server.util.Scheduler @@ -154,7 +154,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig, producerEpoch = RecordBatch.NO_PRODUCER_EPOCH, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = resolvedTxnTimeoutMs, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = collection.mutable.Set.empty[TopicPartition], txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TransactionVersion.TV_0) @@ -182,7 +182,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig, responseCallback(initTransactionError(error)) case Right((coordinatorEpoch, newMetadata)) => - if (newMetadata.txnState == PrepareEpochFence) { + if (newMetadata.txnState == TransactionState.PREPARE_EPOCH_FENCE) { // abort the ongoing transaction and then return CONCURRENT_TRANSACTIONS to let client wait and retry def sendRetriableErrorCallback(error: Errors, newProducerId: Long, newProducerEpoch: Short): Unit = { if (error != Errors.NONE) { @@ -249,11 +249,11 @@ class TransactionCoordinator(txnConfig: TransactionConfig, } else { // caller should have synchronized on txnMetadata already txnMetadata.state match { - case PrepareAbort | PrepareCommit => + case TransactionState.PREPARE_ABORT | TransactionState.PREPARE_COMMIT => // reply to client and let it backoff and retry Left(Errors.CONCURRENT_TRANSACTIONS) - case CompleteAbort | CompleteCommit | Empty => + case TransactionState.COMPLETE_ABORT | TransactionState.COMPLETE_COMMIT | TransactionState.EMPTY => val transitMetadataResult = // If the epoch is exhausted and the expected epoch (if provided) matches it, generate a new producer ID if (txnMetadata.isProducerEpochExhausted && @@ -274,7 +274,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig, case Left(err) => Left(err) } - case Ongoing => + case TransactionState.ONGOING => // indicate to abort the current ongoing txn first. Note that this epoch is never returned to the // user. We will abort the ongoing transaction and return CONCURRENT_TRANSACTIONS to the client. // This forces the client to retry, which will ensure that the epoch is bumped a second time. In @@ -282,7 +282,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig, // then when the client retries, we will generate a new producerId. Right(coordinatorEpoch, txnMetadata.prepareFenceProducerEpoch()) - case Dead | PrepareEpochFence => + case TransactionState.DEAD | TransactionState.PREPARE_EPOCH_FENCE => val errorMsg = s"Found transactionalId $transactionalId with state ${txnMetadata.state}. " + s"This is illegal as we should never have transitioned to this state." fatal(errorMsg) @@ -294,12 +294,13 @@ class TransactionCoordinator(txnConfig: TransactionConfig, def handleListTransactions( filteredProducerIds: Set[Long], filteredStates: Set[String], - filteredDuration: Long = -1L + filteredDuration: Long = -1L, + filteredTransactionalIdPattern: String = null ): ListTransactionsResponseData = { if (!isActive.get()) { new ListTransactionsResponseData().setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code) } else { - txnManager.listTransactionStates(filteredProducerIds, filteredStates, filteredDuration) + txnManager.listTransactionStates(filteredProducerIds, filteredStates, filteredDuration, filteredTransactionalIdPattern) } } @@ -326,7 +327,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig, case Right(Some(coordinatorEpochAndMetadata)) => val txnMetadata = coordinatorEpochAndMetadata.transactionMetadata txnMetadata.inLock { - if (txnMetadata.state == Dead) { + if (txnMetadata.state == TransactionState.DEAD) { // The transaction state is being expired, so ignore it transactionState.setErrorCode(Errors.TRANSACTIONAL_ID_NOT_FOUND.code) } else { @@ -344,7 +345,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig, .setErrorCode(Errors.NONE.code) .setProducerId(txnMetadata.producerId) .setProducerEpoch(txnMetadata.producerEpoch) - .setTransactionState(txnMetadata.state.name) + .setTransactionState(txnMetadata.state.stateName) .setTransactionTimeoutMs(txnMetadata.txnTimeoutMs) .setTransactionStartTimeMs(txnMetadata.txnStartTimestamp) } @@ -377,7 +378,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig, Left(Errors.INVALID_PRODUCER_ID_MAPPING) } else if (txnMetadata.producerEpoch != producerEpoch) { Left(Errors.PRODUCER_FENCED) - } else if (txnMetadata.state == PrepareCommit || txnMetadata.state == PrepareAbort) { + } else if (txnMetadata.state == TransactionState.PREPARE_COMMIT || txnMetadata.state == TransactionState.PREPARE_ABORT) { Left(Errors.CONCURRENT_TRANSACTIONS) } else { Right(partitions.map { part => @@ -434,9 +435,9 @@ class TransactionCoordinator(txnConfig: TransactionConfig, Left(Errors.INVALID_PRODUCER_ID_MAPPING) } else if (txnMetadata.producerEpoch != producerEpoch) { Left(Errors.PRODUCER_FENCED) - } else if (txnMetadata.state == PrepareCommit || txnMetadata.state == PrepareAbort) { + } else if (txnMetadata.state == TransactionState.PREPARE_COMMIT || txnMetadata.state == TransactionState.PREPARE_ABORT) { Left(Errors.CONCURRENT_TRANSACTIONS) - } else if (txnMetadata.state == Ongoing && partitions.subsetOf(txnMetadata.topicPartitions)) { + } else if (txnMetadata.state == TransactionState.ONGOING && partitions.subsetOf(txnMetadata.topicPartitions)) { // this is an optimization: if the partitions are already in the metadata reply OK immediately Left(Errors.NONE) } else { @@ -554,16 +555,16 @@ class TransactionCoordinator(txnConfig: TransactionConfig, // Strict equality is enforced on the client side requests, as they shouldn't bump the producer epoch. else if ((isFromClient && producerEpoch != txnMetadata.producerEpoch) || producerEpoch < txnMetadata.producerEpoch) Left(Errors.PRODUCER_FENCED) - else if (txnMetadata.pendingTransitionInProgress && txnMetadata.pendingState.get != PrepareEpochFence) + else if (txnMetadata.pendingTransitionInProgress && txnMetadata.pendingState.get != TransactionState.PREPARE_EPOCH_FENCE) Left(Errors.CONCURRENT_TRANSACTIONS) else txnMetadata.state match { - case Ongoing => + case TransactionState.ONGOING => val nextState = if (txnMarkerResult == TransactionResult.COMMIT) - PrepareCommit + TransactionState.PREPARE_COMMIT else - PrepareAbort + TransactionState.PREPARE_ABORT - if (nextState == PrepareAbort && txnMetadata.pendingState.contains(PrepareEpochFence)) { + if (nextState == TransactionState.PREPARE_ABORT && txnMetadata.pendingState.contains(TransactionState.PREPARE_EPOCH_FENCE)) { // We should clear the pending state to make way for the transition to PrepareAbort and also bump // the epoch in the transaction metadata we are about to append. isEpochFence = true @@ -573,29 +574,29 @@ class TransactionCoordinator(txnConfig: TransactionConfig, } Right(coordinatorEpoch, txnMetadata.prepareAbortOrCommit(nextState, TransactionVersion.fromFeatureLevel(0), RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false)) - case CompleteCommit => + case TransactionState.COMPLETE_COMMIT => if (txnMarkerResult == TransactionResult.COMMIT) Left(Errors.NONE) else logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) - case CompleteAbort => + case TransactionState.COMPLETE_ABORT => if (txnMarkerResult == TransactionResult.ABORT) Left(Errors.NONE) else logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) - case PrepareCommit => + case TransactionState.PREPARE_COMMIT => if (txnMarkerResult == TransactionResult.COMMIT) Left(Errors.CONCURRENT_TRANSACTIONS) else logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) - case PrepareAbort => + case TransactionState.PREPARE_ABORT => if (txnMarkerResult == TransactionResult.ABORT) Left(Errors.CONCURRENT_TRANSACTIONS) else logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) - case Empty => + case TransactionState.EMPTY => logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) - case Dead | PrepareEpochFence => + case TransactionState.DEAD | TransactionState.PREPARE_EPOCH_FENCE => val errorMsg = s"Found transactionalId $transactionalId with state ${txnMetadata.state}. " + s"This is illegal as we should never have transitioned to this state." fatal(errorMsg) @@ -630,19 +631,19 @@ class TransactionCoordinator(txnConfig: TransactionConfig, else if (txnMetadata.pendingTransitionInProgress) Left(Errors.CONCURRENT_TRANSACTIONS) else txnMetadata.state match { - case Empty| Ongoing | CompleteCommit | CompleteAbort => + case TransactionState.EMPTY| TransactionState.ONGOING | TransactionState.COMPLETE_COMMIT | TransactionState.COMPLETE_ABORT => logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) - case PrepareCommit => + case TransactionState.PREPARE_COMMIT => if (txnMarkerResult != TransactionResult.COMMIT) logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) else Right(txnMetadata, txnMetadata.prepareComplete(time.milliseconds())) - case PrepareAbort => + case TransactionState.PREPARE_ABORT => if (txnMarkerResult != TransactionResult.ABORT) logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) else Right(txnMetadata, txnMetadata.prepareComplete(time.milliseconds())) - case Dead | PrepareEpochFence => + case TransactionState.DEAD | TransactionState.PREPARE_EPOCH_FENCE => val errorMsg = s"Found transactionalId $transactionalId with state ${txnMetadata.state}. " + s"This is illegal as we should never have transitioned to this state." fatal(errorMsg) @@ -775,7 +776,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig, producerEpochCopy = txnMetadata.producerEpoch // PrepareEpochFence has slightly different epoch bumping logic so don't include it here. // Note that, it can only happen when the current state is Ongoing. - isEpochFence = txnMetadata.pendingState.contains(PrepareEpochFence) + isEpochFence = txnMetadata.pendingState.contains(TransactionState.PREPARE_EPOCH_FENCE) // True if the client retried a request that had overflowed the epoch, and a new producer ID is stored in the txnMetadata val retryOnOverflow = !isEpochFence && txnMetadata.prevProducerId == producerId && producerEpoch == Short.MaxValue - 1 && txnMetadata.producerEpoch == 0 @@ -789,11 +790,11 @@ class TransactionCoordinator(txnConfig: TransactionConfig, // Return producer fenced even in the cases where the epoch is higher and could indicate an invalid state transition. // Use the following criteria to determine if a v2 retry is valid: txnMetadata.state match { - case Ongoing | Empty | Dead | PrepareEpochFence => + case TransactionState.ONGOING | TransactionState.EMPTY | TransactionState.DEAD | TransactionState.PREPARE_EPOCH_FENCE => producerEpoch == txnMetadata.producerEpoch - case PrepareCommit | PrepareAbort => + case TransactionState.PREPARE_COMMIT | TransactionState.PREPARE_ABORT => retryOnEpochBump - case CompleteCommit | CompleteAbort => + case TransactionState.COMPLETE_COMMIT | TransactionState.COMPLETE_ABORT => retryOnEpochBump || retryOnOverflow || producerEpoch == txnMetadata.producerEpoch } } else { @@ -817,7 +818,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig, Right(RecordBatch.NO_PRODUCER_ID) } - if (nextState == PrepareAbort && isEpochFence) { + if (nextState == TransactionState.PREPARE_ABORT && isEpochFence) { // We should clear the pending state to make way for the transition to PrepareAbort and also bump // the epoch in the transaction metadata we are about to append. txnMetadata.pendingState = None @@ -831,7 +832,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig, } } - if (txnMetadata.pendingTransitionInProgress && txnMetadata.pendingState.get != PrepareEpochFence) { + if (txnMetadata.pendingTransitionInProgress && txnMetadata.pendingState.get != TransactionState.PREPARE_EPOCH_FENCE) { // This check is performed first so that the pending transition can complete before the next checks. // With TV2, we may be transitioning over a producer epoch overflow, and the producer may be using the // new producer ID that is still only in pending state. @@ -841,14 +842,14 @@ class TransactionCoordinator(txnConfig: TransactionConfig, else if (!isValidEpoch) Left(Errors.PRODUCER_FENCED) else txnMetadata.state match { - case Ongoing => + case TransactionState.ONGOING => val nextState = if (txnMarkerResult == TransactionResult.COMMIT) - PrepareCommit + TransactionState.PREPARE_COMMIT else - PrepareAbort + TransactionState.PREPARE_ABORT generateTxnTransitMetadataForTxnCompletion(nextState, false) - case CompleteCommit => + case TransactionState.COMPLETE_COMMIT => if (txnMarkerResult == TransactionResult.COMMIT) { if (isRetry) Left(Errors.NONE) @@ -859,35 +860,35 @@ class TransactionCoordinator(txnConfig: TransactionConfig, if (isRetry) logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) else - generateTxnTransitMetadataForTxnCompletion(PrepareAbort, true) + generateTxnTransitMetadataForTxnCompletion(TransactionState.PREPARE_ABORT, true) } - case CompleteAbort => + case TransactionState.COMPLETE_ABORT => if (txnMarkerResult == TransactionResult.ABORT) { if (isRetry) Left(Errors.NONE) else - generateTxnTransitMetadataForTxnCompletion(PrepareAbort, true) + generateTxnTransitMetadataForTxnCompletion(TransactionState.PREPARE_ABORT, true) } else { // Commit. logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) } - case PrepareCommit => + case TransactionState.PREPARE_COMMIT => if (txnMarkerResult == TransactionResult.COMMIT) Left(Errors.CONCURRENT_TRANSACTIONS) else logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) - case PrepareAbort => + case TransactionState.PREPARE_ABORT => if (txnMarkerResult == TransactionResult.ABORT) Left(Errors.CONCURRENT_TRANSACTIONS) else logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) - case Empty => + case TransactionState.EMPTY => if (txnMarkerResult == TransactionResult.ABORT) { - generateTxnTransitMetadataForTxnCompletion(PrepareAbort, true) + generateTxnTransitMetadataForTxnCompletion(TransactionState.PREPARE_ABORT, true) } else { logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) } - case Dead | PrepareEpochFence => + case TransactionState.DEAD | TransactionState.PREPARE_EPOCH_FENCE => val errorMsg = s"Found transactionalId $transactionalId with state ${txnMetadata.state}. " + s"This is illegal as we should never have transitioned to this state." fatal(errorMsg) @@ -927,19 +928,19 @@ class TransactionCoordinator(txnConfig: TransactionConfig, else if (txnMetadata.pendingTransitionInProgress) Left(Errors.CONCURRENT_TRANSACTIONS) else txnMetadata.state match { - case Empty| Ongoing | CompleteCommit | CompleteAbort => + case TransactionState.EMPTY | TransactionState.ONGOING | TransactionState.COMPLETE_COMMIT | TransactionState.COMPLETE_ABORT => logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) - case PrepareCommit => + case TransactionState.PREPARE_COMMIT => if (txnMarkerResult != TransactionResult.COMMIT) logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) else Right(txnMetadata, txnMetadata.prepareComplete(time.milliseconds())) - case PrepareAbort => + case TransactionState.PREPARE_ABORT => if (txnMarkerResult != TransactionResult.ABORT) logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult) else Right(txnMetadata, txnMetadata.prepareComplete(time.milliseconds())) - case Dead | PrepareEpochFence => + case TransactionState.DEAD | TransactionState.PREPARE_EPOCH_FENCE => val errorMsg = s"Found transactionalId $transactionalId with state ${txnMetadata.state}. " + s"This is illegal as we should never have transitioned to this state." fatal(errorMsg) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala index 5972418d0c1e2..631a432d3be09 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala @@ -21,6 +21,7 @@ import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.protocol.{ByteBufferAccessor, MessageUtil} import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.TopicPartition +import org.apache.kafka.coordinator.transaction.TransactionState import org.apache.kafka.coordinator.transaction.generated.{CoordinatorRecordType, TransactionLogKey, TransactionLogValue} import org.apache.kafka.server.common.TransactionVersion @@ -61,10 +62,10 @@ object TransactionLog { */ private[transaction] def valueToBytes(txnMetadata: TxnTransitMetadata, transactionVersionLevel: TransactionVersion): Array[Byte] = { - if (txnMetadata.txnState == Empty && txnMetadata.topicPartitions.nonEmpty) + if (txnMetadata.txnState == TransactionState.EMPTY && txnMetadata.topicPartitions.nonEmpty) throw new IllegalStateException(s"Transaction is not expected to have any partitions since its state is ${txnMetadata.txnState}: $txnMetadata") - val transactionPartitions = if (txnMetadata.txnState == Empty) null + val transactionPartitions = if (txnMetadata.txnState == TransactionState.EMPTY) null else txnMetadata.topicPartitions .groupBy(_.topic) .map { case (topic, partitions) => @@ -127,7 +128,7 @@ object TransactionLog { txnLastUpdateTimestamp = value.transactionLastUpdateTimestampMs, clientTransactionVersion = TransactionVersion.fromFeatureLevel(value.clientTransactionVersion)) - if (!transactionMetadata.state.equals(Empty)) + if (!transactionMetadata.state.equals(TransactionState.EMPTY)) value.transactionPartitions.forEach(partitionsSchema => transactionMetadata.addPartitions(partitionsSchema.partitionIds .asScala diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala index aff6874951386..aa8c871b7def4 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala @@ -21,151 +21,11 @@ import kafka.utils.{CoreUtils, Logging, nonthreadsafe} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch +import org.apache.kafka.coordinator.transaction.TransactionState import org.apache.kafka.server.common.TransactionVersion import scala.collection.{immutable, mutable} - -object TransactionState { - val AllStates: Set[TransactionState] = Set( - Empty, - Ongoing, - PrepareCommit, - PrepareAbort, - CompleteCommit, - CompleteAbort, - Dead, - PrepareEpochFence - ) - - def fromName(name: String): Option[TransactionState] = { - AllStates.find(_.name == name) - } - - def fromId(id: Byte): TransactionState = { - id match { - case 0 => Empty - case 1 => Ongoing - case 2 => PrepareCommit - case 3 => PrepareAbort - case 4 => CompleteCommit - case 5 => CompleteAbort - case 6 => Dead - case 7 => PrepareEpochFence - case _ => throw new IllegalStateException(s"Unknown transaction state id $id from the transaction status message") - } - } -} - -private[transaction] sealed trait TransactionState { - def id: Byte - - /** - * Get the name of this state. This is exposed through the `DescribeTransactions` API. - */ - def name: String - - def validPreviousStates: Set[TransactionState] - - def isExpirationAllowed: Boolean = false -} - -/** - * Transaction has not existed yet - * - * transition: received AddPartitionsToTxnRequest => Ongoing - * received AddOffsetsToTxnRequest => Ongoing - * received EndTxnRequest with abort and TransactionV2 enabled => PrepareAbort - */ -private[transaction] case object Empty extends TransactionState { - val id: Byte = 0 - val name: String = "Empty" - val validPreviousStates: Set[TransactionState] = Set(Empty, CompleteCommit, CompleteAbort) - override def isExpirationAllowed: Boolean = true -} - -/** - * Transaction has started and ongoing - * - * transition: received EndTxnRequest with commit => PrepareCommit - * received EndTxnRequest with abort => PrepareAbort - * received AddPartitionsToTxnRequest => Ongoing - * received AddOffsetsToTxnRequest => Ongoing - */ -private[transaction] case object Ongoing extends TransactionState { - val id: Byte = 1 - val name: String = "Ongoing" - val validPreviousStates: Set[TransactionState] = Set(Ongoing, Empty, CompleteCommit, CompleteAbort) -} - -/** - * Group is preparing to commit - * - * transition: received acks from all partitions => CompleteCommit - */ -private[transaction] case object PrepareCommit extends TransactionState { - val id: Byte = 2 - val name: String = "PrepareCommit" - val validPreviousStates: Set[TransactionState] = Set(Ongoing) -} - -/** - * Group is preparing to abort - * - * transition: received acks from all partitions => CompleteAbort - * - * Note, In transaction v2, we allow Empty, CompleteCommit, CompleteAbort to transition to PrepareAbort. because the - * client may not know the txn state on the server side, it needs to send endTxn request when uncertain. - */ -private[transaction] case object PrepareAbort extends TransactionState { - val id: Byte = 3 - val name: String = "PrepareAbort" - val validPreviousStates: Set[TransactionState] = Set(Ongoing, PrepareEpochFence, Empty, CompleteCommit, CompleteAbort) -} - -/** - * Group has completed commit - * - * Will soon be removed from the ongoing transaction cache - */ -private[transaction] case object CompleteCommit extends TransactionState { - val id: Byte = 4 - val name: String = "CompleteCommit" - val validPreviousStates: Set[TransactionState] = Set(PrepareCommit) - override def isExpirationAllowed: Boolean = true -} - -/** - * Group has completed abort - * - * Will soon be removed from the ongoing transaction cache - */ -private[transaction] case object CompleteAbort extends TransactionState { - val id: Byte = 5 - val name: String = "CompleteAbort" - val validPreviousStates: Set[TransactionState] = Set(PrepareAbort) - override def isExpirationAllowed: Boolean = true -} - -/** - * TransactionalId has expired and is about to be removed from the transaction cache - */ -private[transaction] case object Dead extends TransactionState { - val id: Byte = 6 - val name: String = "Dead" - val validPreviousStates: Set[TransactionState] = Set(Empty, CompleteAbort, CompleteCommit) -} - -/** - * We are in the middle of bumping the epoch and fencing out older producers. - */ - -private[transaction] case object PrepareEpochFence extends TransactionState { - val id: Byte = 7 - val name: String = "PrepareEpochFence" - val validPreviousStates: Set[TransactionState] = Set(Ongoing) -} - private[transaction] object TransactionMetadata { def isEpochExhausted(producerEpoch: Short): Boolean = producerEpoch >= Short.MaxValue - 1 } @@ -244,7 +104,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, } def removePartition(topicPartition: TopicPartition): Unit = { - if (state != PrepareCommit && state != PrepareAbort) + if (state != TransactionState.PREPARE_COMMIT && state != TransactionState.PREPARE_ABORT) throw new IllegalStateException(s"Transaction metadata's current state is $state, and its pending state is $pendingState " + s"while trying to remove partitions whose txn marker has been sent, this is not expected") @@ -267,7 +127,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, val bumpedEpoch = if (hasFailedEpochFence) producerEpoch else (producerEpoch + 1).toShort prepareTransitionTo( - state = PrepareEpochFence, + state = TransactionState.PREPARE_EPOCH_FENCE, producerEpoch = bumpedEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH ) @@ -309,7 +169,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, epochBumpResult match { case Right((nextEpoch, lastEpoch)) => Right(prepareTransitionTo( - state = Empty, + state = TransactionState.EMPTY, producerEpoch = nextEpoch, lastProducerEpoch = lastEpoch, txnTimeoutMs = newTxnTimeoutMs, @@ -330,7 +190,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, throw new IllegalStateException("Cannot rotate producer ids while a transaction is still pending") prepareTransitionTo( - state = Empty, + state = TransactionState.EMPTY, producerId = newProducerId, producerEpoch = 0, lastProducerEpoch = if (recordLastEpoch) producerEpoch else RecordBatch.NO_PRODUCER_EPOCH, @@ -343,12 +203,12 @@ private[transaction] class TransactionMetadata(val transactionalId: String, def prepareAddPartitions(addedTopicPartitions: immutable.Set[TopicPartition], updateTimestamp: Long, clientTransactionVersion: TransactionVersion): TxnTransitMetadata = { val newTxnStartTimestamp = state match { - case Empty | CompleteAbort | CompleteCommit => updateTimestamp + case TransactionState.EMPTY | TransactionState.COMPLETE_ABORT | TransactionState.COMPLETE_COMMIT => updateTimestamp case _ => txnStartTimestamp } prepareTransitionTo( - state = Ongoing, + state = TransactionState.ONGOING, topicPartitions = (topicPartitions ++ addedTopicPartitions), txnStartTimestamp = newTxnStartTimestamp, txnLastUpdateTimestamp = updateTimestamp, @@ -379,7 +239,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, } def prepareComplete(updateTimestamp: Long): TxnTransitMetadata = { - val newState = if (state == PrepareCommit) CompleteCommit else CompleteAbort + val newState = if (state == TransactionState.PREPARE_COMMIT) TransactionState.COMPLETE_COMMIT else TransactionState.COMPLETE_ABORT // Since the state change was successfully written to the log, unset the flag for a failed epoch fence hasFailedEpochFence = false @@ -408,7 +268,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, def prepareDead(): TxnTransitMetadata = { prepareTransitionTo( - state = Dead, + state = TransactionState.DEAD, topicPartitions = mutable.Set.empty[TopicPartition] ) } @@ -427,7 +287,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, private def hasPendingTransaction: Boolean = { state match { - case Ongoing | PrepareAbort | PrepareCommit => true + case TransactionState.ONGOING | TransactionState.PREPARE_ABORT | TransactionState.PREPARE_COMMIT => true case _ => false } } @@ -452,7 +312,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, // The epoch is initialized to NO_PRODUCER_EPOCH when the TransactionMetadata // is created for the first time and it could stay like this until transitioning // to Dead. - if (state != Dead && producerEpoch < 0) + if (state != TransactionState.DEAD && producerEpoch < 0) throw new IllegalArgumentException(s"Illegal new producer epoch $producerEpoch") // check that the new state transition is valid and update the pending state if necessary @@ -492,7 +352,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, throwStateTransitionFailure(transitMetadata) } else { toState match { - case Empty => // from initPid + case TransactionState.EMPTY => // from initPid if ((producerEpoch != transitMetadata.producerEpoch && !validProducerEpochBump(transitMetadata)) || transitMetadata.topicPartitions.nonEmpty || transitMetadata.txnStartTimestamp != -1) { @@ -500,7 +360,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, throwStateTransitionFailure(transitMetadata) } - case Ongoing => // from addPartitions + case TransactionState.ONGOING => // from addPartitions if (!validProducerEpoch(transitMetadata) || !topicPartitions.subsetOf(transitMetadata.topicPartitions) || txnTimeoutMs != transitMetadata.txnTimeoutMs) { @@ -508,11 +368,11 @@ private[transaction] class TransactionMetadata(val transactionalId: String, throwStateTransitionFailure(transitMetadata) } - case PrepareAbort | PrepareCommit => // from endTxn + case TransactionState.PREPARE_ABORT | TransactionState.PREPARE_COMMIT => // from endTxn // In V2, we allow state transits from Empty, CompleteCommit and CompleteAbort to PrepareAbort. It is possible // their updated start time is not equal to the current start time. - val allowedEmptyAbort = toState == PrepareAbort && transitMetadata.clientTransactionVersion.supportsEpochBump() && - (state == Empty || state == CompleteCommit || state == CompleteAbort) + val allowedEmptyAbort = toState == TransactionState.PREPARE_ABORT && transitMetadata.clientTransactionVersion.supportsEpochBump() && + (state == TransactionState.EMPTY || state == TransactionState.COMPLETE_COMMIT || state == TransactionState.COMPLETE_ABORT) val validTimestamp = txnStartTimestamp == transitMetadata.txnStartTimestamp || allowedEmptyAbort if (!validProducerEpoch(transitMetadata) || !topicPartitions.equals(transitMetadata.topicPartitions) || @@ -521,14 +381,14 @@ private[transaction] class TransactionMetadata(val transactionalId: String, throwStateTransitionFailure(transitMetadata) } - case CompleteAbort | CompleteCommit => // from write markers + case TransactionState.COMPLETE_ABORT | TransactionState.COMPLETE_COMMIT => // from write markers if (!validProducerEpoch(transitMetadata) || txnTimeoutMs != transitMetadata.txnTimeoutMs || transitMetadata.txnStartTimestamp == -1) { throwStateTransitionFailure(transitMetadata) } - case PrepareEpochFence => + case TransactionState.PREPARE_EPOCH_FENCE => // We should never get here, since once we prepare to fence the epoch, we immediately set the pending state // to PrepareAbort, and then consequently to CompleteAbort after the markers are written.. So we should never // ever try to complete a transition to PrepareEpochFence, as it is not a valid previous state for any other state, and hence @@ -536,7 +396,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, throwStateTransitionFailure(transitMetadata) - case Dead => + case TransactionState.DEAD => // The transactionalId was being expired. The completion of the operation should result in removal of the // the metadata from the cache, so we should never realistically transition to the dead state. throw new IllegalStateException(s"TransactionalId $transactionalId is trying to complete a transition to " + @@ -590,11 +450,11 @@ private[transaction] class TransactionMetadata(val transactionalId: String, val transitLastProducerEpoch = transitMetadata.lastProducerEpoch (isAtLeastTransactionsV2, txnState, transitProducerEpoch) match { - case (true, CompleteCommit | CompleteAbort, epoch) if epoch == 0.toShort => + case (true, TransactionState.COMPLETE_COMMIT | TransactionState.COMPLETE_ABORT, epoch) if epoch == 0.toShort => transitLastProducerEpoch == lastProducerEpoch && transitMetadata.prevProducerId == producerId - case (true, PrepareCommit | PrepareAbort, _) => + case (true, TransactionState.PREPARE_COMMIT | TransactionState.PREPARE_ABORT, _) => transitLastProducerEpoch == producerEpoch && transitProducerId == producerId diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index 3982827254633..b859ed003ad34 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -35,7 +35,7 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicIdPartition, TopicPartition} -import org.apache.kafka.coordinator.transaction.{TransactionLogConfig, TransactionStateManagerConfig} +import org.apache.kafka.coordinator.transaction.{TransactionLogConfig, TransactionState, TransactionStateManagerConfig} import org.apache.kafka.metadata.MetadataCache import org.apache.kafka.server.common.{RequestLocal, TransactionVersion} import org.apache.kafka.server.config.ServerConfigs @@ -43,6 +43,8 @@ import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.Scheduler import org.apache.kafka.storage.internals.log.AppendOrigin +import com.google.re2j.{Pattern, PatternSyntaxException} +import org.apache.kafka.common.errors.InvalidRegularExpression import scala.jdk.CollectionConverters._ import scala.collection.mutable @@ -132,7 +134,7 @@ class TransactionStateManager(brokerId: Int, false } else { txnMetadata.state match { - case Ongoing => + case TransactionState.ONGOING => // Do not apply timeout to distributed two phase commit transactions. (!txnMetadata.isDistributedTwoPhaseCommitTxn) && (txnMetadata.txnStartTimestamp + txnMetadata.txnTimeoutMs < now) @@ -263,7 +265,7 @@ class TransactionStateManager(brokerId: Int, val txnMetadata = txnMetadataCacheEntry.metadataPerTransactionalId.get(transactionalId) txnMetadata.inLock { if (txnMetadataCacheEntry.coordinatorEpoch == idCoordinatorEpochAndMetadata.coordinatorEpoch - && txnMetadata.pendingState.contains(Dead) + && txnMetadata.pendingState.contains(TransactionState.DEAD) && txnMetadata.producerEpoch == idCoordinatorEpochAndMetadata.transitMetadata.producerEpoch && response.error == Errors.NONE) { txnMetadataCacheEntry.metadataPerTransactionalId.remove(transactionalId) @@ -316,7 +318,8 @@ class TransactionStateManager(brokerId: Int, def listTransactionStates( filterProducerIds: Set[Long], filterStateNames: Set[String], - filterDurationMs: Long + filterDurationMs: Long, + filterTransactionalIdPattern: String ): ListTransactionsResponseData = { inReadLock(stateLock) { val response = new ListTransactionsResponseData() @@ -325,15 +328,15 @@ class TransactionStateManager(brokerId: Int, } else { val filterStates = mutable.Set.empty[TransactionState] filterStateNames.foreach { stateName => - TransactionState.fromName(stateName) match { - case Some(state) => filterStates += state - case None => response.unknownStateFilters.add(stateName) - } + TransactionState.fromName(stateName).ifPresentOrElse( + state => filterStates += state, + () => response.unknownStateFilters.add(stateName) + ) } val now : Long = time.milliseconds() - def shouldInclude(txnMetadata: TransactionMetadata): Boolean = { - if (txnMetadata.state == Dead) { + def shouldInclude(txnMetadata: TransactionMetadata, pattern: Pattern): Boolean = { + if (txnMetadata.state == TransactionState.DEAD) { // We filter the `Dead` state since it is a transient state which // indicates that the transactionalId and its metadata are in the // process of expiration and removal. @@ -344,20 +347,31 @@ class TransactionStateManager(brokerId: Int, false } else if (filterDurationMs >= 0 && (now - txnMetadata.txnStartTimestamp) <= filterDurationMs) { false + } else if (pattern != null) { + pattern.matcher(txnMetadata.transactionalId).matches() } else { true } } val states = new java.util.ArrayList[ListTransactionsResponseData.TransactionState] + val pattern = if (filterTransactionalIdPattern != null && filterTransactionalIdPattern.nonEmpty) { + try { + Pattern.compile(filterTransactionalIdPattern) + } + catch { + case e: PatternSyntaxException => + throw new InvalidRegularExpression(String.format("Transaction ID pattern `%s` is not a valid regular expression: %s.", filterTransactionalIdPattern, e.getMessage)) + } + } else null transactionMetadataCache.foreachEntry { (_, cache) => cache.metadataPerTransactionalId.forEach { (_, txnMetadata) => txnMetadata.inLock { - if (shouldInclude(txnMetadata)) { + if (shouldInclude(txnMetadata, pattern)) { states.add(new ListTransactionsResponseData.TransactionState() .setTransactionalId(txnMetadata.transactionalId) .setProducerId(txnMetadata.producerId) - .setTransactionState(txnMetadata.state.name) + .setTransactionState(txnMetadata.state.stateName) ) } } @@ -554,10 +568,10 @@ class TransactionStateManager(brokerId: Int, txnMetadata.inLock { // if state is PrepareCommit or PrepareAbort we need to complete the transaction txnMetadata.state match { - case PrepareAbort => + case TransactionState.PREPARE_ABORT => transactionsPendingForCompletion += TransactionalIdCoordinatorEpochAndTransitMetadata(transactionalId, coordinatorEpoch, TransactionResult.ABORT, txnMetadata, txnMetadata.prepareComplete(time.milliseconds())) - case PrepareCommit => + case TransactionState.PREPARE_COMMIT => transactionsPendingForCompletion += TransactionalIdCoordinatorEpochAndTransitMetadata(transactionalId, coordinatorEpoch, TransactionResult.COMMIT, txnMetadata, txnMetadata.prepareComplete(time.milliseconds())) case _ => diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index db4841e3d22c9..f7b776f1336c2 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -26,9 +26,11 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.InvalidConfigurationException import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record.{MemoryRecords, Records} +import org.apache.kafka.common.utils.LogContext import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} -import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} +import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} +import org.apache.kafka.server.common.OffsetAndEpoch import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.Scheduler @@ -139,14 +141,14 @@ final class KafkaMetadataLog private ( (log.endOffsetForEpoch(epoch).toScala, earliestSnapshotId().toScala) match { case (Some(offsetAndEpoch), Some(snapshotId)) if ( offsetAndEpoch.offset == snapshotId.offset && - offsetAndEpoch.leaderEpoch == epoch) => + offsetAndEpoch.epoch() == epoch) => // The epoch is smaller than the smallest epoch on the log. Override the diverging // epoch to the oldest snapshot which should be the snapshot at the log start offset new OffsetAndEpoch(snapshotId.offset, snapshotId.epoch) case (Some(offsetAndEpoch), _) => - new OffsetAndEpoch(offsetAndEpoch.offset, offsetAndEpoch.leaderEpoch) + new OffsetAndEpoch(offsetAndEpoch.offset, offsetAndEpoch.epoch()) case (None, _) => new OffsetAndEpoch(endOffset.offset, lastFetchedEpoch) @@ -417,7 +419,7 @@ final class KafkaMetadataLog private ( */ private def readSnapshotTimestamp(snapshotId: OffsetAndEpoch): Option[Long] = { readSnapshot(snapshotId).toScala.map { reader => - Snapshots.lastContainedLogTimestamp(reader) + Snapshots.lastContainedLogTimestamp(reader, new LogContext(logIdent)) } } @@ -588,10 +590,7 @@ object KafkaMetadataLog extends Logging { val props = new Properties() props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString) props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) - if (config.internalLogSegmentBytes != null) - props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) - else - 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 @@ -600,7 +599,11 @@ object KafkaMetadataLog extends Logging { LogConfig.validate(props) val defaultLogConfig = new LogConfig(props) - if (defaultLogConfig.retentionMs >= 0) { + if (config.logSegmentBytes < config.logSegmentMinBytes) { + throw new InvalidConfigurationException( + s"Cannot set ${MetadataLogConfig.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}." ) @@ -636,6 +639,12 @@ object KafkaMetadataLog extends Logging { nodeId ) + // Print a warning if users have overridden the internal config + if (config.logSegmentMinBytes != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) { + metadataLog.error(s"Overriding ${MetadataLogConfig.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.") + } + // 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() diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index a65464706c2a5..06cc94b5979b4 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -619,7 +619,7 @@ abstract class AbstractFetcherThread(name: String, if (endOffsetForEpochOpt.isPresent) { val offsetAndEpoch = endOffsetForEpochOpt.get val followerEndOffset = offsetAndEpoch.offset - val followerEpoch = offsetAndEpoch.leaderEpoch + val followerEpoch = offsetAndEpoch.epoch() if (followerEpoch != leaderEpochOffset.leaderEpoch) { // the follower does not know about the epoch that leader replied with // we truncate to the end offset of the largest epoch that is smaller than the diff --git a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala index a2c2bd4d80b35..0cd8e230c9daa 100644 --- a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala +++ b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala @@ -59,7 +59,7 @@ class DefaultAutoTopicCreationManager( channelManager: NodeToControllerChannelManager, groupCoordinator: GroupCoordinator, txnCoordinator: TransactionCoordinator, - shareCoordinator: Option[ShareCoordinator] + shareCoordinator: ShareCoordinator ) extends AutoTopicCreationManager with Logging { private val inflightTopics = Collections.newSetFromMap(new ConcurrentHashMap[String, java.lang.Boolean]()) @@ -198,15 +198,11 @@ class DefaultAutoTopicCreationManager( .setConfigs(convertToTopicConfigCollections( txnCoordinator.transactionTopicConfigs)) case SHARE_GROUP_STATE_TOPIC_NAME => - val props = shareCoordinator match { - case Some(coordinator) => coordinator.shareGroupStateTopicConfigs() - case None => new Properties() - } new CreatableTopic() .setName(topic) .setNumPartitions(config.shareCoordinatorConfig.shareCoordinatorStateTopicNumPartitions()) .setReplicationFactor(config.shareCoordinatorConfig.shareCoordinatorStateTopicReplicationFactor()) - .setConfigs(convertToTopicConfigCollections(props)) + .setConfigs(convertToTopicConfigCollections(shareCoordinator.shareGroupStateTopicConfigs())) case topicName => new CreatableTopic() .setName(topicName) diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 172c02ae924bd..e2d41cac3bb1a 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -126,7 +126,7 @@ class BrokerServer( var transactionCoordinator: TransactionCoordinator = _ - var shareCoordinator: Option[ShareCoordinator] = None + var shareCoordinator: ShareCoordinator = _ var clientToControllerChannelManager: NodeToControllerChannelManager = _ @@ -197,7 +197,7 @@ class BrokerServer( config.dynamicConfig.initialize(Some(clientMetricsReceiverPlugin)) quotaManagers = QuotaFactory.instantiate(config, metrics, time, s"broker-${config.nodeId}-", ProcessRole.BrokerRole.toString) - DynamicBrokerConfig.readDynamicBrokerConfigsFromSnapshot(raftManager, config, quotaManagers) + DynamicBrokerConfig.readDynamicBrokerConfigsFromSnapshot(raftManager, config, quotaManagers, logContext) /* start scheduler */ kafkaScheduler = new KafkaScheduler(config.backgroundThreads) @@ -259,7 +259,13 @@ class BrokerServer( Optional.of(clientMetricsManager) ) - val connectionDisconnectListeners = Seq(clientMetricsManager.connectionDisconnectListener()) + val shareFetchSessionCache : ShareSessionCache = new ShareSessionCache(config.shareGroupConfig.shareGroupMaxShareSessions()) + + val connectionDisconnectListeners = Seq( + clientMetricsManager.connectionDisconnectListener(), + shareFetchSessionCache.connectionDisconnectListener() + ) + // Create and start the socket server acceptor threads so that the bound port is known. // Delay starting processors until the end of the initialization sequence to ensure // that credentials have been loaded before processing authentications. @@ -426,8 +432,6 @@ class BrokerServer( )) val fetchManager = new FetchManager(Time.SYSTEM, new FetchSessionCache(fetchSessionCacheShards)) - val shareFetchSessionCache : ShareSessionCache = new ShareSessionCache(config.shareGroupConfig.shareGroupMaxShareSessions()) - sharePartitionManager = new SharePartitionManager( replicaManager, time, @@ -629,41 +633,35 @@ class BrokerServer( .build() } - private def createShareCoordinator(): Option[ShareCoordinator] = { - if (config.shareGroupConfig.isShareGroupEnabled && - config.shareGroupConfig.shareGroupPersisterClassName().nonEmpty) { - val time = Time.SYSTEM - val timer = new SystemTimerReaper( - "share-coordinator-reaper", - new SystemTimer("share-coordinator") - ) + private def createShareCoordinator(): ShareCoordinator = { + val time = Time.SYSTEM + val timer = new SystemTimerReaper( + "share-coordinator-reaper", + new SystemTimer("share-coordinator") + ) - val serde = new ShareCoordinatorRecordSerde - val loader = new CoordinatorLoaderImpl[CoordinatorRecord]( - time, - replicaManager, - serde, - config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize() - ) - val writer = new CoordinatorPartitionWriter( - replicaManager - ) - Some(new ShareCoordinatorService.Builder(config.brokerId, config.shareCoordinatorConfig) - .withTimer(timer) - .withTime(time) - .withLoader(loader) - .withWriter(writer) - .withCoordinatorRuntimeMetrics(new ShareCoordinatorRuntimeMetrics(metrics)) - .withCoordinatorMetrics(new ShareCoordinatorMetrics(metrics)) - .build()) - } else { - None - } + val serde = new ShareCoordinatorRecordSerde + val loader = new CoordinatorLoaderImpl[CoordinatorRecord]( + time, + replicaManager, + serde, + config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize() + ) + val writer = new CoordinatorPartitionWriter( + replicaManager + ) + new ShareCoordinatorService.Builder(config.brokerId, config.shareCoordinatorConfig) + .withTimer(timer) + .withTime(time) + .withLoader(loader) + .withWriter(writer) + .withCoordinatorRuntimeMetrics(new ShareCoordinatorRuntimeMetrics(metrics)) + .withCoordinatorMetrics(new ShareCoordinatorMetrics(metrics)) + .build() } private def createShareStatePersister(): Persister = { - if (config.shareGroupConfig.isShareGroupEnabled && - config.shareGroupConfig.shareGroupPersisterClassName.nonEmpty) { + if (config.shareGroupConfig.shareGroupPersisterClassName.nonEmpty) { val klass = Utils.loadClass(config.shareGroupConfig.shareGroupPersisterClassName, classOf[Object]).asInstanceOf[Class[Persister]] if (klass.getName.equals(classOf[DefaultStatePersister].getName)) { @@ -671,7 +669,7 @@ class BrokerServer( .newInstance( new PersisterStateManager( NetworkUtils.buildNetworkClient("Persister", config, metrics, Time.SYSTEM, new LogContext(s"[Persister broker=${config.brokerId}]")), - new ShareCoordinatorMetadataCacheHelperImpl(metadataCache, key => shareCoordinator.get.partitionFor(key), config.interBrokerListenerName), + new ShareCoordinatorMetadataCacheHelperImpl(metadataCache, key => shareCoordinator.partitionFor(key), config.interBrokerListenerName), Time.SYSTEM, new SystemTimerReaper( "persister-state-manager-reaper", @@ -782,8 +780,8 @@ class BrokerServer( CoreUtils.swallow(groupConfigManager.close(), this) if (groupCoordinator != null) CoreUtils.swallow(groupCoordinator.shutdown(), this) - if (shareCoordinator.isDefined) - CoreUtils.swallow(shareCoordinator.get.shutdown(), this) + if (shareCoordinator != null) + CoreUtils.swallow(shareCoordinator.shutdown(), this) if (assignmentsManager != null) CoreUtils.swallow(assignmentsManager.close(), this) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index dc4a09e0d23da..94131c65d8ce8 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -34,6 +34,7 @@ import org.apache.kafka.common.metadata.{ConfigRecord, MetadataRecordType} import org.apache.kafka.common.metrics.{Metrics, MetricsReporter} import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable} import org.apache.kafka.common.security.authenticator.LoginManager +import org.apache.kafka.common.utils.LogContext import org.apache.kafka.common.utils.{BufferSupplier, ConfigUtils, Utils} import org.apache.kafka.config import org.apache.kafka.coordinator.transaction.TransactionLogConfig @@ -48,15 +49,16 @@ import org.apache.kafka.server.telemetry.ClientTelemetry import org.apache.kafka.snapshot.RecordsSnapshotReader import org.apache.kafka.storage.internals.log.{LogCleaner, LogConfig} +import scala.util.Using import scala.collection._ import scala.jdk.CollectionConverters._ /** * Dynamic broker configurations may be defined at two levels: *
    - *
  • Per-broker configurations are persisted at the controller and can be described + *
  • Per-broker configurations are persisted at the controller and can be described * or altered using AdminClient with the resource name brokerId.
  • - *
  • Cluster-wide default configurations are persisted at the cluster level and can be + *
  • Cluster-wide default configurations are persisted at the cluster level and can be * described or altered using AdminClient with an empty resource name.
  • *
* The order of precedence for broker configs is: @@ -195,7 +197,8 @@ object DynamicBrokerConfig { private[server] def readDynamicBrokerConfigsFromSnapshot( raftManager: KafkaRaftManager[ApiMessageAndVersion], config: KafkaConfig, - quotaManagers: QuotaFactory.QuotaManagers + quotaManagers: QuotaFactory.QuotaManagers, + logContext: LogContext ): Unit = { def putOrRemoveIfNull(props: Properties, key: String, value: String): Unit = { if (value == null) { @@ -204,38 +207,42 @@ object DynamicBrokerConfig { props.put(key, value) } } - raftManager.replicatedLog.latestSnapshotId().ifPresent(latestSnapshotId => { - raftManager.replicatedLog.readSnapshot(latestSnapshotId).ifPresent(rawSnapshotReader => { - val reader = RecordsSnapshotReader.of( - rawSnapshotReader, - raftManager.recordSerde, - BufferSupplier.create(), - KafkaRaftClient.MAX_BATCH_SIZE_BYTES, - true - ) - val dynamicPerBrokerConfigs = new Properties() - val dynamicDefaultConfigs = new Properties() - while (reader.hasNext) { - val batch = reader.next() - batch.forEach(record => { - if (record.message().apiKey() == MetadataRecordType.CONFIG_RECORD.id) { - val configRecord = record.message().asInstanceOf[ConfigRecord] - if (DynamicBrokerConfig.AllDynamicConfigs.contains(configRecord.name()) && - configRecord.resourceType() == ConfigResource.Type.BROKER.id()) { - if (configRecord.resourceName().isEmpty) { - putOrRemoveIfNull(dynamicDefaultConfigs, configRecord.name(), configRecord.value()) - } else if (configRecord.resourceName() == config.brokerId.toString) { - putOrRemoveIfNull(dynamicPerBrokerConfigs, configRecord.name(), configRecord.value()) - } + raftManager.replicatedLog.latestSnapshotId().ifPresent { latestSnapshotId => + raftManager.replicatedLog.readSnapshot(latestSnapshotId).ifPresent { rawSnapshotReader => + Using.resource( + RecordsSnapshotReader.of( + rawSnapshotReader, + raftManager.recordSerde, + BufferSupplier.create(), + KafkaRaftClient.MAX_BATCH_SIZE_BYTES, + true, + logContext + ) + ) { reader => + val dynamicPerBrokerConfigs = new Properties() + val dynamicDefaultConfigs = new Properties() + while (reader.hasNext) { + val batch = reader.next() + batch.forEach { record => + if (record.message().apiKey() == MetadataRecordType.CONFIG_RECORD.id) { + val configRecord = record.message().asInstanceOf[ConfigRecord] + if (DynamicBrokerConfig.AllDynamicConfigs.contains(configRecord.name()) && + configRecord.resourceType() == ConfigResource.Type.BROKER.id()) { + if (configRecord.resourceName().isEmpty) { + putOrRemoveIfNull(dynamicDefaultConfigs, configRecord.name(), configRecord.value()) + } else if (configRecord.resourceName() == config.brokerId.toString) { + putOrRemoveIfNull(dynamicPerBrokerConfigs, configRecord.name(), configRecord.value()) + } + } } } - }) + } + val configHandler = new BrokerConfigHandler(config, quotaManagers) + configHandler.processConfigChanges("", dynamicPerBrokerConfigs) + configHandler.processConfigChanges(config.brokerId.toString, dynamicPerBrokerConfigs) } - val configHandler = new BrokerConfigHandler(config, quotaManagers) - configHandler.processConfigChanges("", dynamicPerBrokerConfigs) - configHandler.processConfigChanges(config.brokerId.toString, dynamicPerBrokerConfigs) - }) - }) + } + } } } diff --git a/core/src/main/scala/kafka/server/ForwardingManager.scala b/core/src/main/scala/kafka/server/ForwardingManager.scala index c067000bf0c2c..7737d2d2171f2 100644 --- a/core/src/main/scala/kafka/server/ForwardingManager.scala +++ b/core/src/main/scala/kafka/server/ForwardingManager.scala @@ -26,6 +26,7 @@ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, EnvelopeRequest, EnvelopeResponse, RequestContext, RequestHeader} import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} +import org.apache.kafka.server.metrics.ForwardingManagerMetrics import java.util.Optional import java.util.concurrent.TimeUnit @@ -117,7 +118,7 @@ class ForwardingManagerImpl( metrics: Metrics ) extends ForwardingManager with AutoCloseable with Logging { - val forwardingManagerMetrics: ForwardingManagerMetrics = ForwardingManagerMetrics(metrics, channelManager.getTimeoutMs) + val forwardingManagerMetrics: ForwardingManagerMetrics = new ForwardingManagerMetrics(metrics, channelManager.getTimeoutMs) override def forwardRequest( requestContext: RequestContext, @@ -133,7 +134,7 @@ class ForwardingManagerImpl( class ForwardingResponseHandler extends ControllerRequestCompletionHandler { override def onComplete(clientResponse: ClientResponse): Unit = { - forwardingManagerMetrics.queueLength.getAndDecrement() + forwardingManagerMetrics.decrementQueueLength() forwardingManagerMetrics.remoteTimeMsHist.record(clientResponse.requestLatencyMs()) forwardingManagerMetrics.queueTimeMsHist.record(clientResponse.receivedTimeMs() - clientResponse.requestLatencyMs() - requestCreationTimeMs) @@ -174,14 +175,14 @@ class ForwardingManagerImpl( override def onTimeout(): Unit = { debug(s"Forwarding of the request ${requestToString()} failed due to timeout exception") - forwardingManagerMetrics.queueLength.getAndDecrement() + forwardingManagerMetrics.decrementQueueLength() forwardingManagerMetrics.queueTimeMsHist.record(channelManager.getTimeoutMs) val response = requestBody.getErrorResponse(new TimeoutException()) responseCallback(Option(response)) } } - forwardingManagerMetrics.queueLength.getAndIncrement() + forwardingManagerMetrics.incrementQueueLength() channelManager.sendRequest(envelopeRequest, new ForwardingResponseHandler) } diff --git a/core/src/main/scala/kafka/server/ForwardingManagerMetrics.scala b/core/src/main/scala/kafka/server/ForwardingManagerMetrics.scala deleted file mode 100644 index a846f8c49551d..0000000000000 --- a/core/src/main/scala/kafka/server/ForwardingManagerMetrics.scala +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * 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 - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server - -import org.apache.kafka.common.MetricName -import org.apache.kafka.common.metrics.{Gauge, MetricConfig, Metrics} -import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing -import org.apache.kafka.common.metrics.stats.{Percentile, Percentiles} - -import java.util.concurrent.atomic.AtomicInteger - -final class ForwardingManagerMetrics private ( - metrics: Metrics, - timeoutMs: Long, -) extends AutoCloseable { - import ForwardingManagerMetrics._ - - /** - * A histogram describing the amount of time in milliseconds each admin request spends in the broker's forwarding manager queue, waiting to be sent to the controller. - * This does not include the time that the request spends waiting for a response from the controller. - */ - val queueTimeMsHist: LatencyHistogram = new LatencyHistogram(metrics, queueTimeMsName, metricGroupName, timeoutMs) - - /** - * A histogram describing the amount of time in milliseconds each request sent by the ForwardingManager spends waiting for a response. - * This does not include the time spent in the queue. - */ - val remoteTimeMsHist: LatencyHistogram = new LatencyHistogram(metrics, remoteTimeMsName, metricGroupName, timeoutMs) - - val queueLengthName: MetricName = metrics.metricName( - "QueueLength", - metricGroupName, - "The current number of RPCs that are waiting in the broker's forwarding manager queue, waiting to be sent to the controller." - ) - val queueLength: AtomicInteger = new AtomicInteger(0) - metrics.addMetric(queueLengthName, new FuncGauge(_ => queueLength.get())) - - override def close(): Unit = { - queueTimeMsHist.close() - remoteTimeMsHist.close() - metrics.removeMetric(queueLengthName) - } -} - -object ForwardingManagerMetrics { - - val metricGroupName = "ForwardingManager" - val queueTimeMsName = "QueueTimeMs" - val remoteTimeMsName = "RemoteTimeMs" - - final class LatencyHistogram ( - metrics: Metrics, - name: String, - group: String, - maxLatency: Long - ) extends AutoCloseable { - private val sensor = metrics.sensor(name) - val latencyP99Name: MetricName = metrics.metricName(s"$name.p99", group) - val latencyP999Name: MetricName = metrics.metricName(s"$name.p999", group) - - sensor.add(new Percentiles( - 4000, - maxLatency, - BucketSizing.CONSTANT, - new Percentile(latencyP99Name, 99), - new Percentile(latencyP999Name, 99.9) - )) - - override def close(): Unit = { - metrics.removeSensor(name) - metrics.removeMetric(latencyP99Name) - metrics.removeMetric(latencyP999Name) - } - - def record(latencyMs: Long): Unit = sensor.record(latencyMs) - } - - private final class FuncGauge[T](func: Long => T) extends Gauge[T] { - override def value(config: MetricConfig, now: Long): T = { - func(now) - } - } - - def apply(metrics: Metrics, timeoutMs: Long): ForwardingManagerMetrics = new ForwardingManagerMetrics(metrics, timeoutMs) -} diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 2841dda409ef0..08879b38318db 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -62,7 +62,7 @@ import org.apache.kafka.coordinator.share.ShareCoordinator import org.apache.kafka.metadata.{ConfigRepository, MetadataCache} import org.apache.kafka.server.{ApiVersionManager, ClientMetricsManager, DelegationTokenManager, ProcessRole} import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.common.{GroupVersion, RequestLocal, StreamsVersion, TransactionVersion} +import org.apache.kafka.server.common.{GroupVersion, RequestLocal, ShareVersion, StreamsVersion, TransactionVersion} import org.apache.kafka.server.config.DelegationTokenManagerConfigs import org.apache.kafka.server.share.context.ShareFetchContext import org.apache.kafka.server.share.{ErroneousAndValidPartitionData, SharePartitionKey} @@ -91,7 +91,7 @@ class KafkaApis(val requestChannel: RequestChannel, val replicaManager: ReplicaManager, val groupCoordinator: GroupCoordinator, val txnCoordinator: TransactionCoordinator, - val shareCoordinator: Option[ShareCoordinator], + val shareCoordinator: ShareCoordinator, val autoTopicCreationManager: AutoTopicCreationManager, val brokerId: Int, val config: KafkaConfig, @@ -1035,7 +1035,7 @@ class KafkaApis(val requestChannel: RequestChannel, CompletableFuture.allOf(futures.toArray: _*).handle[Unit] { (_, _) => val groupResponses = new ArrayBuffer[OffsetFetchResponseData.OffsetFetchResponseGroup](futures.size) futures.foreach(future => groupResponses += future.get()) - requestHelper.sendMaybeThrottle(request, new OffsetFetchResponse(groupResponses.asJava, request.context.apiVersion)) + requestHelper.sendMaybeThrottle(request, new OffsetFetchResponse.Builder(groupResponses.asJava).build(request.context.apiVersion)) } } @@ -1185,9 +1185,6 @@ class KafkaApis(val requestChannel: RequestChannel, else { if (keyType == CoordinatorType.SHARE.id) { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - if (shareCoordinator.isEmpty) { - return (Errors.INVALID_REQUEST, Node.noNode) - } try { SharePartitionKey.validate(key) } catch { @@ -1205,7 +1202,7 @@ class KafkaApis(val requestChannel: RequestChannel, case CoordinatorType.SHARE => // We know that shareCoordinator is defined at this stage. - (shareCoordinator.get.partitionFor(SharePartitionKey.getInstance(key)), SHARE_GROUP_STATE_TOPIC_NAME) + (shareCoordinator.partitionFor(SharePartitionKey.getInstance(key)), SHARE_GROUP_STATE_TOPIC_NAME) } val topicMetadata = metadataCache.getTopicMetadata(Set(internalTopicName).asJava, request.context.listenerName, false, false).asScala @@ -2500,7 +2497,13 @@ class KafkaApis(val requestChannel: RequestChannel, val filteredProducerIds = listTransactionsRequest.data.producerIdFilters.asScala.map(Long.unbox).toSet val filteredStates = listTransactionsRequest.data.stateFilters.asScala.toSet val durationFilter = listTransactionsRequest.data.durationFilter() - val response = txnCoordinator.handleListTransactions(filteredProducerIds, filteredStates, durationFilter) + val transactionalIdPatternFilter = listTransactionsRequest.data.transactionalIdPattern + val response = txnCoordinator.handleListTransactions( + filteredProducerIds, + filteredStates, + durationFilter, + transactionalIdPatternFilter + ) // The response should contain only transactionalIds that the principal // has `Describe` permission to access. @@ -3044,7 +3047,7 @@ class KafkaApis(val requestChannel: RequestChannel, try { // Creating the shareFetchContext for Share Session Handling. if context creation fails, the request is failed directly here. - shareFetchContext = sharePartitionManager.newContext(groupId, shareFetchData, forgottenTopics, newReqMetadata, isAcknowledgeDataPresent) + shareFetchContext = sharePartitionManager.newContext(groupId, shareFetchData, forgottenTopics, newReqMetadata, isAcknowledgeDataPresent, request.context.connectionId) } catch { case e: Exception => requestHelper.sendMaybeThrottle(request, shareFetchRequest.getErrorResponse(AbstractResponse.DEFAULT_THROTTLE_TIME, e)) @@ -3404,21 +3407,14 @@ class KafkaApis(val requestChannel: RequestChannel, return CompletableFuture.completedFuture[Unit](()) } - shareCoordinator match { - case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - initializeShareGroupStateRequest.getErrorResponse(requestThrottleMs, - new ApiException("Share coordinator is not enabled."))) - CompletableFuture.completedFuture[Unit](()) - - case Some(coordinator) => coordinator.initializeState(request.context, initializeShareGroupStateRequest.data) - .handle[Unit] { (response, exception) => - if (exception != null) { - requestHelper.sendMaybeThrottle(request, initializeShareGroupStateRequest.getErrorResponse(exception)) - } else { - requestHelper.sendMaybeThrottle(request, new InitializeShareGroupStateResponse(response)) - } + shareCoordinator.initializeState(request.context, initializeShareGroupStateRequest.data) + .handle[Unit] { (response, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, initializeShareGroupStateRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, new InitializeShareGroupStateResponse(response)) } - } + } } def handleReadShareGroupStateRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { @@ -3433,20 +3429,14 @@ class KafkaApis(val requestChannel: RequestChannel, return CompletableFuture.completedFuture[Unit](()) } - shareCoordinator match { - case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - readShareGroupStateRequest.getErrorResponse(requestThrottleMs, - new ApiException("Share coordinator is not enabled."))) - CompletableFuture.completedFuture[Unit](()) - case Some(coordinator) => coordinator.readState(request.context, readShareGroupStateRequest.data) - .handle[Unit] { (response, exception) => - if (exception != null) { - requestHelper.sendMaybeThrottle(request, readShareGroupStateRequest.getErrorResponse(exception)) - } else { - requestHelper.sendMaybeThrottle(request, new ReadShareGroupStateResponse(response)) - } + shareCoordinator.readState(request.context, readShareGroupStateRequest.data) + .handle[Unit] { (response, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, readShareGroupStateRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, new ReadShareGroupStateResponse(response)) } - } + } } def handleWriteShareGroupStateRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { @@ -3461,20 +3451,14 @@ class KafkaApis(val requestChannel: RequestChannel, return CompletableFuture.completedFuture[Unit](()) } - shareCoordinator match { - case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - writeShareGroupStateRequest.getErrorResponse(requestThrottleMs, - new ApiException("Share coordinator is not enabled."))) - CompletableFuture.completedFuture[Unit](()) - case Some(coordinator) => coordinator.writeState(request.context, writeShareGroupStateRequest.data) - .handle[Unit] { (response, exception) => - if (exception != null) { - requestHelper.sendMaybeThrottle(request, writeShareGroupStateRequest.getErrorResponse(exception)) - } else { - requestHelper.sendMaybeThrottle(request, new WriteShareGroupStateResponse(response)) - } + shareCoordinator.writeState(request.context, writeShareGroupStateRequest.data) + .handle[Unit] { (response, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, writeShareGroupStateRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, new WriteShareGroupStateResponse(response)) } - } + } } def handleDeleteShareGroupStateRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { @@ -3489,21 +3473,14 @@ class KafkaApis(val requestChannel: RequestChannel, return CompletableFuture.completedFuture[Unit](()) } - shareCoordinator match { - case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - deleteShareGroupStateRequest.getErrorResponse(requestThrottleMs, - new ApiException("Share coordinator is not enabled."))) - CompletableFuture.completedFuture[Unit](()) - - case Some(coordinator) => coordinator.deleteState(request.context, deleteShareGroupStateRequest.data) - .handle[Unit] { (response, exception) => - if (exception != null) { - requestHelper.sendMaybeThrottle(request, deleteShareGroupStateRequest.getErrorResponse(exception)) - } else { - requestHelper.sendMaybeThrottle(request, new DeleteShareGroupStateResponse(response)) - } + shareCoordinator.deleteState(request.context, deleteShareGroupStateRequest.data) + .handle[Unit] { (response, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, deleteShareGroupStateRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, new DeleteShareGroupStateResponse(response)) } - } + } } def handleReadShareGroupStateSummaryRequest(request: RequestChannel.Request): CompletableFuture[Unit] = { @@ -3518,20 +3495,14 @@ class KafkaApis(val requestChannel: RequestChannel, return CompletableFuture.completedFuture[Unit](()) } - shareCoordinator match { - case None => requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - readShareGroupStateSummaryRequest.getErrorResponse(requestThrottleMs, - new ApiException("Share coordinator is not enabled."))) - CompletableFuture.completedFuture[Unit](()) - case Some(coordinator) => coordinator.readStateSummary(request.context, readShareGroupStateSummaryRequest.data) - .handle[Unit] { (response, exception) => - if (exception != null) { - requestHelper.sendMaybeThrottle(request, readShareGroupStateSummaryRequest.getErrorResponse(exception)) - } else { - requestHelper.sendMaybeThrottle(request, new ReadShareGroupStateSummaryResponse(response)) - } + shareCoordinator.readStateSummary(request.context, readShareGroupStateSummaryRequest.data) + .handle[Unit] { (response, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, readShareGroupStateSummaryRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, new ReadShareGroupStateSummaryResponse(response)) } - } + } } def handleDescribeShareGroupOffsetsRequest(request: RequestChannel.Request): Unit = { @@ -3964,8 +3935,12 @@ class KafkaApis(val requestChannel: RequestChannel, .setCurrentLeader(partitionData.currentLeader) } + private def shareVersion(): ShareVersion = { + ShareVersion.fromFeatureLevel(metadataCache.features.finalizedFeatures.getOrDefault(ShareVersion.FEATURE_NAME, 0.toShort)) + } + private def isShareGroupProtocolEnabled: Boolean = { - config.shareGroupConfig.isShareGroupEnabled + config.shareGroupConfig.isShareGroupEnabled || shareVersion().supportsShareGroups } private def updateRecordConversionStats(request: RequestChannel.Request, diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index d95be34ff51de..7fbbe1f19597d 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -71,7 +71,7 @@ class BrokerMetadataPublisher( replicaManager: ReplicaManager, groupCoordinator: GroupCoordinator, txnCoordinator: TransactionCoordinator, - shareCoordinator: Option[ShareCoordinator], + shareCoordinator: ShareCoordinator, var dynamicConfigPublisher: DynamicConfigPublisher, dynamicClientQuotaPublisher: DynamicClientQuotaPublisher, dynamicTopicClusterQuotaPublisher: DynamicTopicClusterQuotaPublisher, @@ -166,18 +166,16 @@ class BrokerMetadataPublisher( case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating txn " + s"coordinator with local changes in $deltaName", t) } - if (shareCoordinator.isDefined) { - try { - updateCoordinator(newImage, - delta, - Topic.SHARE_GROUP_STATE_TOPIC_NAME, - shareCoordinator.get.onElection, - (partitionIndex, leaderEpochOpt) => shareCoordinator.get.onResignation(partitionIndex, toOptionalInt(leaderEpochOpt)) - ) - } catch { - case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating share " + - s"coordinator with local changes in $deltaName", t) - } + try { + updateCoordinator(newImage, + delta, + Topic.SHARE_GROUP_STATE_TOPIC_NAME, + shareCoordinator.onElection, + (partitionIndex, leaderEpochOpt) => shareCoordinator.onResignation(partitionIndex, toOptionalInt(leaderEpochOpt)) + ) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating share " + + s"coordinator with local changes in $deltaName", t) } try { // Notify the group coordinator about deleted topics. @@ -225,7 +223,7 @@ class BrokerMetadataPublisher( try { // Propagate the new image to the share coordinator. - shareCoordinator.foreach(coordinator => coordinator.onNewMetadataImage(newImage, delta)) + shareCoordinator.onNewMetadataImage(newImage, delta) } catch { case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating share " + s"coordinator with local changes in $deltaName", t) @@ -340,14 +338,12 @@ class BrokerMetadataPublisher( } catch { case t: Throwable => fatalFaultHandler.handleFault("Error starting TransactionCoordinator", t) } - if (config.shareGroupConfig.isShareGroupEnabled && shareCoordinator.isDefined) { - try { - // Start the share coordinator. - shareCoordinator.get.startup(() => metadataCache.numPartitions( - Topic.SHARE_GROUP_STATE_TOPIC_NAME).orElse(config.shareCoordinatorConfig.shareCoordinatorStateTopicNumPartitions())) - } catch { - case t: Throwable => fatalFaultHandler.handleFault("Error starting Share coordinator", t) - } + try { + // Start the share coordinator. + shareCoordinator.startup(() => metadataCache.numPartitions(Topic.SHARE_GROUP_STATE_TOPIC_NAME) + .orElse(config.shareCoordinatorConfig.shareCoordinatorStateTopicNumPartitions())) + } catch { + case t: Throwable => fatalFaultHandler.handleFault("Error starting Share coordinator", t) } } diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index a95c77cca4e85..85bc4e1a2692b 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -277,7 +277,7 @@ object DumpLogSegments { println(s"Snapshot end offset: ${path.snapshotId.offset}, epoch: ${path.snapshotId.epoch}") } } - val fileRecords = FileRecords.open(file, false).slice(0, maxBytes) + val fileRecords = FileRecords.open(file, false).slice(0, maxBytes).asInstanceOf[FileRecords] try { var validBytes = 0L var lastOffset = -1L diff --git a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java index 43ece70ca0ee6..1ec8cccffa960 100644 --- a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java +++ b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java @@ -79,6 +79,7 @@ import scala.collection.Seq; import scala.jdk.javaapi.CollectionConverters; +import static kafka.server.share.PendingRemoteFetches.RemoteFetch; import static kafka.server.share.SharePartitionManagerTest.DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL; import static kafka.server.share.SharePartitionManagerTest.buildLogReadResult; import static kafka.server.share.SharePartitionManagerTest.mockReplicaManagerDelayedShareFetch; @@ -136,9 +137,6 @@ public void testDelayedShareFetchTryCompleteReturnsFalseDueToNonAcquirablePartit SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - LinkedHashMap sharePartitions = new LinkedHashMap<>(); sharePartitions.put(tp0, sp0); sharePartitions.put(tp1, sp1); @@ -151,12 +149,17 @@ public void testDelayedShareFetchTryCompleteReturnsFalseDueToNonAcquirablePartit when(sp1.canAcquireRecords()).thenReturn(false); ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(new MockTime()); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withShareGroupMetrics(shareGroupMetrics) + .withFetchId(fetchId) .build()); + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + // Since there is no partition that can be acquired, tryComplete should return false. assertFalse(delayedShareFetch.tryComplete()); assertFalse(delayedShareFetch.isCompleted()); @@ -181,9 +184,6 @@ public void testTryCompleteWhenMinBytesNotSatisfiedOnFirstFetch() { SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - LinkedHashMap sharePartitions = new LinkedHashMap<>(); sharePartitions.put(tp0, sp0); sharePartitions.put(tp1, sp1); @@ -216,6 +216,7 @@ public void testTryCompleteWhenMinBytesNotSatisfiedOnFirstFetch() { Time time = mock(Time.class); when(time.hiResClockMs()).thenReturn(100L).thenReturn(110L); ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(time); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) @@ -224,7 +225,12 @@ public void testTryCompleteWhenMinBytesNotSatisfiedOnFirstFetch() { .withPartitionMaxBytesStrategy(partitionMaxBytesStrategy) .withShareGroupMetrics(shareGroupMetrics) .withTime(time) + .withFetchId(fetchId) .build()); + + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); // Since sp1 cannot be acquired, tryComplete should return false. @@ -253,9 +259,6 @@ public void testTryCompleteWhenMinBytesNotSatisfiedOnSubsequentFetch() { SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - LinkedHashMap sharePartitions = new LinkedHashMap<>(); sharePartitions.put(tp0, sp0); sharePartitions.put(tp1, sp1); @@ -279,12 +282,18 @@ public void testTryCompleteWhenMinBytesNotSatisfiedOnSubsequentFetch() { mockTopicIdPartitionFetchBytes(replicaManager, tp0, hwmOffsetMetadata); BiConsumer exceptionHandler = mockExceptionHandler(); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withReplicaManager(replicaManager) .withExceptionHandler(exceptionHandler) + .withFetchId(fetchId) .build()); + + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); // Since sp1 cannot be acquired, tryComplete should return false. @@ -307,9 +316,6 @@ public void testDelayedShareFetchTryCompleteReturnsTrue() { SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - LinkedHashMap sharePartitions = new LinkedHashMap<>(); sharePartitions.put(tp0, sp0); sharePartitions.put(tp1, sp1); @@ -332,6 +338,7 @@ public void testDelayedShareFetchTryCompleteReturnsTrue() { Time time = mock(Time.class); when(time.hiResClockMs()).thenReturn(120L).thenReturn(140L); ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(time); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) @@ -339,7 +346,12 @@ public void testDelayedShareFetchTryCompleteReturnsTrue() { .withPartitionMaxBytesStrategy(partitionMaxBytesStrategy) .withShareGroupMetrics(shareGroupMetrics) .withTime(time) + .withFetchId(fetchId) .build()); + + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); // Since sp1 can be acquired, tryComplete should return true. @@ -366,9 +378,6 @@ public void testEmptyFutureReturnedByDelayedShareFetchOnComplete() { SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - LinkedHashMap sharePartitions = new LinkedHashMap<>(); sharePartitions.put(tp0, sp0); sharePartitions.put(tp1, sp1); @@ -383,13 +392,19 @@ public void testEmptyFutureReturnedByDelayedShareFetchOnComplete() { Time time = mock(Time.class); when(time.hiResClockMs()).thenReturn(90L).thenReturn(140L); ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(time); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withReplicaManager(replicaManager) .withSharePartitions(sharePartitions) .withShareGroupMetrics(shareGroupMetrics) .withTime(time) + .withFetchId(fetchId) .build()); + + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); delayedShareFetch.forceComplete(); @@ -420,9 +435,6 @@ public void testReplicaManagerFetchShouldHappenOnComplete() { SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - LinkedHashMap sharePartitions = new LinkedHashMap<>(); sharePartitions.put(tp0, sp0); sharePartitions.put(tp1, sp1); @@ -442,6 +454,7 @@ public void testReplicaManagerFetchShouldHappenOnComplete() { Time time = mock(Time.class); when(time.hiResClockMs()).thenReturn(10L).thenReturn(140L); ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(time); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withReplicaManager(replicaManager) @@ -449,7 +462,12 @@ public void testReplicaManagerFetchShouldHappenOnComplete() { .withPartitionMaxBytesStrategy(partitionMaxBytesStrategy) .withShareGroupMetrics(shareGroupMetrics) .withTime(time) + .withFetchId(fetchId) .build()); + + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); delayedShareFetch.forceComplete(); @@ -486,16 +504,20 @@ public void testToCompleteAnAlreadyCompletedFuture() { ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(), future, List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(false); ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(new MockTime()); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withReplicaManager(replicaManager) .withSharePartitions(sharePartitions) .withShareGroupMetrics(shareGroupMetrics) + .withFetchId(fetchId) .build()); + + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); // Force completing the share fetch request for the first time should complete the future with an empty map. @@ -535,10 +557,6 @@ public void testForceCompleteTriggersDelayedActionsQueue() { SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); SharePartition sp2 = mock(SharePartition.class); - // No share partition is available for acquiring initially. - when(sp0.maybeAcquireFetchLock()).thenReturn(false); - when(sp1.maybeAcquireFetchLock()).thenReturn(false); - when(sp2.maybeAcquireFetchLock()).thenReturn(false); LinkedHashMap sharePartitions1 = new LinkedHashMap<>(); sharePartitions1.put(tp0, sp0); @@ -557,12 +575,19 @@ public void testForceCompleteTriggersDelayedActionsQueue() { List delayedShareFetchWatchKeys = new ArrayList<>(); topicIdPartitions1.forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); + Uuid fetchId1 = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch1 = DelayedShareFetchTest.DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch1) .withReplicaManager(replicaManager) .withSharePartitions(sharePartitions1) + .withFetchId(fetchId1) .build(); + // No share partition is available for acquiring initially. + when(sp0.maybeAcquireFetchLock(fetchId1)).thenReturn(false); + when(sp1.maybeAcquireFetchLock(fetchId1)).thenReturn(false); + when(sp2.maybeAcquireFetchLock(fetchId1)).thenReturn(false); + // We add a delayed share fetch entry to the purgatory which will be waiting for completion since neither of the // partitions in the share fetch request can be acquired. delayedShareFetchPurgatory.tryCompleteElseWatch(delayedShareFetch1, delayedShareFetchWatchKeys); @@ -585,15 +610,17 @@ public void testForceCompleteTriggersDelayedActionsQueue() { sharePartitions2.put(tp1, sp1); sharePartitions2.put(tp2, sp2); + Uuid fetchId2 = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch2 = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch2) .withReplicaManager(replicaManager) .withSharePartitions(sharePartitions2) .withPartitionMaxBytesStrategy(partitionMaxBytesStrategy) + .withFetchId(fetchId2) .build()); // sp1 can be acquired now - when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId2)).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); when(sp1.acquire(anyString(), anyInt(), anyInt(), anyLong(), any(FetchPartitionData.class), any())).thenReturn( createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); @@ -679,8 +706,6 @@ public void testExceptionInMinBytesCalculation() { SharePartition sp0 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - LinkedHashMap sharePartitions = new LinkedHashMap<>(); sharePartitions.put(tp0, sp0); @@ -706,6 +731,7 @@ public void testExceptionInMinBytesCalculation() { Time time = mock(Time.class); when(time.hiResClockMs()).thenReturn(100L).thenReturn(110L).thenReturn(170L); ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(time); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) @@ -714,8 +740,11 @@ public void testExceptionInMinBytesCalculation() { .withPartitionMaxBytesStrategy(partitionMaxBytesStrategy) .withShareGroupMetrics(shareGroupMetrics) .withTime(time) + .withFetchId(fetchId) .build()); + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + // Try complete should return false as the share partition has errored out. assertFalse(delayedShareFetch.tryComplete()); // Fetch should remain pending and should be completed on request timeout. @@ -727,10 +756,10 @@ public void testExceptionInMinBytesCalculation() { Mockito.verify(replicaManager, times(1)).readFromLog( any(), any(), any(ReplicaQuota.class), anyBoolean()); Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(any()); - Mockito.verify(sp0, times(1)).releaseFetchLock(); + Mockito.verify(sp0, times(1)).releaseFetchLock(fetchId); // Force complete the request as it's still pending. Return false from the share partition lock acquire. - when(sp0.maybeAcquireFetchLock()).thenReturn(false); + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(false); assertTrue(delayedShareFetch.forceComplete()); assertTrue(delayedShareFetch.isCompleted()); @@ -756,7 +785,6 @@ public void testTryCompleteLocksReleasedOnCompleteException() { TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); SharePartition sp0 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.of(new LogOffsetMetadata(0, 1, 0))); @@ -771,12 +799,17 @@ public void testTryCompleteLocksReleasedOnCompleteException() { mockTopicIdPartitionToReturnDataEqualToMinBytes(replicaManager, tp0, 1); PartitionMaxBytesStrategy partitionMaxBytesStrategy = mockPartitionMaxBytes(Set.of(tp0)); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withReplicaManager(replicaManager) .withPartitionMaxBytesStrategy(partitionMaxBytesStrategy) + .withFetchId(fetchId) .build()); + + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); // Throw exception for onComplete. doThrow(new RuntimeException()).when(delayedShareFetch).onComplete(); @@ -784,7 +817,7 @@ public void testTryCompleteLocksReleasedOnCompleteException() { assertFalse(delayedShareFetch.tryComplete()); Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(any()); - Mockito.verify(sp0, times(1)).releaseFetchLock(); + Mockito.verify(sp0, times(1)).releaseFetchLock(fetchId); } @Test @@ -793,7 +826,6 @@ public void testLocksReleasedForCompletedFetch() { TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); SharePartition sp0 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); LinkedHashMap sharePartitions1 = new LinkedHashMap<>(); @@ -809,18 +841,21 @@ public void testLocksReleasedForCompletedFetch() { PartitionMaxBytesStrategy partitionMaxBytesStrategy = mockPartitionMaxBytes(Set.of(tp0)); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions1) .withReplicaManager(replicaManager) .withPartitionMaxBytesStrategy(partitionMaxBytesStrategy) + .withFetchId(fetchId) .build(); DelayedShareFetch spy = spy(delayedShareFetch); + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); doReturn(false).when(spy).forceComplete(); assertFalse(spy.tryComplete()); - Mockito.verify(sp0, times(1)).releaseFetchLock(); + Mockito.verify(sp0, times(1)).releaseFetchLock(fetchId); assertTrue(delayedShareFetch.lock().tryLock()); delayedShareFetch.lock().unlock(); } @@ -831,7 +866,6 @@ public void testLocksReleasedAcquireException() { TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); SharePartition sp0 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenThrow(new RuntimeException("Acquire exception")); LinkedHashMap sharePartitions = new LinkedHashMap<>(); @@ -841,13 +875,17 @@ public void testLocksReleasedAcquireException() { new CompletableFuture<>(), List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) + .withFetchId(fetchId) .build(); + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.tryComplete()); - Mockito.verify(sp0, times(1)).releaseFetchLock(); + Mockito.verify(sp0, times(1)).releaseFetchLock(fetchId); assertTrue(delayedShareFetch.lock().tryLock()); delayedShareFetch.lock().unlock(); } @@ -858,7 +896,6 @@ public void testTryCompleteWhenPartitionMaxBytesStrategyThrowsException() { TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); SharePartition sp0 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); LinkedHashMap sharePartitions = new LinkedHashMap<>(); sharePartitions.put(tp0, sp0); @@ -873,13 +910,17 @@ public void testTryCompleteWhenPartitionMaxBytesStrategyThrowsException() { PartitionMaxBytesStrategy partitionMaxBytesStrategy = mock(PartitionMaxBytesStrategy.class); when(partitionMaxBytesStrategy.maxBytes(anyInt(), any(), anyInt())).thenThrow(new IllegalArgumentException("Exception thrown")); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withExceptionHandler(mockExceptionHandler()) .withPartitionMaxBytesStrategy(partitionMaxBytesStrategy) + .withFetchId(fetchId) .build()); + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); assertTrue(delayedShareFetch.tryComplete()); assertTrue(delayedShareFetch.isCompleted()); @@ -912,11 +953,6 @@ public void testPartitionMaxBytesFromUniformStrategyWhenAllPartitionsAreAcquirab SharePartition sp3 = mock(SharePartition.class); SharePartition sp4 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - when(sp2.maybeAcquireFetchLock()).thenReturn(true); - when(sp3.maybeAcquireFetchLock()).thenReturn(true); - when(sp4.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); when(sp2.canAcquireRecords()).thenReturn(true); @@ -960,13 +996,21 @@ public void testPartitionMaxBytesFromUniformStrategyWhenAllPartitionsAreAcquirab mockTopicIdPartitionToReturnDataEqualToMinBytes(replicaManager, tp3, 1); mockTopicIdPartitionToReturnDataEqualToMinBytes(replicaManager, tp4, 1); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withReplicaManager(replicaManager) .withPartitionMaxBytesStrategy(PartitionMaxBytesStrategy.type(PartitionMaxBytesStrategy.StrategyType.UNIFORM)) + .withFetchId(fetchId) .build()); + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp2.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp3.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp4.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertTrue(delayedShareFetch.tryComplete()); assertTrue(delayedShareFetch.isCompleted()); @@ -1008,11 +1052,6 @@ public void testPartitionMaxBytesFromUniformStrategyWhenFewPartitionsAreAcquirab SharePartition sp3 = mock(SharePartition.class); SharePartition sp4 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - when(sp2.maybeAcquireFetchLock()).thenReturn(false); - when(sp3.maybeAcquireFetchLock()).thenReturn(true); - when(sp4.maybeAcquireFetchLock()).thenReturn(false); when(sp0.canAcquireRecords()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); when(sp2.canAcquireRecords()).thenReturn(false); @@ -1047,13 +1086,21 @@ public void testPartitionMaxBytesFromUniformStrategyWhenFewPartitionsAreAcquirab mockTopicIdPartitionToReturnDataEqualToMinBytes(replicaManager, tp0, 1); mockTopicIdPartitionToReturnDataEqualToMinBytes(replicaManager, tp1, 1); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withReplicaManager(replicaManager) .withPartitionMaxBytesStrategy(PartitionMaxBytesStrategy.type(PartitionMaxBytesStrategy.StrategyType.UNIFORM)) + .withFetchId(fetchId) .build()); + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp2.maybeAcquireFetchLock(fetchId)).thenReturn(false); + when(sp3.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp4.maybeAcquireFetchLock(fetchId)).thenReturn(false); + assertTrue(delayedShareFetch.tryComplete()); assertTrue(delayedShareFetch.isCompleted()); @@ -1181,10 +1228,6 @@ public void testRemoteStorageFetchTryCompleteReturnsFalse() { SharePartition sp1 = mock(SharePartition.class); SharePartition sp2 = mock(SharePartition.class); - // All the topic partitions are acquirable. - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - when(sp2.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); when(sp2.canAcquireRecords()).thenReturn(true); @@ -1215,18 +1258,25 @@ public void testRemoteStorageFetchTryCompleteReturnsFalse() { when(remoteLogManager.asyncRead(any(), any())).thenReturn(mock(Future.class)); when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withReplicaManager(replicaManager) .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1, tp2))) + .withFetchId(fetchId) .build()); + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp2.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); assertFalse(delayedShareFetch.tryComplete()); assertFalse(delayedShareFetch.isCompleted()); // Remote fetch object gets created for delayed share fetch object. - assertNotNull(delayedShareFetch.remoteFetch()); + assertNotNull(delayedShareFetch.pendingRemoteFetches()); // Verify the locks are released for local log read topic partitions tp0 and tp1. Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0, tp1)); assertTrue(delayedShareFetch.lock().tryLock()); @@ -1238,62 +1288,82 @@ public void testRemoteStorageFetchTryCompleteThrowsException() { ReplicaManager replicaManager = mock(ReplicaManager.class); TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 2)); SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); + SharePartition sp2 = mock(SharePartition.class); - // All the topic partitions are acquirable. - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); + when(sp2.canAcquireRecords()).thenReturn(true); LinkedHashMap sharePartitions = new LinkedHashMap<>(); sharePartitions.put(tp0, sp0); sharePartitions.put(tp1, sp1); + sharePartitions.put(tp2, sp2); CompletableFuture> future = new CompletableFuture<>(); ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), - future, List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, + future, List.of(tp0, tp1, tp2), BATCH_SIZE, MAX_FETCH_RECORDS, BROKER_TOPIC_STATS); when(sp0.nextFetchOffset()).thenReturn(10L); when(sp1.nextFetchOffset()).thenReturn(20L); + when(sp2.nextFetchOffset()).thenReturn(25L); - // Fetch offset does not match with the cached entry for sp0 and sp1. Hence, a replica manager fetch will happen for sp0 and sp1. + // Fetch offset does not match with the cached entry for sp0, sp1 and sp2. Hence, a replica manager fetch will happen for all. when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp2.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); - // Mocking local log read result for tp0 and remote storage read result for tp1. - doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp0), Set.of(tp1))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + // Mocking local log read result for tp0 and remote storage read result for tp1 and tp2. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp0), Set.of(tp1, tp2))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); - // Remote fetch related mocks. Exception will be thrown during the creation of remoteFetch object. + // Remote fetch related mocks. Exception will be thrown during the creation of remoteFetch object for tp2. + // remoteFetchTask gets created for tp1 successfully. + Future remoteFetchTask = mock(Future.class); + doAnswer(invocation -> { + when(remoteFetchTask.isCancelled()).thenReturn(true); + return false; + }).when(remoteFetchTask).cancel(false); RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); - when(remoteLogManager.asyncRead(any(), any())).thenThrow(new RejectedExecutionException("Exception thrown")); + when(remoteLogManager.asyncRead(any(), any())) + .thenReturn(remoteFetchTask) // for tp1 + .thenThrow(new RejectedExecutionException("Exception thrown")); // for tp2 when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); BiConsumer exceptionHandler = mockExceptionHandler(); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withReplicaManager(replicaManager) .withExceptionHandler(exceptionHandler) - .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1))) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1, tp2))) + .withFetchId(fetchId) .build()); + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp2.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); // tryComplete returns true and goes to forceComplete once the exception occurs. assertTrue(delayedShareFetch.tryComplete()); assertTrue(delayedShareFetch.isCompleted()); // The future of shareFetch completes. assertTrue(shareFetch.isCompleted()); + // The remoteFetchTask created for tp1 is cancelled successfully. + assertTrue(remoteFetchTask.isCancelled()); assertFalse(future.isCompletedExceptionally()); - assertEquals(Set.of(tp1), future.join().keySet()); + assertEquals(Set.of(tp1, tp2), future.join().keySet()); // Exception occurred and was handled. - Mockito.verify(exceptionHandler, times(1)).accept(any(), any()); - // Verify the locks are released for both local and remote read topic partitions tp0 and tp1 because of exception occurrence. + Mockito.verify(exceptionHandler, times(2)).accept(any(), any()); + // Verify the locks are released for all local and remote read topic partitions tp0, tp1 and tp2 because of exception occurrence. Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0)); - Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp1)); + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp1, tp2)); Mockito.verify(delayedShareFetch, times(1)).onComplete(); assertTrue(delayedShareFetch.lock().tryLock()); delayedShareFetch.lock().unlock(); @@ -1310,10 +1380,6 @@ public void testRemoteStorageFetchTryCompletionDueToBrokerBecomingOffline() { SharePartition sp1 = mock(SharePartition.class); SharePartition sp2 = mock(SharePartition.class); - // All the topic partitions are acquirable. - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - when(sp2.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); when(sp2.canAcquireRecords()).thenReturn(true); @@ -1361,20 +1427,29 @@ public void testRemoteStorageFetchTryCompletionDueToBrokerBecomingOffline() { when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); when(replicaManager.getPartitionOrException(tp2.topicPartition())).thenThrow(mock(KafkaStorageException.class)); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withReplicaManager(replicaManager) .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1, tp2))) + .withFetchId(fetchId) .build()); + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp2.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); assertTrue(delayedShareFetch.tryComplete()); assertTrue(delayedShareFetch.isCompleted()); // Pending remote fetch object gets created for delayed share fetch. - assertNotNull(delayedShareFetch.remoteFetch()); - assertTrue(delayedShareFetch.remoteFetch().remoteFetchTask().isCancelled()); + assertNotNull(delayedShareFetch.pendingRemoteFetches()); + List remoteFetches = delayedShareFetch.pendingRemoteFetches().remoteFetches(); + assertEquals(1, remoteFetches.size()); + assertTrue(remoteFetches.get(0).remoteFetchTask().isCancelled()); // Partition locks should be released for all 3 topic partitions Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0, tp1, tp2)); assertTrue(shareFetch.isCompleted()); @@ -1393,9 +1468,6 @@ public void testRemoteStorageFetchRequestCompletionOnFutureCompletionFailure() { SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); - // sp0 is acquirable, sp1 is not acquirable. - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(false); when(sp0.canAcquireRecords()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(false); @@ -1429,13 +1501,19 @@ public void testRemoteStorageFetchRequestCompletionOnFutureCompletionFailure() { }).when(remoteLogManager).asyncRead(any(), any()); when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withReplicaManager(replicaManager) .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1))) + .withFetchId(fetchId) .build()); + // sp0 is acquirable, sp1 is not acquirable. + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(false); + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); @@ -1444,7 +1522,7 @@ public void testRemoteStorageFetchRequestCompletionOnFutureCompletionFailure() { assertTrue(delayedShareFetch.isCompleted()); // Pending remote fetch object gets created for delayed share fetch. - assertNotNull(delayedShareFetch.remoteFetch()); + assertNotNull(delayedShareFetch.pendingRemoteFetches()); // Verify the locks are released for tp0. Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0)); assertTrue(shareFetch.isCompleted()); @@ -1461,8 +1539,7 @@ public void testRemoteStorageFetchRequestCompletionOnFutureCompletionSuccessfull SharePartition sp0 = mock(SharePartition.class); - // sp0 is acquirable. - when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); LinkedHashMap sharePartitions = new LinkedHashMap<>(); @@ -1494,13 +1571,18 @@ public void testRemoteStorageFetchRequestCompletionOnFutureCompletionSuccessfull }).when(remoteLogManager).asyncRead(any(), any()); when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withReplicaManager(replicaManager) .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0))) + .withFetchId(fetchId) .build()); + // sp0 is acquirable. + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); @@ -1509,7 +1591,7 @@ public void testRemoteStorageFetchRequestCompletionOnFutureCompletionSuccessfull assertTrue(delayedShareFetch.isCompleted()); // Pending remote fetch object gets created for delayed share fetch. - assertNotNull(delayedShareFetch.remoteFetch()); + assertNotNull(delayedShareFetch.pendingRemoteFetches()); // Verify the locks are released for tp0. Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0)); assertTrue(shareFetch.isCompleted()); @@ -1530,10 +1612,6 @@ public void testRemoteStorageFetchRequestCompletionAlongWithLocalLogRead() { SharePartition sp1 = mock(SharePartition.class); SharePartition sp2 = mock(SharePartition.class); - // All the topic partitions are acquirable. - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - when(sp2.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); when(sp2.canAcquireRecords()).thenReturn(true); @@ -1584,19 +1662,26 @@ public void testRemoteStorageFetchRequestCompletionAlongWithLocalLogRead() { }).when(remoteLogManager).asyncRead(any(), any()); when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withReplicaManager(replicaManager) .withSharePartitions(sharePartitions) .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1, tp2))) + .withFetchId(fetchId) .build()); + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp2.maybeAcquireFetchLock(fetchId)).thenReturn(true); + assertFalse(delayedShareFetch.isCompleted()); assertTrue(delayedShareFetch.tryComplete()); assertTrue(delayedShareFetch.isCompleted()); // Pending remote fetch object gets created for delayed share fetch. - assertNotNull(delayedShareFetch.remoteFetch()); + assertNotNull(delayedShareFetch.pendingRemoteFetches()); // the future of shareFetch completes. assertTrue(shareFetch.isCompleted()); assertEquals(Set.of(tp0, tp1, tp2), future.join().keySet()); @@ -1610,7 +1695,7 @@ public void testRemoteStorageFetchRequestCompletionAlongWithLocalLogRead() { } @Test - public void testRemoteStorageFetchOnlyHappensForFirstTopicPartition() { + public void testRemoteStorageFetchHappensForAllTopicPartitions() { ReplicaManager replicaManager = mock(ReplicaManager.class); TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); @@ -1618,9 +1703,6 @@ public void testRemoteStorageFetchOnlyHappensForFirstTopicPartition() { SharePartition sp0 = mock(SharePartition.class); SharePartition sp1 = mock(SharePartition.class); - // sp0 and sp1 are acquirable. - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); @@ -1660,31 +1742,37 @@ public void testRemoteStorageFetchOnlyHappensForFirstTopicPartition() { }).when(remoteLogManager).asyncRead(any(), any()); when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withReplicaManager(replicaManager) .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1))) + .withFetchId(fetchId) .build()); + // sp0 and sp1 are acquirable. + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp1.maybeAcquireFetchLock(fetchId)).thenReturn(true); + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + when(sp1.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); assertFalse(delayedShareFetch.isCompleted()); assertTrue(delayedShareFetch.tryComplete()); assertTrue(delayedShareFetch.isCompleted()); // Pending remote fetch object gets created for delayed share fetch. - assertNotNull(delayedShareFetch.remoteFetch()); - // Verify the locks are released separately for tp1 (from tryComplete). - Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp1)); - // From onComplete, the locks will be released for both tp0 and tp1. tp0 because it was acquired from - // tryComplete and has remote fetch processed. tp1 will be reacquired in onComplete when we check for local log read. + assertNotNull(delayedShareFetch.pendingRemoteFetches()); + // Verify the locks are released for both tp0 and tp1. Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0, tp1)); assertTrue(shareFetch.isCompleted()); - // Share fetch response only contains the first remote storage fetch topic partition - tp0. - assertEquals(Set.of(tp0), future.join().keySet()); + // Share fetch response contains both remote storage fetch topic partitions. + assertEquals(Set.of(tp0, tp1), future.join().keySet()); assertEquals(Errors.NONE.code(), future.join().get(tp0).errorCode()); + assertEquals(Errors.NONE.code(), future.join().get(tp1).errorCode()); assertTrue(delayedShareFetch.lock().tryLock()); delayedShareFetch.lock().unlock(); } @@ -1759,8 +1847,9 @@ static class DelayedShareFetchBuilder { private LinkedHashMap sharePartitions = mock(LinkedHashMap.class); private PartitionMaxBytesStrategy partitionMaxBytesStrategy = mock(PartitionMaxBytesStrategy.class); private Time time = new MockTime(); - private final Optional remoteFetch = Optional.empty(); + private final Optional pendingRemoteFetches = Optional.empty(); private ShareGroupMetrics shareGroupMetrics = mock(ShareGroupMetrics.class); + private Uuid fetchId = Uuid.randomUuid(); DelayedShareFetchBuilder withShareFetchData(ShareFetch shareFetch) { this.shareFetch = shareFetch; @@ -1797,6 +1886,11 @@ private DelayedShareFetchBuilder withTime(Time time) { return this; } + private DelayedShareFetchBuilder withFetchId(Uuid fetchId) { + this.fetchId = fetchId; + return this; + } + public static DelayedShareFetchBuilder builder() { return new DelayedShareFetchBuilder(); } @@ -1810,7 +1904,8 @@ public DelayedShareFetch build() { partitionMaxBytesStrategy, shareGroupMetrics, time, - remoteFetch); + pendingRemoteFetches, + fetchId); } } } diff --git a/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java b/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java index 3bec497d7a1f2..e3a77158dafc4 100644 --- a/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java +++ b/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java @@ -45,9 +45,15 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.ArgumentsProvider; +import org.junit.jupiter.params.provider.ArgumentsSource; import org.mockito.Mockito; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -56,6 +62,7 @@ import java.util.OptionalLong; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; +import java.util.stream.Stream; import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.createFileRecords; import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.createShareAcquiredRecords; @@ -462,11 +469,9 @@ public void testProcessFetchResponseWithOffsetFetchException() { Mockito.verify(sp0, times(0)).updateCacheAndOffsets(any(Long.class)); } - @Test - public void testMaybeSliceFetchRecordsSingleBatch() throws IOException { - // Create 1 batch of records with 10 records. - FileRecords records = createFileRecords(Map.of(5L, 10)); - + @ParameterizedTest(name = "{0}") + @ArgumentsSource(RecordsArgumentsProvider.class) + public void testMaybeSliceFetchRecordsSingleBatch(String name, Records records) { // Acquire all offsets, should return same records. List acquiredRecords = List.of(new AcquiredRecords().setFirstOffset(5).setLastOffset(14).setDeliveryCount((short) 1)); Records slicedRecords = ShareFetchUtils.maybeSliceFetchRecords(records, new ShareAcquiredRecords(acquiredRecords, 10)); @@ -498,15 +503,9 @@ public void testMaybeSliceFetchRecordsSingleBatch() throws IOException { assertEquals(records, slicedRecords); } - @Test - public void testMaybeSliceFetchRecordsMultipleBatches() throws IOException { - // Create 3 batches of records with 3, 2 and 4 records respectively. - LinkedHashMap recordsPerOffset = new LinkedHashMap<>(); - recordsPerOffset.put(0L, 3); - recordsPerOffset.put(3L, 2); - recordsPerOffset.put(7L, 4); // Gap of 2 offsets between batches. - FileRecords records = createFileRecords(recordsPerOffset); - + @ParameterizedTest(name = "{0}") + @ArgumentsSource(MultipleBatchesRecordsArgumentsProvider.class) + public void testMaybeSliceFetchRecordsMultipleBatches(String name, Records records) { // Acquire all offsets, should return same records. List acquiredRecords = List.of(new AcquiredRecords().setFirstOffset(0).setLastOffset(10).setDeliveryCount((short) 1)); Records slicedRecords = ShareFetchUtils.maybeSliceFetchRecords(records, new ShareAcquiredRecords(acquiredRecords, 11)); @@ -617,10 +616,9 @@ public void testMaybeSliceFetchRecordsMultipleBatches() throws IOException { assertEquals(records.sizeInBytes(), slicedRecords.sizeInBytes()); } - @Test - public void testMaybeSliceFetchRecordsException() throws IOException { - // Create 1 batch of records with 3 records. - FileRecords records = createFileRecords(Map.of(0L, 3)); + @ParameterizedTest(name = "{0}") + @ArgumentsSource(MultipleBatchesRecordsArgumentsProvider.class) + public void testMaybeSliceFetchRecordsException(String name, Records records) { // Send empty acquired records which should trigger an exception and same file records should // be returned. The method doesn't expect empty acquired records. Records slicedRecords = ShareFetchUtils.maybeSliceFetchRecords( @@ -628,14 +626,41 @@ public void testMaybeSliceFetchRecordsException() throws IOException { assertEquals(records, slicedRecords); } - @Test - public void testMaybeSliceFetchRecordsNonFileRecords() { - // Send memory records which should be returned as is. - try (MemoryRecordsBuilder records = memoryRecordsBuilder(2, 0)) { - List acquiredRecords = List.of(new AcquiredRecords().setFirstOffset(0).setLastOffset(1).setDeliveryCount((short) 1)); - Records slicedRecords = ShareFetchUtils.maybeSliceFetchRecords( - records.build(), new ShareAcquiredRecords(acquiredRecords, 2)); - assertEquals(records.build(), slicedRecords); + private static class RecordsArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(ExtensionContext context) throws Exception { + return Stream.of( + Arguments.of("FileRecords", createFileRecords(Map.of(5L, 10))), + Arguments.of("MemoryRecords", createMemoryRecords(5L, 10)) + ); + } + + private MemoryRecords createMemoryRecords(long baseOffset, int numRecords) { + try (MemoryRecordsBuilder recordsBuilder = memoryRecordsBuilder(numRecords, baseOffset)) { + return recordsBuilder.build(); + } + } + } + + private static class MultipleBatchesRecordsArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(ExtensionContext context) throws Exception { + LinkedHashMap recordsPerOffset = new LinkedHashMap<>(); + recordsPerOffset.put(0L, 3); + recordsPerOffset.put(3L, 2); + recordsPerOffset.put(7L, 4); // Gap of 2 offsets between batches. + return Stream.of( + Arguments.of("FileRecords", createFileRecords(recordsPerOffset)), + Arguments.of("MemoryRecords", createMemoryRecords(recordsPerOffset)) + ); + } + + private MemoryRecords createMemoryRecords(Map recordsPerOffset) { + ByteBuffer buffer = ByteBuffer.allocate(1024); + recordsPerOffset.forEach((offset, numOfRecords) -> memoryRecordsBuilder(buffer, numOfRecords, offset).close()); + buffer.flip(); + + return MemoryRecords.readableRecords(buffer); } } } diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index a45724dd1e767..15b943499e839 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -153,6 +153,7 @@ public class SharePartitionManagerTest { FetchRequest.ORDINARY_CONSUMER_ID, -1, DELAYED_SHARE_FETCH_MAX_WAIT_MS, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty(), true); private static final String TIMER_NAME_PREFIX = "share-partition-manager"; + private static final String CONNECTION_ID = "id-1"; static final int DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL = 1000; @@ -200,12 +201,12 @@ public void testNewContextReturnsFinalContextWithoutRequestData() { List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context1); assertFalse(((ShareSessionContext) context1).isSubsequent()); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(memberId, ShareRequestMetadata.FINAL_EPOCH); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), List.of(), reqMetadata2, true); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), List.of(), reqMetadata2, true, CONNECTION_ID); assertEquals(FinalContext.class, context2.getClass()); } @@ -217,7 +218,6 @@ public void testNewContextReturnsFinalContextWithRequestData() { .build(); Uuid tpId0 = Uuid.randomUuid(); - Uuid tpId1 = Uuid.randomUuid(); TopicIdPartition tp0 = new TopicIdPartition(tpId0, new TopicPartition("foo", 0)); TopicIdPartition tp1 = new TopicIdPartition(tpId0, new TopicPartition("foo", 1)); @@ -228,16 +228,15 @@ public void testNewContextReturnsFinalContextWithRequestData() { List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context1); assertFalse(((ShareSessionContext) context1).isSubsequent()); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(memberId, ShareRequestMetadata.FINAL_EPOCH); - // shareFetch is not empty, but the maxBytes of topic partition is 0, which means this is added only for acknowledgements. - // New context should be created successfully - List reqData3 = List.of(new TopicIdPartition(tpId1, new TopicPartition("foo", 0))); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData3, List.of(), reqMetadata2, true); + // Sending a Request with FINAL_EPOCH. This should return a FinalContext. + List reqData2 = List.of(tp0, tp1); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, true, CONNECTION_ID); assertEquals(FinalContext.class, context2.getClass()); } @@ -260,16 +259,16 @@ public void testNewContextReturnsFinalContextWhenTopicPartitionsArePresentInRequ List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context1); assertFalse(((ShareSessionContext) context1).isSubsequent()); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(memberId, ShareRequestMetadata.FINAL_EPOCH); // shareFetch is not empty, and it contains tpId1, which should return FinalContext instance since it is FINAL_EPOCH - List reqData3 = List.of(new TopicIdPartition(tpId1, new TopicPartition("foo", 0))); + List reqData2 = List.of(new TopicIdPartition(tpId1, new TopicPartition("foo", 0))); assertInstanceOf(FinalContext.class, - sharePartitionManager.newContext(groupId, reqData3, List.of(), reqMetadata2, true)); + sharePartitionManager.newContext(groupId, reqData2, List.of(), reqMetadata2, true, CONNECTION_ID)); } @Test @@ -295,7 +294,7 @@ public void testNewContext() { List reqData2 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context2); assertFalse(((ShareSessionContext) context2).isSubsequent()); @@ -314,16 +313,16 @@ public void testNewContext() { // Test trying to create a new session with an invalid epoch assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, "id-2")); // Test trying to create a new session with a non-existent session key Uuid memberId4 = Uuid.randomUuid(); assertThrows(ShareSessionNotFoundException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(memberId4, 1), true)); + new ShareRequestMetadata(memberId4, 1), true, "id-3")); // Continue the first share session we created. ShareFetchContext context5 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); + new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context5); assertTrue(((ShareSessionContext) context5).isSubsequent()); @@ -341,18 +340,18 @@ public void testNewContext() { // Test setting an invalid share session epoch. assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, CONNECTION_ID)); // Test generating a throttled response for a subsequent share session ShareFetchContext context7 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); + new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true, CONNECTION_ID); ShareFetchResponse resp7 = context7.throttleResponse(100); assertEquals(Errors.NONE, resp7.error()); assertEquals(100, resp7.throttleTimeMs()); // Get the final share session. ShareFetchContext context8 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); + new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true, CONNECTION_ID); assertEquals(FinalContext.class, context8.getClass()); assertEquals(1, cache.size()); @@ -389,7 +388,7 @@ public void testSubsequentShareSession() { String groupId = "grp"; ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context1); LinkedHashMap respData1 = new LinkedHashMap<>(); @@ -405,7 +404,7 @@ public void testSubsequentShareSession() { List removed2 = new ArrayList<>(); removed2.add(tp0); ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, removed2, - new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); + new ShareRequestMetadata(reqMetadata1.memberId(), 1), true, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context2); Set expectedTopicIdPartitions2 = new HashSet<>(); @@ -452,7 +451,7 @@ public void testZeroSizeShareSession() { String groupId = "grp"; ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context1); LinkedHashMap respData1 = new LinkedHashMap<>(); @@ -469,7 +468,7 @@ public void testZeroSizeShareSession() { removed2.add(foo0); removed2.add(foo1); ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), removed2, - new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); + new ShareRequestMetadata(reqMetadata1.memberId(), 1), true, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context2); LinkedHashMap respData2 = new LinkedHashMap<>(); @@ -495,14 +494,14 @@ public void testToForgetPartitions() { List reqData1 = List.of(foo, bar); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context1); assertPartitionsPresent((ShareSessionContext) context1, List.of(foo, bar)); mockUpdateAndGenerateResponseData(context1, groupId, reqMetadata1.memberId()); ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), List.of(foo), - new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); + new ShareRequestMetadata(reqMetadata1.memberId(), 1), true, CONNECTION_ID); // So foo is removed but not the others. assertPartitionsPresent((ShareSessionContext) context2, List.of(bar)); @@ -510,7 +509,7 @@ public void testToForgetPartitions() { mockUpdateAndGenerateResponseData(context2, groupId, reqMetadata1.memberId()); ShareFetchContext context3 = sharePartitionManager.newContext(groupId, List.of(), List.of(bar), - new ShareRequestMetadata(reqMetadata1.memberId(), 2), true); + new ShareRequestMetadata(reqMetadata1.memberId(), 2), true, CONNECTION_ID); assertPartitionsPresent((ShareSessionContext) context3, List.of()); } @@ -537,7 +536,7 @@ public void testShareSessionUpdateTopicIdsBrokerSide() { List reqData1 = List.of(foo, bar); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context1); assertFalse(((ShareSessionContext) context1).isSubsequent()); @@ -553,7 +552,7 @@ public void testShareSessionUpdateTopicIdsBrokerSide() { // Create a subsequent share fetch request as though no topics changed. ShareFetchContext context2 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); + new ShareRequestMetadata(reqMetadata1.memberId(), 1), true, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context2); assertTrue(((ShareSessionContext) context2).isSubsequent()); @@ -587,7 +586,7 @@ public void testGetErroneousAndValidTopicIdPartitions() { List reqData2 = List.of(tp0, tp1, tpNull1); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context2); assertFalse(((ShareSessionContext) context2).isSubsequent()); assertErroneousAndValidTopicIdPartitions(context2.getErroneousAndValidTopicIdPartitions(), List.of(tpNull1), List.of(tp0, tp1)); @@ -609,15 +608,15 @@ public void testGetErroneousAndValidTopicIdPartitions() { // Test trying to create a new session with an invalid epoch assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, CONNECTION_ID)); // Test trying to create a new session with a non-existent session key assertThrows(ShareSessionNotFoundException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(Uuid.randomUuid(), 1), true)); + new ShareRequestMetadata(Uuid.randomUuid(), 1), true, CONNECTION_ID)); // Continue the first share session we created. ShareFetchContext context5 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); + new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context5); assertTrue(((ShareSessionContext) context5).isSubsequent()); @@ -628,12 +627,12 @@ public void testGetErroneousAndValidTopicIdPartitions() { // Test setting an invalid share session epoch. assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, CONNECTION_ID)); // Test generating a throttled response for a subsequent share session List reqData7 = List.of(tpNull2); ShareFetchContext context7 = sharePartitionManager.newContext(groupId, reqData7, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); + new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true, CONNECTION_ID); // Check for throttled response ShareFetchResponse resp7 = context7.throttleResponse(100); assertEquals(Errors.NONE, resp7.error()); @@ -643,7 +642,7 @@ public void testGetErroneousAndValidTopicIdPartitions() { // Get the final share session. ShareFetchContext context8 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); + new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true, CONNECTION_ID); assertEquals(FinalContext.class, context8.getClass()); assertEquals(1, cache.size()); @@ -688,7 +687,7 @@ public void testShareFetchContextResponseSize() { short version = ApiKeys.SHARE_FETCH.latestVersion(); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context2); assertFalse(((ShareSessionContext) context2).isSubsequent()); @@ -708,17 +707,17 @@ public void testShareFetchContextResponseSize() { // Test trying to create a new session with an invalid epoch assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, CONNECTION_ID)); // Test trying to create a new session with a non-existent session key Uuid memberId4 = Uuid.randomUuid(); assertThrows(ShareSessionNotFoundException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(memberId4, 1), true)); + new ShareRequestMetadata(memberId4, 1), true, CONNECTION_ID)); // Continue the first share session we created. List reqData5 = List.of(tp2); ShareFetchContext context5 = sharePartitionManager.newContext(groupId, reqData5, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); + new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context5); assertTrue(((ShareSessionContext) context5).isSubsequent()); @@ -733,11 +732,11 @@ public void testShareFetchContextResponseSize() { // Test setting an invalid share session epoch. assertThrows(InvalidShareSessionEpochException.class, () -> sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true)); + new ShareRequestMetadata(shareSessionKey2.memberId(), 5), true, CONNECTION_ID)); // Test generating a throttled response for a subsequent share session ShareFetchContext context7 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); + new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true, CONNECTION_ID); int respSize7 = context7.responseSize(respData2, version); ShareFetchResponse resp7 = context7.throttleResponse(100); @@ -748,7 +747,7 @@ public void testShareFetchContextResponseSize() { // Get the final share session. ShareFetchContext context8 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); + new ShareRequestMetadata(reqMetadata2.memberId(), ShareRequestMetadata.FINAL_EPOCH), true, CONNECTION_ID); assertEquals(FinalContext.class, context8.getClass()); assertEquals(1, cache.size()); @@ -794,7 +793,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { List reqData1 = List.of(tp0, tp1); ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(memberId1, ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false); + ShareFetchContext context1 = sharePartitionManager.newContext(groupId, reqData1, EMPTY_PART_LIST, reqMetadata1, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context1); assertFalse(((ShareSessionContext) context1).isSubsequent()); @@ -815,7 +814,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { List reqData2 = List.of(tp2); ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(memberId2, ShareRequestMetadata.INITIAL_EPOCH); - ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false); + ShareFetchContext context2 = sharePartitionManager.newContext(groupId, reqData2, EMPTY_PART_LIST, reqMetadata2, false, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context2); assertFalse(((ShareSessionContext) context2).isSubsequent()); @@ -833,7 +832,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { // Continue the first share session we created. List reqData3 = List.of(tp2); ShareFetchContext context3 = sharePartitionManager.newContext(groupId, reqData3, EMPTY_PART_LIST, - new ShareRequestMetadata(shareSessionKey1.memberId(), 1), true); + new ShareRequestMetadata(shareSessionKey1.memberId(), 1), true, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context3); assertTrue(((ShareSessionContext) context3).isSubsequent()); @@ -848,7 +847,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { // Continue the second session we created. List reqData4 = List.of(tp3); ShareFetchContext context4 = sharePartitionManager.newContext(groupId, reqData4, List.of(tp2), - new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true); + new ShareRequestMetadata(shareSessionKey2.memberId(), 1), true, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context4); assertTrue(((ShareSessionContext) context4).isSubsequent()); @@ -861,7 +860,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { // Get the final share session. ShareFetchContext context5 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata1.memberId(), ShareRequestMetadata.FINAL_EPOCH), true); + new ShareRequestMetadata(reqMetadata1.memberId(), ShareRequestMetadata.FINAL_EPOCH), true, CONNECTION_ID); assertEquals(FinalContext.class, context5.getClass()); LinkedHashMap respData5 = new LinkedHashMap<>(); @@ -876,7 +875,7 @@ public void testCachedTopicPartitionsForValidShareSessions() { // Continue the second share session . ShareFetchContext context6 = sharePartitionManager.newContext(groupId, List.of(), List.of(tp3), - new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true); + new ShareRequestMetadata(shareSessionKey2.memberId(), 2), true, CONNECTION_ID); assertInstanceOf(ShareSessionContext.class, context6); assertTrue(((ShareSessionContext) context6).isSubsequent()); @@ -1016,10 +1015,10 @@ public void testMultipleConcurrentShareFetches() throws InterruptedException { when(sp2.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null)); when(sp3.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null)); // Required mocks so that the share partitions can acquire record. - when(sp0.maybeAcquireFetchLock()).thenReturn(true); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); - when(sp2.maybeAcquireFetchLock()).thenReturn(true); - when(sp3.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.maybeAcquireFetchLock(any())).thenReturn(true); + when(sp1.maybeAcquireFetchLock(any())).thenReturn(true); + when(sp2.maybeAcquireFetchLock(any())).thenReturn(true); + when(sp3.maybeAcquireFetchLock(any())).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); when(sp2.canAcquireRecords()).thenReturn(true); @@ -1120,7 +1119,7 @@ public void testReplicaManagerFetchShouldNotProceed() { List topicIdPartitions = List.of(tp0); SharePartition sp0 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.maybeAcquireFetchLock(any())).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(false); when(sp0.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null)); Map partitionCacheMap = new HashMap<>(); @@ -1689,9 +1688,9 @@ public void testAcknowledgeCompletesDelayedShareFetchRequest() { mockTopicIdPartitionToReturnDataEqualToMinBytes(mockReplicaManager, tp1, 2); // Initially you cannot acquire records for both sp1 and sp2. - when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock(any())).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(false); - when(sp2.maybeAcquireFetchLock()).thenReturn(true); + when(sp2.maybeAcquireFetchLock(any())).thenReturn(true); when(sp2.canAcquireRecords()).thenReturn(false); when(sp1.acquire(anyString(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn(ShareAcquiredRecords.empty()); when(sp2.acquire(anyString(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn(ShareAcquiredRecords.empty()); @@ -1798,11 +1797,11 @@ public void testAcknowledgeDoesNotCompleteDelayedShareFetchRequest() { mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory); // Initially you cannot acquire records for both all 3 share partitions. - when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock(any())).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(false); - when(sp2.maybeAcquireFetchLock()).thenReturn(true); + when(sp2.maybeAcquireFetchLock(any())).thenReturn(true); when(sp2.canAcquireRecords()).thenReturn(false); - when(sp3.maybeAcquireFetchLock()).thenReturn(true); + when(sp3.maybeAcquireFetchLock(any())).thenReturn(true); when(sp3.canAcquireRecords()).thenReturn(false); List delayedShareFetchWatchKeys = new ArrayList<>(); @@ -1904,9 +1903,9 @@ public void testReleaseSessionCompletesDelayedShareFetchRequest() { mockTopicIdPartitionToReturnDataEqualToMinBytes(mockReplicaManager, tp1, 1); // Initially you cannot acquire records for both sp1 and sp2. - when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock(any())).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(false); - when(sp2.maybeAcquireFetchLock()).thenReturn(true); + when(sp2.maybeAcquireFetchLock(any())).thenReturn(true); when(sp2.canAcquireRecords()).thenReturn(false); List delayedShareFetchWatchKeys = new ArrayList<>(); @@ -2008,11 +2007,11 @@ public void testReleaseSessionDoesNotCompleteDelayedShareFetchRequest() { mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory); // Initially you cannot acquire records for both all 3 share partitions. - when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock(any())).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(false); - when(sp2.maybeAcquireFetchLock()).thenReturn(true); + when(sp2.maybeAcquireFetchLock(any())).thenReturn(true); when(sp2.canAcquireRecords()).thenReturn(false); - when(sp3.maybeAcquireFetchLock()).thenReturn(true); + when(sp3.maybeAcquireFetchLock(any())).thenReturn(true); when(sp3.canAcquireRecords()).thenReturn(false); List delayedShareFetchWatchKeys = new ArrayList<>(); @@ -2506,7 +2505,7 @@ public void testSharePartitionPartialInitializationFailure() throws Exception { Map partitionCacheMap = new HashMap<>(); partitionCacheMap.put(new SharePartitionKey(groupId, tp1), sp1); - when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock(any())).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); when(sp1.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null)); when(sp1.acquire(anyString(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn(new ShareAcquiredRecords(List.of(), 0)); @@ -2570,7 +2569,7 @@ public void testReplicaManagerFetchException() { List topicIdPartitions = List.of(tp0); SharePartition sp0 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.maybeAcquireFetchLock(any())).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); when(sp0.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null)); Map partitionCacheMap = new HashMap<>(); @@ -2623,14 +2622,14 @@ public void testReplicaManagerFetchMultipleSharePartitionsException() { List topicIdPartitions = List.of(tp0, tp1); SharePartition sp0 = mock(SharePartition.class); - when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.maybeAcquireFetchLock(any())).thenReturn(true); when(sp0.canAcquireRecords()).thenReturn(true); when(sp0.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null)); SharePartition sp1 = mock(SharePartition.class); // Do not make the share partition acquirable hence it shouldn't be removed from the cache, // as it won't be part of replica manager readFromLog request. - when(sp1.maybeAcquireFetchLock()).thenReturn(false); + when(sp1.maybeAcquireFetchLock(any())).thenReturn(false); when(sp1.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null)); Map partitionCacheMap = new HashMap<>(); @@ -2663,7 +2662,7 @@ public void testReplicaManagerFetchMultipleSharePartitionsException() { // Make sp1 acquirable and add sp0 back in partition cache. Both share partitions should be // removed from the cache. - when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock(any())).thenReturn(true); when(sp1.canAcquireRecords()).thenReturn(true); partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0); // Throw FencedStateEpochException from replica manager fetch which should evict instance from the cache. diff --git a/core/src/test/java/kafka/server/share/SharePartitionTest.java b/core/src/test/java/kafka/server/share/SharePartitionTest.java index 64781648774d6..ac8e9ba0fd6d8 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionTest.java @@ -55,6 +55,7 @@ import org.apache.kafka.coordinator.group.GroupConfigManager; import org.apache.kafka.coordinator.group.ShareGroupAutoOffsetResetStrategy; import org.apache.kafka.server.share.acknowledge.ShareAcknowledgementBatch; +import org.apache.kafka.server.share.fetch.DelayedShareFetchGroupKey; import org.apache.kafka.server.share.fetch.ShareAcquiredRecords; import org.apache.kafka.server.share.metrics.SharePartitionMetrics; import org.apache.kafka.server.share.persister.NoOpStatePersister; @@ -67,8 +68,7 @@ import org.apache.kafka.server.storage.log.FetchIsolation; import org.apache.kafka.server.storage.log.FetchPartitionData; import org.apache.kafka.server.util.FutureUtils; -import org.apache.kafka.server.util.timer.SystemTimer; -import org.apache.kafka.server.util.timer.SystemTimerReaper; +import org.apache.kafka.server.util.timer.MockTimer; import org.apache.kafka.server.util.timer.Timer; import org.apache.kafka.storage.internals.log.OffsetResultHolder; import org.apache.kafka.test.TestUtils; @@ -117,7 +117,7 @@ public class SharePartitionTest { private static final Time MOCK_TIME = new MockTime(); private static final short MAX_IN_FLIGHT_MESSAGES = 200; private static final int ACQUISITION_LOCK_TIMEOUT_MS = 100; - private static final int DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS = 300; + private static final int DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS = 120; private static final int BATCH_SIZE = 500; private static final int DEFAULT_FETCH_OFFSET = 0; private static final int MAX_FETCH_RECORDS = Integer.MAX_VALUE; @@ -129,8 +129,7 @@ public class SharePartitionTest { @BeforeEach public void setUp() { kafka.utils.TestUtils.clearYammerMetrics(); - mockTimer = new SystemTimerReaper("share-group-lock-timeout-test-reaper", - new SystemTimer("share-group-lock-test-timeout")); + mockTimer = new MockTimer(); sharePartitionMetrics = new SharePartitionMetrics(GROUP_ID, TOPIC_ID_PARTITION.topic(), TOPIC_ID_PARTITION.partition()); } @@ -1605,12 +1604,14 @@ public void testMaybeAcquireAndReleaseFetchLock() { .withSharePartitionMetrics(sharePartitionMetrics) .build(); + Uuid fetchId = Uuid.randomUuid(); + sharePartition.maybeInitialize(); - assertTrue(sharePartition.maybeAcquireFetchLock()); + assertTrue(sharePartition.maybeAcquireFetchLock(fetchId)); // Lock cannot be acquired again, as already acquired. - assertFalse(sharePartition.maybeAcquireFetchLock()); + assertFalse(sharePartition.maybeAcquireFetchLock(fetchId)); // Release the lock. - sharePartition.releaseFetchLock(); + sharePartition.releaseFetchLock(fetchId); assertEquals(1, sharePartitionMetrics.fetchLockTimeMs().count()); assertEquals(10, sharePartitionMetrics.fetchLockTimeMs().sum()); @@ -1619,9 +1620,9 @@ public void testMaybeAcquireAndReleaseFetchLock() { assertEquals(100, sharePartitionMetrics.fetchLockRatio().mean()); // Lock can be acquired again. - assertTrue(sharePartition.maybeAcquireFetchLock()); + assertTrue(sharePartition.maybeAcquireFetchLock(fetchId)); // Release lock to update metrics and verify. - sharePartition.releaseFetchLock(); + sharePartition.releaseFetchLock(fetchId); assertEquals(2, sharePartitionMetrics.fetchLockTimeMs().count()); assertEquals(40, sharePartitionMetrics.fetchLockTimeMs().sum()); @@ -1649,14 +1650,15 @@ public void testRecordFetchLockRatioMetric() { .thenReturn(80L) // for time when lock is released .thenReturn(160L); // to update lock idle duration while acquiring lock again. - assertTrue(sharePartition.maybeAcquireFetchLock()); - sharePartition.releaseFetchLock(); + Uuid fetchId = Uuid.randomUuid(); + assertTrue(sharePartition.maybeAcquireFetchLock(fetchId)); + sharePartition.releaseFetchLock(fetchId); // Acquired time is 70 but last lock acquisition time was still 0, as it's the first request // when last acquisition time was recorded. The last acquisition time should be updated to 80. assertEquals(2, sharePartitionMetrics.fetchLockRatio().count()); assertEquals(100, sharePartitionMetrics.fetchLockRatio().mean()); - assertTrue(sharePartition.maybeAcquireFetchLock()); + assertTrue(sharePartition.maybeAcquireFetchLock(fetchId)); // Update metric again with 0 as acquire time and 80 as idle duration ms. sharePartition.recordFetchLockRatioMetric(0); assertEquals(3, sharePartitionMetrics.fetchLockRatio().count()); @@ -1672,7 +1674,11 @@ public void testRecordFetchLockRatioMetric() { @Test public void testAcknowledgeSingleRecordBatch() { - SharePartition sharePartition = SharePartitionBuilder.builder().withState(SharePartitionState.ACTIVE).build(); + ReplicaManager replicaManager = Mockito.mock(ReplicaManager.class); + SharePartition sharePartition = SharePartitionBuilder.builder() + .withReplicaManager(replicaManager) + .withState(SharePartitionState.ACTIVE) + .build(); MemoryRecords records1 = memoryRecords(1, 0); MemoryRecords records2 = memoryRecords(1, 1); @@ -1695,11 +1701,18 @@ public void testAcknowledgeSingleRecordBatch() { assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(1L).batchState()); assertEquals(1, sharePartition.cachedState().get(1L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(1L).offsetState()); + // Should not invoke completeDelayedShareFetchRequest as the first offset is not acknowledged yet. + Mockito.verify(replicaManager, Mockito.times(0)) + .completeDelayedShareFetchRequest(new DelayedShareFetchGroupKey(GROUP_ID, TOPIC_ID_PARTITION)); } @Test public void testAcknowledgeMultipleRecordBatch() { - SharePartition sharePartition = SharePartitionBuilder.builder().withState(SharePartitionState.ACTIVE).build(); + ReplicaManager replicaManager = Mockito.mock(ReplicaManager.class); + SharePartition sharePartition = SharePartitionBuilder.builder() + .withReplicaManager(replicaManager) + .withState(SharePartitionState.ACTIVE) + .build(); MemoryRecords records = memoryRecords(10, 5); List acquiredRecordsList = fetchAcquiredRecords(sharePartition, records, 10); @@ -1713,6 +1726,9 @@ public void testAcknowledgeMultipleRecordBatch() { assertEquals(15, sharePartition.nextFetchOffset()); assertEquals(0, sharePartition.cachedState().size()); + // Should invoke completeDelayedShareFetchRequest as the start offset is moved. + Mockito.verify(replicaManager, Mockito.times(1)) + .completeDelayedShareFetchRequest(new DelayedShareFetchGroupKey(GROUP_ID, TOPIC_ID_PARTITION)); } @Test @@ -2925,6 +2941,7 @@ public void testAcquisitionLockForAcquiringSingleRecord() throws InterruptedExce assertEquals(1, sharePartition.timer().size()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.nextFetchOffset() == 0 && sharePartition.cachedState().get(0L).batchState() == RecordState.AVAILABLE && @@ -2951,6 +2968,7 @@ public void testAcquisitionLockForAcquiringMultipleRecords() throws InterruptedE assertNotNull(sharePartition.cachedState().get(10L).batchAcquisitionLockTimeoutTask()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && sharePartition.nextFetchOffset() == 10 @@ -2985,6 +3003,7 @@ public void testAcquisitionLockForAcquiringMultipleRecordsWithOverlapAndNewBatch assertEquals(2, sharePartition.timer().size()); // Allowing acquisition lock to expire. The acquisition lock timeout will cause release of records for all the acquired records. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && sharePartition.nextFetchOffset() == 0 && @@ -3012,6 +3031,7 @@ public void testAcquisitionLockForAcquiringSameBatchAgain() throws InterruptedEx assertEquals(1, sharePartition.timer().size()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && sharePartition.nextFetchOffset() == 10 && @@ -3128,6 +3148,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleRecordBatchWithGapOffsets( // Allowing acquisition lock to expire. The acquisition lock timeout will cause release of records for batch with starting offset 1. // Since, other records have been acknowledged. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && sharePartition.nextFetchOffset() == 1 && @@ -3155,6 +3176,7 @@ public void testAcquisitionLockForAcquiringSubsetBatchAgain() throws Interrupted assertEquals(1, sharePartition.timer().size()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && sharePartition.nextFetchOffset() == 10 && @@ -3179,6 +3201,7 @@ public void testAcquisitionLockForAcquiringSubsetBatchAgain() throws Interrupted assertEquals(3, sharePartition.timer().size()); // Allowing acquisition lock to expire for the acquired subset batch. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> { Map expectedOffsetStateMap = new HashMap<>(); @@ -3259,6 +3282,7 @@ public void testAcquisitionLockOnAcknowledgingMultipleSubsetRecordBatchWithGapOf assertEquals(3, sharePartition.timer().size()); // Allowing acquisition lock to expire for the offsets that have not been acknowledged yet. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> { Map expectedOffsetStateMap1 = new HashMap<>(); @@ -3321,6 +3345,7 @@ public void testAcquisitionLockTimeoutCauseMaxDeliveryCountExceed() throws Inter assertEquals(2, sharePartition.timer().size()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && sharePartition.nextFetchOffset() == 0 && @@ -3338,6 +3363,7 @@ public void testAcquisitionLockTimeoutCauseMaxDeliveryCountExceed() throws Inter assertEquals(1, sharePartition.timer().size()); // Allowing acquisition lock to expire to archive the records that reach max delivery count. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && sharePartition.nextFetchOffset() == 0 && @@ -3363,6 +3389,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForward() throws InterruptedE assertEquals(1, sharePartition.timer().size()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && sharePartition.nextFetchOffset() == 0 && @@ -3386,6 +3413,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForward() throws InterruptedE assertNull(sharePartition.cachedState().get(0L).offsetState().get(9L).acquisitionLockTimeoutTask()); // Allowing acquisition lock to expire to archive the records that reach max delivery count. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> { Map expectedOffsetStateMap = new HashMap<>(); @@ -3436,6 +3464,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForwardAndClearCachedState() assertEquals(1, sharePartition.timer().size()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && sharePartition.nextFetchOffset() == 0 && @@ -3450,6 +3479,7 @@ public void testAcquisitionLockTimeoutCauseSPSOMoveForwardAndClearCachedState() assertEquals(1, sharePartition.timer().size()); // Allowing acquisition lock to expire to archive the records that reach max delivery count. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && // After the second failed attempt to acknowledge the record batch successfully, the record batch is archived. @@ -3473,6 +3503,7 @@ public void testAcknowledgeAfterAcquisitionLockTimeout() throws InterruptedExcep assertEquals(1, sharePartition.timer().size()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && sharePartition.nextFetchOffset() == 5 && @@ -3531,6 +3562,7 @@ public void testAcquisitionLockAfterDifferentAcknowledges() throws InterruptedEx assertEquals(1, sharePartition.timer().size()); // Allowing acquisition lock to expire will only affect the offsets that have not been acknowledged yet. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> { // Check cached state. @@ -3576,6 +3608,7 @@ public void testAcquisitionLockOnBatchWithWriteShareGroupStateFailure() throws I assertNotNull(sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask()); // Allowing acquisition lock to expire. Even if write share group state RPC fails, state transition still happens. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.timer().size() == 0 && sharePartition.nextFetchOffset() == 5 && @@ -3616,6 +3649,7 @@ public void testAcquisitionLockOnOffsetWithWriteShareGroupStateFailure() throws Mockito.when(persister.writeState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(writeShareGroupStateResult)); // Allowing acquisition lock to expire. Even if write share group state RPC fails, state transition still happens. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> { Map expectedOffsetStateMap = new HashMap<>(); @@ -3652,7 +3686,8 @@ public void testReleaseSingleRecordBatch() { assertEquals(0, sharePartition.nextFetchOffset()); assertEquals(1, sharePartition.cachedState().size()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(0L).batchState()); - assertEquals(1, sharePartition.cachedState().get(0L).batchDeliveryCount()); + // Release delivery count. + assertEquals(0, sharePartition.cachedState().get(0L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(0L).offsetState()); } @@ -3669,7 +3704,7 @@ public void testReleaseMultipleRecordBatch() { assertEquals(5, sharePartition.nextFetchOffset()); assertEquals(1, sharePartition.cachedState().size()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(5L).batchState()); - assertEquals(1, sharePartition.cachedState().get(5L).batchDeliveryCount()); + assertEquals(0, sharePartition.cachedState().get(5L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(5L).offsetState()); } @@ -3731,7 +3766,7 @@ public void testReleaseAcknowledgedMultipleSubsetRecordBatch() { assertEquals(5, sharePartition.nextFetchOffset()); // Check cached state. Map expectedOffsetStateMap = new HashMap<>(); - expectedOffsetStateMap.put(5L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(5L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(6L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(5L).offsetState()); @@ -3745,8 +3780,8 @@ public void testReleaseAcknowledgedMultipleSubsetRecordBatch() { expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); } @@ -3792,8 +3827,8 @@ public void testReleaseAcquiredRecordsWithAnotherMember() { expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); // Release acquired records for "member-2". @@ -3815,8 +3850,8 @@ public void testReleaseAcquiredRecordsWithAnotherMember() { expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); } @@ -3862,8 +3897,8 @@ public void testReleaseAcquiredRecordsWithAnotherMemberAndSubsetAcknowledged() { expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); // Ack subset of records by "member-2". @@ -3879,7 +3914,7 @@ public void testReleaseAcquiredRecordsWithAnotherMemberAndSubsetAcknowledged() { // Check cached state. expectedOffsetStateMap.clear(); expectedOffsetStateMap.put(5L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(6L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(6L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(5L).offsetState()); expectedOffsetStateMap.clear(); expectedOffsetStateMap.put(10L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); @@ -3891,8 +3926,8 @@ public void testReleaseAcquiredRecordsWithAnotherMemberAndSubsetAcknowledged() { expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); } @@ -3926,14 +3961,14 @@ public void testReleaseAcquiredRecordsAfterDifferentAcknowledges() { Map expectedOffsetStateMap = new HashMap<>(); expectedOffsetStateMap.put(5L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(6L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(7L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(7L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(8L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(9L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(5L).offsetState()); } @Test - public void testMaxDeliveryCountLimitExceededForRecordsSubsetAfterReleaseAcquiredRecords() { + public void testMaxDeliveryCountLimitNotExceededForRecordsSubsetAfterReleaseAcquiredRecords() { SharePartition sharePartition = SharePartitionBuilder.builder() .withMaxDeliveryCount(2) .withState(SharePartitionState.ACTIVE) @@ -3955,12 +3990,12 @@ public void testMaxDeliveryCountLimitExceededForRecordsSubsetAfterReleaseAcquire assertEquals(0, sharePartition.nextFetchOffset()); assertEquals(2, sharePartition.cachedState().size()); - assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(10L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(10L).batchState()); assertNull(sharePartition.cachedState().get(10L).offsetState()); } @Test - public void testMaxDeliveryCountLimitExceededForRecordsSubsetAfterReleaseAcquiredRecordsSubset() { + public void testMaxDeliveryCountLimitNotExceededForRecordsSubsetAfterReleaseAcquiredRecordsSubset() { SharePartition sharePartition = SharePartitionBuilder.builder() .withMaxDeliveryCount(2) .withState(SharePartitionState.ACTIVE) @@ -3999,21 +4034,21 @@ public void testMaxDeliveryCountLimitExceededForRecordsSubsetAfterReleaseAcquire assertNotNull(sharePartition.cachedState().get(10L).offsetState()); assertThrows(IllegalStateException.class, () -> sharePartition.cachedState().get(15L).batchState()); assertNotNull(sharePartition.cachedState().get(10L).offsetState()); - assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(20L).batchState()); + assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(20L).batchState()); assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(20L).batchMemberId()); assertNull(sharePartition.cachedState().get(20L).offsetState()); Map expectedOffsetStateMap = new HashMap<>(); - expectedOffsetStateMap.put(10L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(11L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(12L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(13L, new InFlightState(RecordState.ARCHIVED, (short) 2, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(14L, new InFlightState(RecordState.ARCHIVED, (short) 2, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(10L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(11L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(12L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(13L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(14L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); expectedOffsetStateMap.clear(); - expectedOffsetStateMap.put(15L, new InFlightState(RecordState.ARCHIVED, (short) 2, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ARCHIVED, (short) 2, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(15L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(16L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(19L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); @@ -4050,9 +4085,10 @@ public void testMaxDeliveryCountLimitExceededForRecordsSubsetCacheCleared() { fetchAcquiredRecords(sharePartition, records2, 2); fetchAcquiredRecords(sharePartition, records3, 5); - CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); - assertNull(releaseResult.join()); - assertFalse(releaseResult.isCompletedExceptionally()); + sharePartition.acknowledge(MEMBER_ID, new ArrayList<>(List.of( + new ShareAcknowledgementBatch(13, 16, List.of((byte) 2)), + new ShareAcknowledgementBatch(20, 24, List.of((byte) 2)) + ))); assertEquals(25, sharePartition.nextFetchOffset()); assertEquals(0, sharePartition.cachedState().size()); @@ -4172,7 +4208,7 @@ public void testAcquisitionLockOnReleasingMultipleRecordBatch() { assertEquals(5, sharePartition.nextFetchOffset()); assertEquals(1, sharePartition.cachedState().size()); assertEquals(RecordState.AVAILABLE, sharePartition.cachedState().get(5L).batchState()); - assertEquals(1, sharePartition.cachedState().get(5L).batchDeliveryCount()); + assertEquals(0, sharePartition.cachedState().get(5L).batchDeliveryCount()); assertNull(sharePartition.cachedState().get(5L).offsetState()); // Acquisition lock timer task would be cancelled by the release acquired records operation. assertNull(sharePartition.cachedState().get(5L).batchAcquisitionLockTimeoutTask()); @@ -4182,7 +4218,6 @@ public void testAcquisitionLockOnReleasingMultipleRecordBatch() { @Test public void testAcquisitionLockOnReleasingAcknowledgedMultipleSubsetRecordBatchWithGapOffsets() { SharePartition sharePartition = SharePartitionBuilder.builder() - .withDefaultAcquisitionLockTimeoutMs(ACQUISITION_LOCK_TIMEOUT_MS) .withState(SharePartitionState.ACTIVE) .build(); MemoryRecords records1 = memoryRecords(2, 5); @@ -4215,7 +4250,7 @@ public void testAcquisitionLockOnReleasingAcknowledgedMultipleSubsetRecordBatchW assertEquals(5, sharePartition.nextFetchOffset()); // Check cached state. Map expectedOffsetStateMap = new HashMap<>(); - expectedOffsetStateMap.put(5L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(5L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(6L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(5L).offsetState()); @@ -4229,8 +4264,8 @@ public void testAcquisitionLockOnReleasingAcknowledgedMultipleSubsetRecordBatchW expectedOffsetStateMap.put(16L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(17L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(18L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(19L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(20L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); // Acquisition lock timer task would be cancelled by the release acquired records operation. @@ -4816,7 +4851,7 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovement() { expectedOffsetStateMap.put(21L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(22L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(23L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(24L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(24L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(20L).offsetState()); @@ -4830,8 +4865,8 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovement() { expectedOffsetStateMap.put(35L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(36L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); expectedOffsetStateMap.put(37L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(38L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(39L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(38L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(39L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(35L).offsetState()); } @@ -4887,14 +4922,60 @@ public void testReleaseAcquiredRecordsBatchesPostStartOffsetMovementToMiddleOfBa Map expectedOffsetStateMap = new HashMap<>(); expectedOffsetStateMap.put(10L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(11L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(12L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(13L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); - expectedOffsetStateMap.put(14L, new InFlightState(RecordState.AVAILABLE, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(11L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(12L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(13L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(14L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); } + @Test + public void testReleaseAcquiredRecordsDecreaseDeliveryCount() { + SharePartition sharePartition = SharePartitionBuilder.builder().withState(SharePartitionState.ACTIVE).build(); + + fetchAcquiredRecords(sharePartition, memoryRecords(5, 5), 5); + fetchAcquiredRecords(sharePartition, memoryRecords(5, 10), 5); + + sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(12, 13, List.of((byte) 1)))); + + // LSO is at 11. + sharePartition.updateCacheAndOffsets(11); + + assertEquals(15, sharePartition.nextFetchOffset()); + assertEquals(11, sharePartition.startOffset()); + assertEquals(14, sharePartition.endOffset()); + assertEquals(2, sharePartition.cachedState().size()); + + // Before release, the delivery count was incremented. + Map expectedOffsetStateMap = new HashMap<>(); + expectedOffsetStateMap.put(10L, new InFlightState(RecordState.ACQUIRED, (short) 1, MEMBER_ID)); + expectedOffsetStateMap.put(11L, new InFlightState(RecordState.ACQUIRED, (short) 1, MEMBER_ID)); + expectedOffsetStateMap.put(12L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(13L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(14L, new InFlightState(RecordState.ACQUIRED, (short) 1, MEMBER_ID)); + assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); + + // Release acquired records. + CompletableFuture releaseResult = sharePartition.releaseAcquiredRecords(MEMBER_ID); + assertNull(releaseResult.join()); + assertFalse(releaseResult.isCompletedExceptionally()); + + // Check delivery count. + assertEquals(EMPTY_MEMBER_ID, sharePartition.cachedState().get(5L).batchMemberId()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(5L).batchState()); + assertEquals(1, sharePartition.cachedState().get(5L).batchDeliveryCount()); + + // After release, the delivery count was decremented. + expectedOffsetStateMap = new HashMap<>(); + expectedOffsetStateMap.put(10L, new InFlightState(RecordState.ARCHIVED, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(11L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(12L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(13L, new InFlightState(RecordState.ACKNOWLEDGED, (short) 1, EMPTY_MEMBER_ID)); + expectedOffsetStateMap.put(14L, new InFlightState(RecordState.AVAILABLE, (short) 0, EMPTY_MEMBER_ID)); + assertEquals(expectedOffsetStateMap, sharePartition.cachedState().get(10L).offsetState()); + } + @Test public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovement() throws InterruptedException { SharePartition sharePartition = SharePartitionBuilder.builder() @@ -4929,6 +5010,7 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovement() throws assertEquals(7, sharePartition.cachedState().size()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> { Map expectedOffsetStateMap1 = new HashMap<>(); @@ -4987,6 +5069,7 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovementToStartOf assertEquals(2, sharePartition.cachedState().size()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.cachedState().get(5L).batchMemberId().equals(EMPTY_MEMBER_ID) && sharePartition.cachedState().get(5L).batchState() == RecordState.ARCHIVED && @@ -5015,6 +5098,7 @@ public void testAcquisitionLockTimeoutForBatchesPostStartOffsetMovementToMiddleO assertEquals(2, sharePartition.cachedState().size()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> { Map expectedOffsetStateMap = new HashMap<>(); @@ -5040,7 +5124,6 @@ public void testScheduleAcquisitionLockTimeoutValueFromGroupConfig() { Mockito.when(groupConfig.shareRecordLockDurationMs()).thenReturn(expectedDurationMs); SharePartition sharePartition = SharePartitionBuilder.builder() - .withDefaultAcquisitionLockTimeoutMs(ACQUISITION_LOCK_TIMEOUT_MS) .withGroupConfigManager(groupConfigManager).build(); SharePartition.AcquisitionLockTimerTask timerTask = sharePartition.scheduleAcquisitionLockTimeout(MEMBER_ID, 100L, 200L); @@ -5063,7 +5146,6 @@ public void testScheduleAcquisitionLockTimeoutValueUpdatesSuccessfully() { .thenReturn(expectedDurationMs2); SharePartition sharePartition = SharePartitionBuilder.builder() - .withDefaultAcquisitionLockTimeoutMs(ACQUISITION_LOCK_TIMEOUT_MS) .withGroupConfigManager(groupConfigManager).build(); SharePartition.AcquisitionLockTimerTask timerTask1 = sharePartition.scheduleAcquisitionLockTimeout(MEMBER_ID, 100L, 200L); @@ -5209,6 +5291,7 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledge() throws In assertNotNull(sharePartition.cachedState().get(2L).batchAcquisitionLockTimeoutTask()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.nextFetchOffset() == 7 && sharePartition.cachedState().isEmpty() && sharePartition.startOffset() == 7 && sharePartition.endOffset() == 7, @@ -5258,6 +5341,7 @@ public void testLsoMovementThenAcquisitionLockTimeoutThenAcknowledgeBatchLastOff assertNotNull(sharePartition.cachedState().get(1L).batchAcquisitionLockTimeoutTask()); // Allowing acquisition lock to expire. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); TestUtils.waitForCondition( () -> sharePartition.nextFetchOffset() == 3 && sharePartition.cachedState().isEmpty() && sharePartition.startOffset() == 3 && sharePartition.endOffset() == 3, @@ -6668,7 +6752,7 @@ public void testAcquireWhenBatchesRemovedForFetchOffsetForSameCachedBatch() { }); }); } - + private String assertionFailedMessage(SharePartition sharePartition, Map> offsets) { StringBuilder errorMessage = new StringBuilder(ACQUISITION_LOCK_NEVER_GOT_RELEASED + String.format( " timer size: %d, next fetch offset: %d\n", @@ -7068,6 +7152,26 @@ public void testFetchAbortedTransactionRecordBatchesForAbortedAndCommittedTransa assertEquals(1, actual.get(3).producerId()); } + @Test + public void testFetchLockReleasedByDifferentId() { + SharePartition sharePartition = SharePartitionBuilder.builder() + .withState(SharePartitionState.ACTIVE) + .build(); + Uuid fetchId1 = Uuid.randomUuid(); + Uuid fetchId2 = Uuid.randomUuid(); + + // Initially, fetch lock is not acquired. + assertNull(sharePartition.fetchLock()); + // fetchId1 acquires the fetch lock. + assertTrue(sharePartition.maybeAcquireFetchLock(fetchId1)); + // If we release fetch lock by fetchId2, it will work. Currently, we have kept the release of fetch lock as non-strict + // such that even if the caller's id for releasing fetch lock does not match the id that holds the lock, we will + // still release it. This has been done to avoid the scenarios where we hold the fetch lock for a share partition + // forever due to faulty code. In the future, we plan to make the locks handling strict, then this test case needs to be updated. + sharePartition.releaseFetchLock(fetchId2); + assertNull(sharePartition.fetchLock()); // Fetch lock has been released. + } + /** * This function produces transactional data of a given no. of records followed by a transactional marker (COMMIT/ABORT). */ diff --git a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala index dc836352787b2..54f6d71a27853 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala @@ -80,11 +80,13 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { val tp = new TopicPartition(topic, part) val logDir = "logDir" val group = "my-group" + val shareGroup = "share-group" val protocolType = "consumer" val protocolName = "consumer-range" val clusterResource = new ResourcePattern(CLUSTER, Resource.CLUSTER_NAME, LITERAL) val topicResource = new ResourcePattern(TOPIC, topic, LITERAL) val groupResource = new ResourcePattern(GROUP, group, LITERAL) + val shareGroupResource = new ResourcePattern(GROUP, shareGroup, LITERAL) val transactionalIdResource = new ResourcePattern(TRANSACTIONAL_ID, transactionalId, LITERAL) producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "1") diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index d75bdc9df6de4..c0325a4048b10 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -17,7 +17,7 @@ import java.time.Duration import java.util import java.util.concurrent.{ExecutionException, Semaphore} import java.util.regex.Pattern -import java.util.{Collections, Optional, Properties} +import java.util.{Collections, Comparator, Optional, Properties} import kafka.utils.{TestInfoUtils, TestUtils} import kafka.utils.TestUtils.waitUntilTrue import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ListGroupsOptions, NewTopic} @@ -37,18 +37,17 @@ import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProt import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.{OffsetForLeaderPartition, OffsetForLeaderTopic, OffsetForLeaderTopicCollection} -import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, OffsetFetchRequestData, ProduceRequestData, SyncGroupRequestData, WriteTxnMarkersRequestData} +import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ConsumerGroupDescribeRequestData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteShareGroupOffsetsRequestData, DeleteShareGroupStateRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeShareGroupOffsetsRequestData, DescribeTransactionsRequestData, FetchResponseData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, InitializeShareGroupStateRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, OffsetFetchRequestData, OffsetFetchResponseData, ProduceRequestData, ReadShareGroupStateRequestData, ReadShareGroupStateSummaryRequestData, ShareAcknowledgeRequestData, ShareFetchRequestData, ShareGroupDescribeRequestData, ShareGroupHeartbeatRequestData, SyncGroupRequestData, WriteShareGroupStateRequestData, WriteTxnMarkersRequestData} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord} -import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData import org.apache.kafka.common.requests._ import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.resource.ResourceType._ import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourcePatternFilter, ResourceType} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.Utils -import org.apache.kafka.common.{ElectionType, IsolationLevel, KafkaException, TopicPartition, Uuid, requests} +import org.apache.kafka.common.{ElectionType, IsolationLevel, KafkaException, TopicIdPartition, TopicPartition, Uuid, requests} import org.apache.kafka.test.{TestUtils => JTestUtils} import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST @@ -73,6 +72,11 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { val groupDeleteAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW))) val groupDescribeConfigsAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE_CONFIGS, ALLOW))) val groupAlterConfigsAcl = Map(groupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER_CONFIGS, ALLOW))) + val shareGroupReadAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW))) + val shareGroupDescribeAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW))) + val shareGroupDeleteAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW))) + val shareGroupDescribeConfigsAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE_CONFIGS, ALLOW))) + val shareGroupAlterConfigsAcl = Map(shareGroupResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER_CONFIGS, ALLOW))) val clusterAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CLUSTER_ACTION, ALLOW))) val clusterCreateAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CREATE, ALLOW))) val clusterAlterAcl = Map(clusterResource -> Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER, ALLOW))) @@ -118,7 +122,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }), ApiKeys.OFFSET_COMMIT -> ((resp: requests.OffsetCommitResponse) => Errors.forCode( resp.data.topics().get(0).partitions().get(0).errorCode)), - ApiKeys.OFFSET_FETCH -> ((resp: requests.OffsetFetchResponse) => resp.groupLevelError(group)), + ApiKeys.OFFSET_FETCH -> ((resp: requests.OffsetFetchResponse) => Errors.forCode(resp.group(group).errorCode())), ApiKeys.FIND_COORDINATOR -> ((resp: FindCoordinatorResponse) => { Errors.forCode(resp.data.coordinators.asScala.find(g => group == g.key).head.errorCode) }), @@ -199,7 +203,26 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }), ApiKeys.CONSUMER_GROUP_HEARTBEAT -> ((resp: ConsumerGroupHeartbeatResponse) => Errors.forCode(resp.data.errorCode)), ApiKeys.CONSUMER_GROUP_DESCRIBE -> ((resp: ConsumerGroupDescribeResponse) => - Errors.forCode(resp.data.groups.asScala.find(g => group == g.groupId).head.errorCode)) + Errors.forCode(resp.data.groups.asScala.find(g => group == g.groupId).head.errorCode)), + ApiKeys.SHARE_GROUP_HEARTBEAT -> ((resp: ShareGroupHeartbeatResponse) => Errors.forCode(resp.data.errorCode)), + ApiKeys.SHARE_GROUP_DESCRIBE -> ((resp: ShareGroupDescribeResponse) => + Errors.forCode(resp.data.groups.asScala.find(g => shareGroup == g.groupId).head.errorCode)), + ApiKeys.SHARE_FETCH -> ((resp: ShareFetchResponse) => Errors.forCode(resp.data.errorCode)), + ApiKeys.SHARE_ACKNOWLEDGE -> ((resp: ShareAcknowledgeResponse) => Errors.forCode(resp.data.errorCode)), + ApiKeys.INITIALIZE_SHARE_GROUP_STATE -> ((resp: InitializeShareGroupStateResponse) => Errors.forCode( + resp.data.results.get(0).partitions.get(0).errorCode)), + ApiKeys.READ_SHARE_GROUP_STATE -> ((resp: ReadShareGroupStateResponse) => Errors.forCode( + resp.data.results.get(0).partitions.get(0).errorCode)), + ApiKeys.WRITE_SHARE_GROUP_STATE -> ((resp: WriteShareGroupStateResponse) => Errors.forCode( + resp.data.results.get(0).partitions.get(0).errorCode)), + ApiKeys.DELETE_SHARE_GROUP_STATE -> ((resp: DeleteShareGroupStateResponse) => Errors.forCode( + resp.data.results.get(0).partitions.get(0).errorCode)), + ApiKeys.READ_SHARE_GROUP_STATE_SUMMARY -> ((resp: ReadShareGroupStateSummaryResponse) => Errors.forCode( + resp.data.results.get(0).partitions.get(0).errorCode)), + ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> ((resp: DescribeShareGroupOffsetsResponse) => Errors.forCode( + resp.data.groups.asScala.find(g => shareGroup == g.groupId).head.errorCode)), + ApiKeys.DELETE_SHARE_GROUP_OFFSETS -> ((resp: DeleteShareGroupOffsetsResponse) => Errors.forCode( + resp.data.errorCode)) ) def findErrorForTopicId(id: Uuid, response: AbstractResponse): Errors = { @@ -255,7 +278,18 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ApiKeys.DESCRIBE_PRODUCERS -> topicReadAcl, ApiKeys.DESCRIBE_TRANSACTIONS -> transactionalIdDescribeAcl, ApiKeys.CONSUMER_GROUP_HEARTBEAT -> groupReadAcl, - ApiKeys.CONSUMER_GROUP_DESCRIBE -> groupDescribeAcl + ApiKeys.CONSUMER_GROUP_DESCRIBE -> groupDescribeAcl, + ApiKeys.SHARE_GROUP_HEARTBEAT -> (shareGroupReadAcl ++ topicDescribeAcl), + ApiKeys.SHARE_GROUP_DESCRIBE -> (shareGroupDescribeAcl ++ topicDescribeAcl), + ApiKeys.SHARE_FETCH -> (shareGroupReadAcl ++ topicReadAcl), + ApiKeys.SHARE_ACKNOWLEDGE -> (shareGroupReadAcl ++ topicReadAcl), + ApiKeys.INITIALIZE_SHARE_GROUP_STATE -> clusterAcl, + ApiKeys.READ_SHARE_GROUP_STATE -> clusterAcl, + ApiKeys.WRITE_SHARE_GROUP_STATE -> clusterAcl, + ApiKeys.DELETE_SHARE_GROUP_STATE -> clusterAcl, + ApiKeys.READ_SHARE_GROUP_STATE_SUMMARY -> clusterAcl, + ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> (shareGroupDescribeAcl ++ topicDescribeAcl), + ApiKeys.DELETE_SHARE_GROUP_OFFSETS -> (shareGroupDeleteAcl ++ topicReadAcl) ) private def createMetadataRequest(allowAutoTopicCreation: Boolean) = { @@ -655,6 +689,120 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { .setGroupIds(List(group).asJava) .setIncludeAuthorizedOperations(false)).build() + private def shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( + new ShareGroupHeartbeatRequestData() + .setGroupId(shareGroup) + .setMemberEpoch(0) + .setSubscribedTopicNames(List(topic).asJava)).build(ApiKeys.SHARE_GROUP_HEARTBEAT.latestVersion) + + + private def shareGroupDescribeRequest = new ShareGroupDescribeRequest.Builder( + new ShareGroupDescribeRequestData() + .setGroupIds(List(shareGroup).asJava) + .setIncludeAuthorizedOperations(false)).build(ApiKeys.SHARE_GROUP_DESCRIBE.latestVersion) + + + private def createShareFetchRequest = { + val metadata: ShareRequestMetadata = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH) + val send: Seq[TopicIdPartition] = Seq( + new TopicIdPartition(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID), new TopicPartition(topic, part))) + val ackMap = new util.HashMap[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] + requests.ShareFetchRequest.Builder.forConsumer(shareGroup, metadata, 100, 0, Int.MaxValue, 500, 500, + send.asJava, Seq.empty.asJava, ackMap).build() + } + + private def shareAcknowledgeRequest = { + val shareAcknowledgeRequestData = new ShareAcknowledgeRequestData() + .setGroupId(shareGroup) + .setMemberId(Uuid.randomUuid().toString) + .setShareSessionEpoch(1) + .setTopics(List(new ShareAcknowledgeRequestData.AcknowledgeTopic() + .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) + .setPartitions(List( + new ShareAcknowledgeRequestData.AcknowledgePartition() + .setPartitionIndex(part) + .setAcknowledgementBatches(List( + new ShareAcknowledgeRequestData.AcknowledgementBatch() + .setFirstOffset(0) + .setLastOffset(1) + .setAcknowledgeTypes(Collections.singletonList(1.toByte)) + ).asJava) + ).asJava) + ).asJava) + + new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData).build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) + } + + private def initializeShareGroupStateRequest = new InitializeShareGroupStateRequest.Builder( + new InitializeShareGroupStateRequestData() + .setGroupId(shareGroup) + .setTopics(List(new InitializeShareGroupStateRequestData.InitializeStateData() + .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) + .setPartitions(List(new InitializeShareGroupStateRequestData.PartitionData() + .setPartition(part) + ).asJava) + ).asJava)).build() + + private def readShareGroupStateRequest = new ReadShareGroupStateRequest.Builder( + new ReadShareGroupStateRequestData() + .setGroupId(shareGroup) + .setTopics(List(new ReadShareGroupStateRequestData.ReadStateData() + .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) + .setPartitions(List(new ReadShareGroupStateRequestData.PartitionData() + .setPartition(part) + .setLeaderEpoch(0) + ).asJava) + ).asJava)).build() + + private def writeShareGroupStateRequest = new WriteShareGroupStateRequest.Builder( + new WriteShareGroupStateRequestData() + .setGroupId(shareGroup) + .setTopics(List(new WriteShareGroupStateRequestData.WriteStateData() + .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) + .setPartitions(List(new WriteShareGroupStateRequestData.PartitionData() + .setPartition(part) + ).asJava) + ).asJava)).build() + + private def deleteShareGroupStateRequest = new DeleteShareGroupStateRequest.Builder( + new DeleteShareGroupStateRequestData() + .setGroupId(shareGroup) + .setTopics(List(new DeleteShareGroupStateRequestData.DeleteStateData() + .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) + .setPartitions(List(new DeleteShareGroupStateRequestData.PartitionData() + .setPartition(part) + ).asJava) + ).asJava)).build() + + private def readShareGroupStateSummaryRequest = new ReadShareGroupStateSummaryRequest.Builder( + new ReadShareGroupStateSummaryRequestData() + .setGroupId(shareGroup) + .setTopics(List(new ReadShareGroupStateSummaryRequestData.ReadStateSummaryData() + .setTopicId(getTopicIds().getOrElse(tp.topic, Uuid.ZERO_UUID)) + .setPartitions(List(new ReadShareGroupStateSummaryRequestData.PartitionData() + .setPartition(part) + .setLeaderEpoch(0) + ).asJava) + ).asJava)).build(ApiKeys.READ_SHARE_GROUP_STATE_SUMMARY.latestVersion) + + private def describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequest.Builder( + new DescribeShareGroupOffsetsRequestData() + .setGroups(List(new DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestGroup() + .setGroupId(shareGroup) + .setTopics(List(new DescribeShareGroupOffsetsRequestData.DescribeShareGroupOffsetsRequestTopic() + .setTopicName(topic) + .setPartitions(List(Integer.valueOf(part) + ).asJava) + ).asJava) + ).asJava)).build(ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS.latestVersion) + + private def deleteShareGroupOffsetsRequest = new DeleteShareGroupOffsetsRequest.Builder( + new DeleteShareGroupOffsetsRequestData() + .setGroupId(shareGroup) + .setTopics(List(new DeleteShareGroupOffsetsRequestData.DeleteShareGroupOffsetsRequestTopic() + .setTopicName(topic) + ).asJava)).build(ApiKeys.DELETE_SHARE_GROUP_OFFSETS.latestVersion) + private def sendRequests(requestKeyToRequest: mutable.Map[ApiKeys, AbstractRequest], topicExists: Boolean = true, topicNames: Map[Uuid, String] = getTopicNames()) = { for ((key, request) <- requestKeyToRequest) { @@ -669,6 +817,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { // In KRaft mode, trying to delete a topic that doesn't exist but that you do have // describe permission for will give UNKNOWN_TOPIC_OR_PARTITION. true + } else if (resourceToAcls.size > 1) { + false } else { describeAcls == acls } @@ -684,7 +834,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } @ParameterizedTest - @ValueSource(strings = Array("kraft")) + @ValueSource(strings = Array("kip932")) def testAuthorizationWithTopicExisting(quorum: String): Unit = { //First create the topic so we have a valid topic ID sendRequests(mutable.Map(ApiKeys.CREATE_TOPICS -> createTopicsRequest)) @@ -723,6 +873,18 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ApiKeys.WRITE_TXN_MARKERS -> writeTxnMarkersRequest, ApiKeys.CONSUMER_GROUP_HEARTBEAT -> consumerGroupHeartbeatRequest, ApiKeys.CONSUMER_GROUP_DESCRIBE -> consumerGroupDescribeRequest, + ApiKeys.SHARE_GROUP_HEARTBEAT -> shareGroupHeartbeatRequest, + ApiKeys.SHARE_GROUP_DESCRIBE -> shareGroupDescribeRequest, + ApiKeys.SHARE_FETCH -> createShareFetchRequest, + ApiKeys.SHARE_ACKNOWLEDGE -> shareAcknowledgeRequest, + ApiKeys.INITIALIZE_SHARE_GROUP_STATE -> initializeShareGroupStateRequest, + ApiKeys.READ_SHARE_GROUP_STATE -> readShareGroupStateRequest, + ApiKeys.WRITE_SHARE_GROUP_STATE -> writeShareGroupStateRequest, + ApiKeys.DELETE_SHARE_GROUP_STATE -> deleteShareGroupStateRequest, + ApiKeys.READ_SHARE_GROUP_STATE_SUMMARY -> readShareGroupStateSummaryRequest, + ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> describeShareGroupOffsetsRequest, + ApiKeys.DELETE_SHARE_GROUP_OFFSETS -> deleteShareGroupOffsetsRequest, + // Delete the topic last ApiKeys.DELETE_TOPICS -> deleteTopicsRequest ) @@ -752,7 +914,10 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ApiKeys.CREATE_PARTITIONS -> createPartitionsRequest, ApiKeys.DELETE_GROUPS -> deleteGroupsRequest, ApiKeys.OFFSET_FOR_LEADER_EPOCH -> offsetsForLeaderEpochRequest, - ApiKeys.ELECT_LEADERS -> electLeadersRequest + ApiKeys.ELECT_LEADERS -> electLeadersRequest, + ApiKeys.SHARE_FETCH -> createShareFetchRequest, + ApiKeys.SHARE_ACKNOWLEDGE -> shareAcknowledgeRequest, + ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS -> describeShareGroupOffsetsRequest ) sendRequests(requestKeyToRequest, false, topicNames) @@ -1443,15 +1608,20 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { // without describe permission on the topic, we shouldn't be able to fetch offsets val offsetFetchRequest = createOffsetFetchRequestAllPartitions var offsetFetchResponse = connectAndReceive[OffsetFetchResponse](offsetFetchRequest) - assertEquals(Errors.NONE, offsetFetchResponse.groupLevelError(group)) - assertTrue(offsetFetchResponse.partitionDataMap(group).isEmpty) + assertEquals(Errors.NONE, Errors.forCode(offsetFetchResponse.group(group).errorCode())) + assertTrue(offsetFetchResponse.group(group).topics.isEmpty) // now add describe permission on the topic and verify that the offset can be fetched addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) offsetFetchResponse = connectAndReceive[OffsetFetchResponse](offsetFetchRequest) - assertEquals(Errors.NONE, offsetFetchResponse.groupLevelError(group)) - assertTrue(offsetFetchResponse.partitionDataMap(group).containsKey(tp)) - assertEquals(offset, offsetFetchResponse.partitionDataMap(group).get(tp).offset) + assertEquals(Errors.NONE, Errors.forCode(offsetFetchResponse.group(group).errorCode())) + assertEquals( + offset, + offsetFetchResponse.group(group).topics.asScala + .find(_.name == tp.topic) + .flatMap(_.partitions.asScala.find(_.partitionIndex == tp.partition).map(_.committedOffset)) + .getOrElse(-1L) + ) } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames) @@ -1486,21 +1656,33 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { createTopicWithBrokerPrincipal(topics(0)) createTopicWithBrokerPrincipal(topics(1), numPartitions = 2) createTopicWithBrokerPrincipal(topics(2), numPartitions = 3) - groupResources.foreach(r => { + groupResources.foreach { r => addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), r) - }) - topicResources.foreach(t => { + } + topicResources.foreach { t => addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), t) - }) + } val offset = 15L val leaderEpoch: Optional[Integer] = Optional.of(1) val metadata = "metadata" + def assertResponse( + expected: OffsetFetchResponseData.OffsetFetchResponseGroup, + actual: OffsetFetchResponseData.OffsetFetchResponseGroup + ): Unit = { + actual.topics.sort((t1, t2) => t1.name.compareTo(t2.name)) + actual.topics.asScala.foreach { topic => + topic.partitions.sort(Comparator.comparingInt[OffsetFetchResponseData.OffsetFetchResponsePartitions](_.partitionIndex)) + } + + assertEquals(expected, actual) + } + def commitOffsets(tpList: util.List[TopicPartition]): Unit = { val consumer = createConsumer() consumer.assign(tpList) - val offsets = tpList.asScala.map{ + val offsets = tpList.asScala.map { tp => (tp, new OffsetAndMetadata(offset, leaderEpoch, metadata)) }.toMap.asJava consumer.commitSync(offsets) @@ -1516,98 +1698,298 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { removeAllClientAcls() - def verifyPartitionData(partitionData: OffsetFetchResponse.PartitionData): Unit = { - assertTrue(!partitionData.hasError) - assertEquals(offset, partitionData.offset) - assertEquals(metadata, partitionData.metadata) - assertEquals(leaderEpoch.get(), partitionData.leaderEpoch.get()) - } - - def verifyResponse(groupLevelResponse: Errors, - partitionData: util.Map[TopicPartition, PartitionData], - topicList: util.List[TopicPartition]): Unit = { - assertEquals(Errors.NONE, groupLevelResponse) - assertTrue(partitionData.size() == topicList.size()) - topicList.forEach(t => verifyPartitionData(partitionData.get(t))) - } - // test handling partial errors, where one group is fully authorized, some groups don't have // the right topic authorizations, and some groups have no authorization addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResources(0)) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResources(1)) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResources(3)) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResources(0)) + val offsetFetchRequest = createOffsetFetchRequest(groupToPartitionMap) var offsetFetchResponse = connectAndReceive[OffsetFetchResponse](offsetFetchRequest) - offsetFetchResponse.data().groups().forEach(g => - g.groupId() match { + + offsetFetchResponse.data.groups.forEach { g => + g.groupId match { case "group1" => - verifyResponse(offsetFetchResponse.groupLevelError(groups(0)), offsetFetchResponse - .partitionDataMap(groups(0)), topic1List) + assertResponse( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(g.groupId) + .setTopics(List( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(0)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava) + ).asJava), + offsetFetchResponse.group(g.groupId) + ) + case "group2" => - assertEquals(Errors.NONE, offsetFetchResponse.groupLevelError(groups(1))) - val group2Response = offsetFetchResponse.partitionDataMap(groups(1)) - assertTrue(group2Response.size() == 3) - assertTrue(group2Response.keySet().containsAll(topic1And2List)) - verifyPartitionData(group2Response.get(topic1And2List.get(0))) - assertTrue(group2Response.get(topic1And2List.get(1)).hasError) - assertTrue(group2Response.get(topic1And2List.get(2)).hasError) - assertEquals(OffsetFetchResponse.UNAUTHORIZED_PARTITION, group2Response.get(topic1And2List.get(1))) - assertEquals(OffsetFetchResponse.UNAUTHORIZED_PARTITION, group2Response.get(topic1And2List.get(2))) + assertResponse( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(g.groupId) + .setTopics(List( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(0)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava), + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(1)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code) + .setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(1) + .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code) + .setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + ).asJava) + ).asJava), + offsetFetchResponse.group(g.groupId) + ) + case "group3" => - assertEquals(Errors.GROUP_AUTHORIZATION_FAILED, offsetFetchResponse.groupLevelError(groups(2))) - assertTrue(offsetFetchResponse.partitionDataMap(groups(2)).size() == 0) + assertResponse( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(g.groupId) + .setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code), + offsetFetchResponse.group(g.groupId) + ) + case "group4" => - verifyResponse(offsetFetchResponse.groupLevelError(groups(3)), offsetFetchResponse - .partitionDataMap(groups(3)), topic1List) + assertResponse( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(g.groupId) + .setTopics(List( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(0)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava) + ).asJava), + offsetFetchResponse.group(g.groupId) + ) + case "group5" => - assertEquals(Errors.GROUP_AUTHORIZATION_FAILED, offsetFetchResponse.groupLevelError(groups(4))) - assertTrue(offsetFetchResponse.partitionDataMap(groups(4)).size() == 0) - }) + assertResponse( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(g.groupId) + .setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code), + offsetFetchResponse.group(g.groupId) + ) + } + } // test that after adding some of the ACLs, we get no group level authorization errors, but // still get topic level authorization errors for topics we don't have ACLs for addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResources(2)) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResources(4)) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResources(1)) + offsetFetchResponse = connectAndReceive[OffsetFetchResponse](offsetFetchRequest) - offsetFetchResponse.data().groups().forEach(g => - g.groupId() match { + + offsetFetchResponse.data.groups.forEach { g => + g.groupId match { case "group1" => - verifyResponse(offsetFetchResponse.groupLevelError(groups(0)), offsetFetchResponse - .partitionDataMap(groups(0)), topic1List) + assertResponse( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(g.groupId) + .setTopics(List( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(0)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava) + ).asJava), + offsetFetchResponse.group(g.groupId) + ) + case "group2" => - verifyResponse(offsetFetchResponse.groupLevelError(groups(1)), offsetFetchResponse - .partitionDataMap(groups(1)), topic1And2List) + assertResponse( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(g.groupId) + .setTopics(List( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(0)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava), + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(1)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(1) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava) + ).asJava), + offsetFetchResponse.group(g.groupId) + ) + case "group3" => - assertEquals(Errors.NONE, offsetFetchResponse.groupLevelError(groups(2))) - val group3Response = offsetFetchResponse.partitionDataMap(groups(2)) - assertTrue(group3Response.size() == 6) - assertTrue(group3Response.keySet().containsAll(allTopicsList)) - verifyPartitionData(group3Response.get(allTopicsList.get(0))) - verifyPartitionData(group3Response.get(allTopicsList.get(1))) - verifyPartitionData(group3Response.get(allTopicsList.get(2))) - assertTrue(group3Response.get(allTopicsList.get(3)).hasError) - assertTrue(group3Response.get(allTopicsList.get(4)).hasError) - assertTrue(group3Response.get(allTopicsList.get(5)).hasError) - assertEquals(OffsetFetchResponse.UNAUTHORIZED_PARTITION, group3Response.get(allTopicsList.get(3))) - assertEquals(OffsetFetchResponse.UNAUTHORIZED_PARTITION, group3Response.get(allTopicsList.get(4))) - assertEquals(OffsetFetchResponse.UNAUTHORIZED_PARTITION, group3Response.get(allTopicsList.get(5))) + assertResponse( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(g.groupId) + .setTopics(List( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(0)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava), + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(1)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(1) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava), + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(2)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code) + .setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(1) + .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code) + .setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(2) + .setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code) + .setCommittedOffset(OffsetFetchResponse.INVALID_OFFSET) + .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) + .setMetadata(OffsetFetchResponse.NO_METADATA) + ).asJava) + ).asJava), + offsetFetchResponse.group(g.groupId) + ) + case "group4" => - verifyResponse(offsetFetchResponse.groupLevelError(groups(3)), offsetFetchResponse - .partitionDataMap(groups(3)), topic1And2List) + assertResponse( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(g.groupId) + .setTopics(List( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(0)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava), + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(1)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(1) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava) + ).asJava), + offsetFetchResponse.group(g.groupId) + ) + case "group5" => - verifyResponse(offsetFetchResponse.groupLevelError(groups(4)), offsetFetchResponse - .partitionDataMap(groups(4)), topic1And2List) - }) + assertResponse( + new OffsetFetchResponseData.OffsetFetchResponseGroup() + .setGroupId(g.groupId) + .setTopics(List( + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(0)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava), + new OffsetFetchResponseData.OffsetFetchResponseTopics() + .setName(topics(1)) + .setPartitions(List( + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(0) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata), + new OffsetFetchResponseData.OffsetFetchResponsePartitions() + .setPartitionIndex(1) + .setCommittedOffset(offset) + .setCommittedLeaderEpoch(leaderEpoch.get) + .setMetadata(metadata) + ).asJava) + ).asJava), + offsetFetchResponse.group(g.groupId) + ) + } + } // test that after adding all necessary ACLs, we get no partition level or group level errors // from the offsetFetch response addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResources(2)) offsetFetchResponse = connectAndReceive[OffsetFetchResponse](offsetFetchRequest) - offsetFetchResponse.data.groups.asScala.map(_.groupId).foreach( groupId => - verifyResponse(offsetFetchResponse.groupLevelError(groupId), offsetFetchResponse.partitionDataMap(groupId), partitionMap(groupId)) - ) + offsetFetchResponse.data.groups.forEach { group => + assertEquals(Errors.NONE.code, group.errorCode) + group.topics.forEach { topic => + topic.partitions.forEach { partition => + assertEquals(Errors.NONE.code, partition.errorCode) + } + } + } } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames) @@ -2653,6 +3035,476 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendAndReceiveRegexHeartbeat(member1Response, interBrokerListenerName, Some(0), fullRequest = true) } + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareGroupHeartbeatWithGroupReadAndTopicDescribeAcl(quorum: String): Unit = { + addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) + addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) + + val request = shareGroupHeartbeatRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareGroupHeartbeatWithOperationAll(quorum: String): Unit = { + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) + addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) + + val request = shareGroupHeartbeatRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareGroupHeartbeatWithoutGroupReadOrTopicDescribeAcl(quorum: String): Unit = { + removeAllClientAcls() + + val request = shareGroupHeartbeatRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareGroupHeartbeatWithoutGroupReadAcl(quorum: String): Unit = { + addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) + + val request = shareGroupHeartbeatRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareGroupHeartbeatWithoutTopicDescribeAcl(quorum: String): Unit = { + addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) + + val request = shareGroupHeartbeatRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + private def createShareGroupToDescribe(): Unit = { + createTopicWithBrokerPrincipal(topic) + addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), shareGroupResource) + addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) + shareConsumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, shareGroup) + val consumer = createShareConsumer() + consumer.subscribe(Collections.singleton(topic)) + consumer.poll(Duration.ofMillis(500L)) + removeAllClientAcls() + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareGroupDescribeWithGroupDescribeAndTopicDescribeAcl(quorum: String): Unit = { + createShareGroupToDescribe() + addAndVerifyAcls(shareGroupDescribeAcl(shareGroupResource), shareGroupResource) + addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) + + val request = shareGroupDescribeRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareGroupDescribeWithOperationAll(quorum: String): Unit = { + createShareGroupToDescribe() + + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) + addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) + + val request = shareGroupDescribeRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareGroupDescribeWithoutGroupDescribeAcl(quorum: String): Unit = { + createShareGroupToDescribe() + addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) + + val request = shareGroupDescribeRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareGroupDescribeWithoutGroupDescribeOrTopicDescribeAcl(quorum: String): Unit = { + createShareGroupToDescribe() + + val request = shareGroupDescribeRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareFetchWithGroupReadAndTopicReadAcl(quorum: String): Unit = { + addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) + addAndVerifyAcls(topicReadAcl(topicResource), topicResource) + + val request = createShareFetchRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareFetchWithOperationAll(quorum: String): Unit = { + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) + addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) + + val request = createShareFetchRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareFetchWithoutGroupReadOrTopicReadAcl(quorum: String): Unit = { + removeAllClientAcls() + + val request = createShareFetchRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareFetchWithoutGroupReadAcl(quorum: String): Unit = { + addAndVerifyAcls(topicReadAcl(topicResource), topicResource) + + val request = createShareFetchRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareFetchWithoutTopicReadAcl(quorum: String): Unit = { + createTopicWithBrokerPrincipal(topic) + addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) + + val request = createShareFetchRequest + val response = connectAndReceive[ShareFetchResponse](request, listenerName = listenerName) + assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED, Errors.forCode(response.data.responses.get(0).partitions.get(0).errorCode)) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareAcknowledgeWithGroupReadAndTopicReadAcl(quorum: String): Unit = { + addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) + addAndVerifyAcls(topicReadAcl(topicResource), topicResource) + + val request = shareAcknowledgeRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareAcknowledgeWithOperationAll(quorum: String): Unit = { + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) + addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) + + val request = shareAcknowledgeRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareAcknowledgeWithoutGroupReadOrTopicReadAcl(quorum: String): Unit = { + removeAllClientAcls() + + val request = shareAcknowledgeRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testShareAcknowledgeFetchWithoutGroupReadAcl(quorum: String): Unit = { + addAndVerifyAcls(topicReadAcl(topicResource), topicResource) + + val request = shareAcknowledgeRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testInitializeShareGroupStateWithClusterAcl(quorum: String): Unit = { + addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) + + val request = initializeShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testInitializeShareGroupStateWithOperationAll(quorum: String): Unit = { + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) + + val request = initializeShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testInitializeShareGroupStateWithoutClusterAcl(quorum: String): Unit = { + removeAllClientAcls() + + val request = initializeShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testReadShareGroupStateWithClusterAcl(quorum: String): Unit = { + addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) + + val request = readShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testReadShareGroupStateWithOperationAll(quorum: String): Unit = { + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) + + val request = readShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testReadShareGroupStateWithoutClusterAcl(quorum: String): Unit = { + removeAllClientAcls() + + val request = readShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testWriteShareGroupStateWithClusterAcl(quorum: String): Unit = { + addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) + + val request = writeShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testWriteShareGroupStateWithOperationAll(quorum: String): Unit = { + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) + + val request = writeShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testWriteShareGroupStateWithoutClusterAcl(quorum: String): Unit = { + removeAllClientAcls() + + val request = writeShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDeleteShareGroupStateWithClusterAcl(quorum: String): Unit = { + addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) + + val request = deleteShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDeleteShareGroupStateWithOperationAll(quorum: String): Unit = { + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) + + val request = deleteShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDeleteShareGroupStateWithoutClusterAcl(quorum: String): Unit = { + removeAllClientAcls() + + val request = deleteShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testReadShareGroupStateSummaryWithClusterAcl(quorum: String): Unit = { + addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) + + val request = readShareGroupStateSummaryRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testReadShareGroupStateSummaryWithOperationAll(quorum: String): Unit = { + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) + + val request = readShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testReadShareGroupStateSummaryWithoutClusterAcl(quorum: String): Unit = { + removeAllClientAcls() + + val request = readShareGroupStateRequest + val resource = Set[ResourceType](CLUSTER) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDescribeShareGroupOffsetsWithGroupDescribeAndTopicDescribeAcl(quorum: String): Unit = { + addAndVerifyAcls(shareGroupDescribeAcl(shareGroupResource), shareGroupResource) + addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) + + val request = describeShareGroupOffsetsRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDescribeShareGroupOffsetsWithOperationAll(quorum: String): Unit = { + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) + addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) + + val request = describeShareGroupOffsetsRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDescribeShareGroupOffsetsWithoutGroupDescribeOrTopicDescribeAcl(quorum: String): Unit = { + removeAllClientAcls() + + val request = describeShareGroupOffsetsRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDescribeShareGroupOffsetsWithoutGroupDescribeAcl(quorum: String): Unit = { + addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) + + val request = describeShareGroupOffsetsRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDescribeShareGroupOffsetsWithoutTopicDescribeAcl(quorum: String): Unit = { + addAndVerifyAcls(shareGroupDescribeAcl(shareGroupResource), shareGroupResource) + + val request = describeShareGroupOffsetsRequest + val response = connectAndReceive[DescribeShareGroupOffsetsResponse](request, listenerName = listenerName) + assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED, Errors.forCode(response.data.groups.get(0).topics.get(0).partitions.get(0).errorCode)) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDeleteShareGroupOffsetsWithGroupDeleteAndTopicReadAcl(quorum: String): Unit = { + addAndVerifyAcls(shareGroupDeleteAcl(shareGroupResource), shareGroupResource) + addAndVerifyAcls(topicReadAcl(topicResource), topicResource) + + val request = deleteShareGroupOffsetsRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDeleteShareGroupOffsetsWithOperationAll(quorum: String): Unit = { + val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) + addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) + addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) + + val request = deleteShareGroupOffsetsRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDeleteShareGroupOffsetsWithoutGroupDeleteOrTopicReadAcl(quorum: String): Unit = { + removeAllClientAcls() + + val request = deleteShareGroupOffsetsRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDeleteShareGroupOffsetsWithoutGroupDeleteAcl(quorum: String): Unit = { + addAndVerifyAcls(topicReadAcl(topicResource), topicResource) + + val request = deleteShareGroupOffsetsRequest + val resource = Set[ResourceType](GROUP, TOPIC) + sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) + } + + @ParameterizedTest + @ValueSource(strings = Array("kip932")) + def testDeleteShareGroupOffsetsWithoutTopicReadAcl(quorum: String): Unit = { + addAndVerifyAcls(shareGroupDeleteAcl(shareGroupResource), shareGroupResource) + + val request = deleteShareGroupOffsetsRequest + val response = connectAndReceive[DeleteShareGroupOffsetsResponse](request, listenerName = listenerName) + assertEquals(1, response.data.responses.size) + assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, response.data.responses.get(0).errorCode, s"Unexpected response $response") + } + private def sendAndReceiveFirstRegexHeartbeat(memberId: String, listenerName: ListenerName): ConsumerGroupHeartbeatResponseData = { val request = new ConsumerGroupHeartbeatRequest.Builder( diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index e35db63e81270..e062dcc09fa08 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -30,7 +30,6 @@ import kafka.security.JaasTestUtils import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} import org.apache.kafka.common.network.{ConnectionMode, ListenerName} import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, Serializer} -import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.MetadataLogConfig import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs} @@ -71,9 +70,6 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties, logDirCount = logDirCount) configureListeners(cfgs) modifyConfigs(cfgs) - if (isShareGroupTest()) { - cfgs.foreach(_.setProperty(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer,share")) - } cfgs.foreach(_.setProperty(MetadataLogConfig.METADATA_LOG_DIR_CONFIG, TestUtils.tempDir().getAbsolutePath)) insertControllerListenersIfNeeded(cfgs) cfgs.map(KafkaConfig.fromProps) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index f0353009b6d1b..0ab2328c53f79 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -1568,7 +1568,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @MethodSource(Array("getTestGroupProtocolParametersAll")) def testDeleteRecordsAfterCorruptRecords(groupProtocol: String): Unit = { val config = new Properties() - config.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "200") + config.put(TopicConfig.SEGMENT_BYTES_CONFIG, "200") createTopic(topic, numPartitions = 1, replicationFactor = 1, config) client = createAdminClient diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index f1c708acfdd3b..7197212d893dc 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -14,6 +14,7 @@ package kafka.api import kafka.api.BaseConsumerTest.{DeserializerImpl, SerializerImpl} +import java.lang.{Long => JLong} import java.time.Duration import java.util import java.util.Arrays.asList @@ -873,4 +874,22 @@ class PlaintextConsumerTest extends BaseConsumerTest { waitTimeMs=leaveGroupTimeoutMs ) } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedGroupProtocolNames) + @MethodSource(Array("getTestGroupProtocolParametersAll")) + def testOffsetRelatedWhenTimeoutZero(groupProtocol: String): Unit = { + val consumer = createConsumer() + val result1 = consumer.beginningOffsets(util.List.of(tp), Duration.ZERO) + assertNotNull(result1) + assertEquals(0, result1.size()) + + val result2 = consumer.endOffsets(util.List.of(tp), Duration.ZERO) + assertNotNull(result2) + assertEquals(0, result2.size()) + + val result3 = consumer.offsetsForTimes(Map[TopicPartition, JLong]((tp, 0)).asJava, Duration.ZERO) + assertNotNull(result3) + assertEquals(1, result3.size()) + assertNull(result3.get(tp)) + } } diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 4ef32c2f2ebea..66fca0db9badc 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -567,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 -> "3000000").asJava + val configsOverride = Map(TopicConfig.SEGMENT_BYTES_CONFIG -> "100000").asJava 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)) @@ -581,7 +581,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu val topicConfigs = result.config(topic1).get().entries.asScala assertTrue(topicConfigs.nonEmpty) val segmentBytesConfig = topicConfigs.find(_.name == TopicConfig.SEGMENT_BYTES_CONFIG).get - assertEquals(3000000, segmentBytesConfig.value.toLong) + assertEquals(100000, segmentBytesConfig.value.toLong) assertEquals(ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, segmentBytesConfig.source) val compressionConfig = topicConfigs.find(_.name == TopicConfig.COMPRESSION_TYPE_CONFIG).get assertEquals(ServerLogConfigs.COMPRESSION_TYPE_DEFAULT, compressionConfig.value) diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 9d1f699f238ac..3c5cd9396bdba 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -653,7 +653,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) => diff --git a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala b/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala deleted file mode 100644 index 490ebc48c1648..0000000000000 --- a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * 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 - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server - -import org.apache.kafka.common.test.api.{ClusterTest, ClusterTests, Type} -import kafka.utils.TestUtils -import org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType -import org.apache.kafka.clients.admin.{FeatureUpdate, UpdateFeaturesOptions} -import org.apache.kafka.common.test.ClusterInstance -import org.apache.kafka.server.common.MetadataVersion -import org.junit.jupiter.api.Assertions.assertEquals - -import scala.jdk.CollectionConverters._ - -class MetadataVersionIntegrationTest { - @ClusterTests(value = Array( - new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_3_IV3), - new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_4_IV0), - new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_5_IV0), - new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_6_IV0), - new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_6_IV1) - )) - def testBasicMetadataVersionUpgrade(clusterInstance: ClusterInstance): Unit = { - val admin = clusterInstance.admin() - try { - val describeResult = admin.describeFeatures() - val ff = describeResult.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME) - assertEquals(ff.minVersionLevel(), clusterInstance.config().metadataVersion().featureLevel()) - assertEquals(ff.maxVersionLevel(), clusterInstance.config().metadataVersion().featureLevel()) - - // Update to new version - val updateVersion = MetadataVersion.IBP_3_7_IV1.featureLevel.shortValue - val updateResult = admin.updateFeatures( - Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions()) - updateResult.all().get() - - // Verify that new version is visible on broker - TestUtils.waitUntilTrue(() => { - val describeResult2 = admin.describeFeatures() - val ff2 = describeResult2.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME) - ff2.minVersionLevel() == updateVersion && ff2.maxVersionLevel() == updateVersion - }, "Never saw metadata.version increase on broker") - } finally { - admin.close() - } - } - - @ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_9_IV0) - def testUpgradeSameVersion(clusterInstance: ClusterInstance): Unit = { - val admin = clusterInstance.admin() - try { - val updateVersion = MetadataVersion.IBP_3_9_IV0.featureLevel.shortValue - val updateResult = admin.updateFeatures( - Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions()) - updateResult.all().get() - } finally { - admin.close() - } - } - - @ClusterTest(types = Array(Type.KRAFT)) - def testDefaultIsLatestVersion(clusterInstance: ClusterInstance): Unit = { - val admin = clusterInstance.admin() - try { - val describeResult = admin.describeFeatures() - val ff = describeResult.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME) - assertEquals(ff.minVersionLevel(), MetadataVersion.latestTesting().featureLevel(), - "If this test fails, check the default MetadataVersion in the @ClusterTest annotation") - assertEquals(ff.maxVersionLevel(), MetadataVersion.latestTesting().featureLevel()) - } finally { - admin.close() - } - } -} diff --git a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala index 2288d37aaadc1..14d679f25d3a1 100644 --- a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala +++ b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala @@ -20,6 +20,7 @@ package kafka.server import kafka.utils.TestUtils import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import org.apache.kafka.common.utils.BufferSupplier +import org.apache.kafka.common.utils.LogContext import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.raft.MetadataLogConfig import org.apache.kafka.snapshot.RecordsSnapshotReader @@ -79,7 +80,8 @@ class RaftClusterSnapshotTest { new MetadataRecordSerde(), BufferSupplier.create(), 1, - true + true, + new LogContext() ) ) { snapshot => // Check that the snapshot is non-empty diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 83914f63a4c98..3b7aa0decef46 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.record.ArbitraryMemoryRecords import org.apache.kafka.common.record.InvalidMemoryRecordsProvider import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} import org.apache.kafka.common.utils.Utils -import org.apache.kafka.raft.{KafkaRaftClient, LogAppendInfo, LogOffsetMetadata, MetadataLogConfig, OffsetAndEpoch, QuorumConfig, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} +import org.apache.kafka.raft.{KafkaRaftClient, LogAppendInfo, LogOffsetMetadata, MetadataLogConfig, QuorumConfig, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} import org.apache.kafka.raft.internals.BatchBuilder import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} @@ -43,7 +43,7 @@ import org.junit.jupiter.params.provider.ArgumentsSource import net.jqwik.api.AfterFailureMode import net.jqwik.api.ForAll import net.jqwik.api.Property -import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.server.common.OffsetAndEpoch import java.io.File import java.nio.ByteBuffer @@ -78,13 +78,13 @@ final class KafkaMetadataLogTest { props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240)) props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024)) - assertThrows(classOf[ConfigException], () => { + assertThrows(classOf[InvalidConfigurationException], () => { val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) }) - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10 * 1024 * 1024)) + props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) @@ -689,6 +689,7 @@ final class KafkaMetadataLogTest { val recordSize = 64 val config = new MetadataLogConfig( DefaultMetadataLogConfig.logSegmentBytes, + DefaultMetadataLogConfig.logSegmentMinBytes, DefaultMetadataLogConfig.logSegmentMillis, DefaultMetadataLogConfig.retentionMaxBytes, DefaultMetadataLogConfig.retentionMillis, @@ -907,6 +908,7 @@ final class KafkaMetadataLogTest { @Test def testAdvanceLogStartOffsetAfterCleaning(): Unit = { val config = new MetadataLogConfig( + 512, 512, 10 * 1000, 256, @@ -943,6 +945,7 @@ final class KafkaMetadataLogTest { def testDeleteSnapshots(): Unit = { // Generate some logs and a few snapshots, set retention low and verify that cleaning occurs val config = new MetadataLogConfig( + 1024, 1024, 10 * 1000, 1024, @@ -976,6 +979,7 @@ final class KafkaMetadataLogTest { def testSoftRetentionLimit(): Unit = { // Set retention equal to the segment size and generate slightly more than one segment of logs val config = new MetadataLogConfig( + 10240, 10240, 10 * 1000, 10240, @@ -1019,6 +1023,7 @@ final class KafkaMetadataLogTest { @Test def testSegmentsLessThanLatestSnapshot(): Unit = { val config = new MetadataLogConfig( + 10240, 10240, 10 * 1000, 10240, @@ -1077,6 +1082,7 @@ object KafkaMetadataLogTest { } val DefaultMetadataLogConfig = new MetadataLogConfig( + 100 * 1024, 100 * 1024, 10 * 1000, 100 * 1024, diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index fff1930a71836..d9eaa7b2aacc0 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -100,7 +100,7 @@ class AbstractPartitionTest { def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 512: java.lang.Integer) + logProps.put(TopicConfig.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) } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 77b098cf68298..4a3051ddc9567 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -362,7 +362,7 @@ class PartitionLockTest extends Logging { private def createLogProperties(overrides: Map[String, String]): Properties = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 512: java.lang.Integer) + logProps.put(TopicConfig.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) } diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala index 0bfd67d31c3d1..5db59dd51fe29 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala @@ -28,8 +28,8 @@ import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.clients.{ClientResponse, NetworkClient} import org.apache.kafka.common.internals.Topic -import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME +import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.{ApiKeys, Errors} @@ -37,7 +37,7 @@ import org.apache.kafka.common.record.{FileRecords, MemoryRecords, RecordBatch, import org.apache.kafka.common.requests._ import org.apache.kafka.common.utils.{LogContext, MockTime, ProducerIdAndEpoch} import org.apache.kafka.common.{Node, TopicPartition, Uuid} -import org.apache.kafka.coordinator.transaction.ProducerIdManager +import org.apache.kafka.coordinator.transaction.{ProducerIdManager, TransactionState} import org.apache.kafka.metadata.MetadataCache import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion, RequestLocal, TransactionVersion} import org.apache.kafka.server.storage.log.FetchIsolation @@ -468,7 +468,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren val txnMetadata = transactionMetadata(txn).getOrElse(throw new IllegalStateException(s"Transaction not found $txn")) txnRecords += new SimpleRecord(txn.txnMessageKeyBytes, TransactionLog.valueToBytes(txnMetadata.prepareNoTransit(), TransactionVersion.TV_2)) - txnMetadata.state = PrepareCommit + txnMetadata.state = TransactionState.PREPARE_COMMIT txnRecords += new SimpleRecord(txn.txnMessageKeyBytes, TransactionLog.valueToBytes(txnMetadata.prepareNoTransit(), TransactionVersion.TV_2)) prepareTxnLog(partitionId) @@ -513,7 +513,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren producerEpoch = (Short.MaxValue - 1).toShort, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 60000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = collection.mutable.Set.empty[TopicPartition], txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TransactionVersion.TV_0) @@ -544,7 +544,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren override def awaitAndVerify(txn: Transaction): Unit = { val initPidResult = result.getOrElse(throw new IllegalStateException("InitProducerId has not completed")) assertEquals(Errors.NONE, initPidResult.error) - verifyTransaction(txn, Empty) + verifyTransaction(txn, TransactionState.EMPTY) } } @@ -564,7 +564,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren override def awaitAndVerify(txn: Transaction): Unit = { val error = result.getOrElse(throw new IllegalStateException("AddPartitionsToTransaction has not completed")) assertEquals(Errors.NONE, error) - verifyTransaction(txn, Ongoing) + verifyTransaction(txn, TransactionState.ONGOING) } } @@ -585,7 +585,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren if (!txn.ended) { txn.ended = true assertEquals(Errors.NONE, error) - val expectedState = if (transactionResult(txn) == TransactionResult.COMMIT) CompleteCommit else CompleteAbort + val expectedState = if (transactionResult(txn) == TransactionResult.COMMIT) TransactionState.COMPLETE_COMMIT else TransactionState.COMPLETE_ABORT verifyTransaction(txn, expectedState) } else assertEquals(Errors.INVALID_TXN_STATE, error) @@ -606,7 +606,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren override def await(): Unit = { allTransactions.foreach { txn => if (txnStateManager.partitionFor(txn.transactionalId) == txnTopicPartitionId) { - verifyTransaction(txn, CompleteCommit) + verifyTransaction(txn, TransactionState.COMPLETE_COMMIT) } } } diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala index 94ccd6dc03da7..bad1d6e91f66c 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala @@ -22,7 +22,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.{AddPartitionsToTxnResponse, TransactionResult} import org.apache.kafka.common.utils.{LogContext, MockTime, ProducerIdAndEpoch} -import org.apache.kafka.coordinator.transaction.{ProducerIdManager, TransactionStateManagerConfig} +import org.apache.kafka.coordinator.transaction.{ProducerIdManager, TransactionState, TransactionStateManagerConfig} import org.apache.kafka.server.common.TransactionVersion import org.apache.kafka.server.common.TransactionVersion.{TV_0, TV_2} import org.apache.kafka.server.util.MockScheduler @@ -197,7 +197,7 @@ class TransactionCoordinatorTest { initPidGenericMocks(transactionalId) val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, (Short.MaxValue - 1).toShort, - (Short.MaxValue - 2).toShort, txnTimeoutMs, Empty, mutable.Set.empty, time.milliseconds(), time.milliseconds(), TV_0) + (Short.MaxValue - 2).toShort, txnTimeoutMs, TransactionState.EMPTY, mutable.Set.empty, time.milliseconds(), time.milliseconds(), TV_0) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -229,11 +229,11 @@ class TransactionCoordinatorTest { initPidGenericMocks(transactionalId) val txnMetadata1 = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, (Short.MaxValue - 1).toShort, - (Short.MaxValue - 2).toShort, txnTimeoutMs, Ongoing, mutable.Set.empty, time.milliseconds(), time.milliseconds(), TV_2) - // We start with txnMetadata1 so we can transform the metadata to PrepareCommit. + (Short.MaxValue - 2).toShort, txnTimeoutMs, TransactionState.ONGOING, mutable.Set.empty, time.milliseconds(), time.milliseconds(), TV_2) + // We start with txnMetadata1 so we can transform the metadata to TransactionState.PREPARE_COMMIT. val txnMetadata2 = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, (Short.MaxValue - 1).toShort, - (Short.MaxValue - 2).toShort, txnTimeoutMs, Ongoing, mutable.Set.empty, time.milliseconds(), time.milliseconds(), TV_2) - val transitMetadata = txnMetadata2.prepareAbortOrCommit(PrepareCommit, TV_2, producerId2, time.milliseconds(), false) + (Short.MaxValue - 2).toShort, txnTimeoutMs, TransactionState.ONGOING, mutable.Set.empty, time.milliseconds(), time.milliseconds(), TV_2) + val transitMetadata = txnMetadata2.prepareAbortOrCommit(TransactionState.PREPARE_COMMIT, TV_2, producerId2, time.milliseconds(), false) txnMetadata2.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata2.producerId) @@ -342,7 +342,7 @@ class TransactionCoordinatorTest { } // If producer ID is not the same, return INVALID_PRODUCER_ID_MAPPING val wrongPidTxnMetadata = new TransactionMetadata(transactionalId, 1, 0, RecordBatch.NO_PRODUCER_ID, - 0, RecordBatch.NO_PRODUCER_EPOCH, 0, PrepareCommit, partitions, 0, 0, TV_0) + 0, RecordBatch.NO_PRODUCER_EPOCH, 0, TransactionState.PREPARE_COMMIT, partitions, 0, 0, TV_0) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(new CoordinatorEpochAndTxnMetadata(coordinatorEpoch, wrongPidTxnMetadata)))) @@ -353,7 +353,7 @@ class TransactionCoordinatorTest { // If producer epoch is not equal, return PRODUCER_FENCED val oldEpochTxnMetadata = new TransactionMetadata(transactionalId, 0, 0, RecordBatch.NO_PRODUCER_ID, - 0, RecordBatch.NO_PRODUCER_EPOCH, 0, PrepareCommit, partitions, 0, 0, TV_0) + 0, RecordBatch.NO_PRODUCER_EPOCH, 0, TransactionState.PREPARE_COMMIT, partitions, 0, 0, TV_0) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(new CoordinatorEpochAndTxnMetadata(coordinatorEpoch, oldEpochTxnMetadata)))) @@ -364,7 +364,7 @@ class TransactionCoordinatorTest { // If the txn state is Prepare or AbortCommit, we return CONCURRENT_TRANSACTIONS val emptyTxnMetadata = new TransactionMetadata(transactionalId, 0, 0, RecordBatch.NO_PRODUCER_ID, - 0, RecordBatch.NO_PRODUCER_EPOCH, 0, PrepareCommit, partitions, 0, 0, TV_0) + 0, RecordBatch.NO_PRODUCER_EPOCH, 0, TransactionState.PREPARE_COMMIT, partitions, 0, 0, TV_0) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(new CoordinatorEpochAndTxnMetadata(coordinatorEpoch, emptyTxnMetadata)))) @@ -375,8 +375,8 @@ class TransactionCoordinatorTest { // Pending state does not matter, we will just check if the partitions are in the txnMetadata. val ongoingTxnMetadata = new TransactionMetadata(transactionalId, 0, 0, RecordBatch.NO_PRODUCER_ID, - 0, RecordBatch.NO_PRODUCER_EPOCH, 0, Ongoing, mutable.Set.empty, 0, 0, TV_0) - ongoingTxnMetadata.pendingState = Some(CompleteCommit) + 0, RecordBatch.NO_PRODUCER_EPOCH, 0, TransactionState.ONGOING, mutable.Set.empty, 0, 0, TV_0) + ongoingTxnMetadata.pendingState = Some(TransactionState.COMPLETE_COMMIT) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(new CoordinatorEpochAndTxnMetadata(coordinatorEpoch, ongoingTxnMetadata)))) @@ -388,16 +388,16 @@ class TransactionCoordinatorTest { @Test def shouldRespondWithConcurrentTransactionsOnAddPartitionsWhenStateIsPrepareCommit(): Unit = { - validateConcurrentTransactions(PrepareCommit) + validateConcurrentTransactions(TransactionState.PREPARE_COMMIT) } @Test def shouldRespondWithConcurrentTransactionOnAddPartitionsWhenStateIsPrepareAbort(): Unit = { - validateConcurrentTransactions(PrepareAbort) + validateConcurrentTransactions(TransactionState.PREPARE_ABORT) } def validateConcurrentTransactions(state: TransactionState): Unit = { - // Since the clientTransactionVersion doesn't matter, use 2 since the states are PrepareCommit and PrepareAbort. + // Since the clientTransactionVersion doesn't matter, use 2 since the states are TransactionState.PREPARE_COMMIT and TransactionState.PREPARE_ABORT. when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, 0, 0, RecordBatch.NO_PRODUCER_ID, @@ -409,11 +409,11 @@ class TransactionCoordinatorTest { @Test def shouldRespondWithProducerFencedOnAddPartitionsWhenEpochsAreDifferent(): Unit = { - // Since the clientTransactionVersion doesn't matter, use 2 since the state is PrepareCommit. + // Since the clientTransactionVersion doesn't matter, use 2 since the state is TransactionState.PREPARE_COMMIT. when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, 0, 0, RecordBatch.NO_PRODUCER_ID, - 10, 9, 0, PrepareCommit, mutable.Set.empty, 0, 0, TV_2))))) + 10, 9, 0, TransactionState.PREPARE_COMMIT, mutable.Set.empty, 0, 0, TV_2))))) coordinator.handleAddPartitionsToTransaction(transactionalId, 0L, 0, partitions, errorsCallback, TV_2) assertEquals(Errors.PRODUCER_FENCED, error) @@ -421,24 +421,24 @@ class TransactionCoordinatorTest { @Test def shouldAppendNewMetadataToLogOnAddPartitionsWhenPartitionsAdded(): Unit = { - validateSuccessfulAddPartitions(Empty, 0) + validateSuccessfulAddPartitions(TransactionState.EMPTY, 0) } @Test def shouldRespondWithSuccessOnAddPartitionsWhenStateIsOngoing(): Unit = { - validateSuccessfulAddPartitions(Ongoing, 0) + validateSuccessfulAddPartitions(TransactionState.ONGOING, 0) } @ParameterizedTest @ValueSource(shorts = Array(0, 2)) def shouldRespondWithSuccessOnAddPartitionsWhenStateIsCompleteCommit(clientTransactionVersion: Short): Unit = { - validateSuccessfulAddPartitions(CompleteCommit, clientTransactionVersion) + validateSuccessfulAddPartitions(TransactionState.COMPLETE_COMMIT, clientTransactionVersion) } @ParameterizedTest @ValueSource(shorts = Array(0, 2)) def shouldRespondWithSuccessOnAddPartitionsWhenStateIsCompleteAbort(clientTransactionVersion: Short): Unit = { - validateSuccessfulAddPartitions(CompleteAbort, clientTransactionVersion) + validateSuccessfulAddPartitions(TransactionState.COMPLETE_ABORT, clientTransactionVersion) } def validateSuccessfulAddPartitions(previousState: TransactionState, transactionVersion: Short): Unit = { @@ -467,7 +467,7 @@ class TransactionCoordinatorTest { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, 0, 0, RecordBatch.NO_PRODUCER_ID, - 0, RecordBatch.NO_PRODUCER_EPOCH, 0, Empty, partitions, 0, 0, TV_0))))) + 0, RecordBatch.NO_PRODUCER_EPOCH, 0, TransactionState.EMPTY, partitions, 0, 0, TV_0))))) coordinator.handleAddPartitionsToTransaction(transactionalId, 0L, 0, partitions, errorsCallback, TV_0) assertEquals(Errors.NONE, error) @@ -484,7 +484,7 @@ class TransactionCoordinatorTest { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, 0, 0, RecordBatch.NO_PRODUCER_ID, - 0, RecordBatch.NO_PRODUCER_EPOCH, 0, Ongoing, partitions, 0, 0, TV_0))))) + 0, RecordBatch.NO_PRODUCER_EPOCH, 0, TransactionState.ONGOING, partitions, 0, 0, TV_0))))) coordinator.handleVerifyPartitionsInTransaction(transactionalId, 0L, 0, partitions, verifyPartitionsInTxnCallback) errors.foreach { case (_, error) => @@ -503,7 +503,7 @@ class TransactionCoordinatorTest { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, 0, 0, RecordBatch.NO_PRODUCER_ID, - 0, RecordBatch.NO_PRODUCER_EPOCH, 0, Empty, partitions, 0, 0, TV_0))))) + 0, RecordBatch.NO_PRODUCER_EPOCH, 0, TransactionState.EMPTY, partitions, 0, 0, TV_0))))) val extraPartitions = partitions ++ Set(new TopicPartition("topic2", 0)) @@ -532,7 +532,7 @@ class TransactionCoordinatorTest { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, 10, 10, RecordBatch.NO_PRODUCER_ID, - 0, RecordBatch.NO_PRODUCER_EPOCH, 0, Ongoing, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_0))))) + 0, RecordBatch.NO_PRODUCER_EPOCH, 0, TransactionState.ONGOING, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_0))))) coordinator.handleEndTransaction(transactionalId, 0, 0, TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback) assertEquals(Errors.INVALID_PRODUCER_ID_MAPPING, error) @@ -546,7 +546,7 @@ class TransactionCoordinatorTest { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, producerEpoch, - (producerEpoch - 1).toShort, 1, Ongoing, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_0))))) + (producerEpoch - 1).toShort, 1, TransactionState.ONGOING, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_0))))) coordinator.handleEndTransaction(transactionalId, producerId, 0, TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback) assertEquals(Errors.PRODUCER_FENCED, error) @@ -560,7 +560,7 @@ class TransactionCoordinatorTest { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, producerEpoch, - (producerEpoch - 1).toShort, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) + (producerEpoch - 1).toShort, 1, TransactionState.COMPLETE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) val epoch = if (isRetry) producerEpoch - 1 else producerEpoch coordinator.handleEndTransaction(transactionalId, producerId, epoch.toShort, TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback) @@ -587,7 +587,7 @@ class TransactionCoordinatorTest { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, producerEpoch, - (producerEpoch - 1).toShort, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) + (producerEpoch - 1).toShort, 1, TransactionState.COMPLETE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) val epoch = if (isRetry) producerEpoch - 1 else producerEpoch coordinator.handleEndTransaction(transactionalId, producerId, epoch.toShort, TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback) @@ -604,7 +604,7 @@ class TransactionCoordinatorTest { def testEndTxnWhenStatusIsCompleteAbortAndResultIsAbortInV1(isRetry: Boolean): Unit = { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(0) val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - producerEpoch, (producerEpoch - 1).toShort, 1, CompleteAbort, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) + producerEpoch, (producerEpoch - 1).toShort, 1, TransactionState.COMPLETE_ABORT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -623,7 +623,7 @@ class TransactionCoordinatorTest { def shouldReturnOkOnEndTxnWhenStatusIsCompleteAbortAndResultIsAbortInV2(isRetry: Boolean): Unit = { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(2) val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - producerEpoch, (producerEpoch - 1).toShort, 1, CompleteAbort, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) + producerEpoch, (producerEpoch - 1).toShort, 1, TransactionState.COMPLETE_ABORT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -660,7 +660,7 @@ class TransactionCoordinatorTest { def shouldReturnInvalidTxnRequestOnEndTxnRequestWhenStatusIsCompleteAbortAndResultIsNotAbort(transactionVersion: Short): Unit = { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(transactionVersion) val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - producerEpoch, (producerEpoch - 1).toShort, 1, CompleteAbort, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) + producerEpoch, (producerEpoch - 1).toShort, 1, TransactionState.COMPLETE_ABORT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -673,7 +673,7 @@ class TransactionCoordinatorTest { def shouldReturnInvalidTxnRequestOnEndTxnRequestWhenStatusIsCompleteCommitAndResultIsNotCommit(): Unit = { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(0) val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - producerEpoch, (producerEpoch - 1).toShort,1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) + producerEpoch, (producerEpoch - 1).toShort,1, TransactionState.COMPLETE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -687,7 +687,7 @@ class TransactionCoordinatorTest { def testEndTxnRequestWhenStatusIsCompleteCommitAndResultIsAbortInV1(isRetry: Boolean): Unit = { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(0) val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - producerEpoch, (producerEpoch - 1).toShort, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) + producerEpoch, (producerEpoch - 1).toShort, 1, TransactionState.COMPLETE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -706,7 +706,7 @@ class TransactionCoordinatorTest { def testEndTxnRequestWhenStatusIsCompleteCommitAndResultIsAbortInV2(isRetry: Boolean): Unit = { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(2) val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - producerEpoch, (producerEpoch - 1).toShort, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) + producerEpoch, (producerEpoch - 1).toShort, 1, TransactionState.COMPLETE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -737,7 +737,7 @@ class TransactionCoordinatorTest { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(transactionVersion) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_ID, producerEpoch, (producerEpoch - 1).toShort, 1, PrepareCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) + RecordBatch.NO_PRODUCER_ID, producerEpoch, (producerEpoch - 1).toShort, 1, TransactionState.PREPARE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) coordinator.handleEndTransaction(transactionalId, producerId, requestEpoch(clientTransactionVersion), TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback) assertEquals(Errors.CONCURRENT_TRANSACTIONS, error) @@ -750,7 +750,7 @@ class TransactionCoordinatorTest { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(transactionVersion) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, PrepareAbort, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) + RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, TransactionState.PREPARE_ABORT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) coordinator.handleEndTransaction(transactionalId, producerId, requestEpoch(clientTransactionVersion), TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback) assertEquals(Errors.INVALID_TXN_STATE, error) @@ -762,7 +762,7 @@ class TransactionCoordinatorTest { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(0) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, Empty, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) + RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, TransactionState.EMPTY, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.ABORT, clientTransactionVersion, endTxnCallback) assertEquals(Errors.INVALID_TXN_STATE, error) @@ -775,7 +775,7 @@ class TransactionCoordinatorTest { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(2) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, Empty, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) + RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, TransactionState.EMPTY, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) val epoch = if (isRetry) producerEpoch - 1 else producerEpoch coordinator.handleEndTransaction(transactionalId, producerId, epoch.toShort, TransactionResult.ABORT, clientTransactionVersion, endTxnCallback) @@ -804,7 +804,7 @@ class TransactionCoordinatorTest { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(2) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, Empty, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) + RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, TransactionState.EMPTY, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) val epoch = if (isRetry) producerEpoch - 1 else producerEpoch coordinator.handleEndTransaction(transactionalId, producerId, epoch.toShort, TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback) @@ -820,7 +820,7 @@ class TransactionCoordinatorTest { def shouldReturnInvalidTxnRequestOnEndTxnV2IfNotEndTxnV2Retry(): Unit = { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, PrepareCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2))))) + RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, TransactionState.PREPARE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2))))) // If producerEpoch is the same, this is not a retry of the EndTxnRequest, but the next EndTxnRequest. Return PRODUCER_FENCED. coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.COMMIT, TV_2, endTxnCallback) @@ -829,7 +829,7 @@ class TransactionCoordinatorTest { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2))))) + RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, TransactionState.COMPLETE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2))))) // If producerEpoch is the same, this is not a retry of the EndTxnRequest, but the next EndTxnRequest. Return INVALID_TXN_STATE. coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.COMMIT, TV_2, endTxnCallback) @@ -841,7 +841,7 @@ class TransactionCoordinatorTest { def shouldReturnOkOnEndTxnV2IfEndTxnV2RetryEpochOverflow(): Unit = { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, - producerId2, Short.MaxValue, (Short.MaxValue - 1).toShort, 1, PrepareCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2))))) + producerId2, Short.MaxValue, (Short.MaxValue - 1).toShort, 1, TransactionState.PREPARE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2))))) // Return CONCURRENT_TRANSACTIONS while transaction is still completing coordinator.handleEndTransaction(transactionalId, producerId, (Short.MaxValue - 1).toShort, TransactionResult.COMMIT, TV_2, endTxnCallback) @@ -850,7 +850,7 @@ class TransactionCoordinatorTest { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId2, producerId, - RecordBatch.NO_PRODUCER_ID, 0, RecordBatch.NO_PRODUCER_EPOCH, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2))))) + RecordBatch.NO_PRODUCER_ID, 0, RecordBatch.NO_PRODUCER_EPOCH, 1, TransactionState.COMPLETE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2))))) coordinator.handleEndTransaction(transactionalId, producerId, (Short.MaxValue - 1).toShort, TransactionResult.COMMIT, TV_2, endTxnCallback) assertEquals(Errors.NONE, error) @@ -863,7 +863,7 @@ class TransactionCoordinatorTest { @Test def shouldReturnConcurrentTxnOnAddPartitionsIfEndTxnV2EpochOverflowAndNotComplete(): Unit = { val prepareWithPending = new TransactionMetadata(transactionalId, producerId, producerId, - producerId2, Short.MaxValue, (Short.MaxValue - 1).toShort, 1, PrepareCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2) + producerId2, Short.MaxValue, (Short.MaxValue - 1).toShort, 1, TransactionState.PREPARE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2) val txnTransitMetadata = prepareWithPending.prepareComplete(time.milliseconds()) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) @@ -875,7 +875,7 @@ class TransactionCoordinatorTest { verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId)) prepareWithPending.completeTransitionTo(txnTransitMetadata) - assertEquals(CompleteCommit, prepareWithPending.state) + assertEquals(TransactionState.COMPLETE_COMMIT, prepareWithPending.state) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, prepareWithPending)))) when(transactionManager.appendTransactionToLog( @@ -897,7 +897,7 @@ class TransactionCoordinatorTest { @ValueSource(shorts = Array(0, 2)) def shouldAppendPrepareCommitToLogOnEndTxnWhenStatusIsOngoingAndResultIsCommit(transactionVersion: Short): Unit = { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(transactionVersion) - mockPrepare(PrepareCommit, clientTransactionVersion) + mockPrepare(TransactionState.PREPARE_COMMIT, clientTransactionVersion) coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback) verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId)) @@ -914,7 +914,7 @@ class TransactionCoordinatorTest { @ValueSource(shorts = Array(0, 2)) def shouldAppendPrepareAbortToLogOnEndTxnWhenStatusIsOngoingAndResultIsAbort(transactionVersion: Short): Unit = { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(transactionVersion) - mockPrepare(PrepareAbort, clientTransactionVersion) + mockPrepare(TransactionState.PREPARE_ABORT, clientTransactionVersion) coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.ABORT, clientTransactionVersion, endTxnCallback) verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId)) @@ -989,7 +989,7 @@ class TransactionCoordinatorTest { when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, metadataEpoch, 1, - 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) + 1, TransactionState.COMPLETE_COMMIT, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion))))) coordinator.handleEndTransaction(transactionalId, producerId, requestEpoch, TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback) assertEquals(Errors.PRODUCER_FENCED, error) @@ -998,29 +998,29 @@ class TransactionCoordinatorTest { @Test def shouldIncrementEpochAndUpdateMetadataOnHandleInitPidWhenExistingEmptyTransaction(): Unit = { - validateIncrementEpochAndUpdateMetadata(Empty, 0) + validateIncrementEpochAndUpdateMetadata(TransactionState.EMPTY, 0) } @ParameterizedTest @ValueSource(shorts = Array(0, 2)) def shouldIncrementEpochAndUpdateMetadataOnHandleInitPidWhenExistingCompleteTransaction(clientTransactionVersion: Short): Unit = { - validateIncrementEpochAndUpdateMetadata(CompleteAbort, clientTransactionVersion) + validateIncrementEpochAndUpdateMetadata(TransactionState.COMPLETE_ABORT, clientTransactionVersion) } @ParameterizedTest @ValueSource(shorts = Array(0, 2)) def shouldIncrementEpochAndUpdateMetadataOnHandleInitPidWhenExistingCompleteCommitTransaction(clientTransactionVersion: Short): Unit = { - validateIncrementEpochAndUpdateMetadata(CompleteCommit, clientTransactionVersion) + validateIncrementEpochAndUpdateMetadata(TransactionState.COMPLETE_COMMIT, clientTransactionVersion) } @Test def shouldWaitForCommitToCompleteOnHandleInitPidAndExistingTransactionInPrepareCommitState(): Unit = { - validateRespondsWithConcurrentTransactionsOnInitPidWhenInPrepareState(PrepareCommit) + validateRespondsWithConcurrentTransactionsOnInitPidWhenInPrepareState(TransactionState.PREPARE_COMMIT) } @Test def shouldWaitForCommitToCompleteOnHandleInitPidAndExistingTransactionInPrepareAbortState(): Unit = { - validateRespondsWithConcurrentTransactionsOnInitPidWhenInPrepareState(PrepareAbort) + validateRespondsWithConcurrentTransactionsOnInitPidWhenInPrepareState(TransactionState.PREPARE_ABORT) } @ParameterizedTest(name = "enableTwoPCFlag={0}, keepPreparedTxn={1}") @@ -1030,7 +1030,7 @@ class TransactionCoordinatorTest { keepPreparedTxn: Boolean ): Unit = { val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - producerEpoch, (producerEpoch - 1).toShort, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0) + producerEpoch, (producerEpoch - 1).toShort, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_0) when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt())) .thenReturn(true) @@ -1041,7 +1041,7 @@ class TransactionCoordinatorTest { when(transactionManager.transactionVersionLevel()).thenReturn(TV_0) val originalMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0) + (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_0) when(transactionManager.appendTransactionToLog( ArgumentMatchers.eq(transactionalId), ArgumentMatchers.eq(coordinatorEpoch), @@ -1066,7 +1066,7 @@ class TransactionCoordinatorTest { verify(transactionManager).appendTransactionToLog( ArgumentMatchers.eq(transactionalId), ArgumentMatchers.eq(coordinatorEpoch), - ArgumentMatchers.eq(originalMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false)), + ArgumentMatchers.eq(originalMetadata.prepareAbortOrCommit(TransactionState.PREPARE_ABORT, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false)), any(), any(), any()) @@ -1075,13 +1075,13 @@ class TransactionCoordinatorTest { @Test def shouldFailToAbortTransactionOnHandleInitPidWhenProducerEpochIsSmaller(): Unit = { val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - producerEpoch, (producerEpoch - 1).toShort, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0) + producerEpoch, (producerEpoch - 1).toShort, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_0) when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt())) .thenReturn(true) val bumpedTxnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - (producerEpoch + 2).toShort, (producerEpoch - 1).toShort, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0) + (producerEpoch + 2).toShort, (producerEpoch - 1).toShort, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_0) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -1106,7 +1106,7 @@ class TransactionCoordinatorTest { @Test def shouldNotRepeatedlyBumpEpochDueToInitPidDuringOngoingTxnIfAppendToLogFails(): Unit = { val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0) + producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_0) when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt())) .thenReturn(true) @@ -1120,8 +1120,8 @@ class TransactionCoordinatorTest { when(transactionManager.transactionVersionLevel()).thenReturn(TV_0) val originalMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0) - val txnTransitMetadata = originalMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false) + (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_0) + val txnTransitMetadata = originalMetadata.prepareAbortOrCommit(TransactionState.PREPARE_ABORT, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false) when(transactionManager.appendTransactionToLog( ArgumentMatchers.eq(transactionalId), ArgumentMatchers.eq(coordinatorEpoch), @@ -1198,14 +1198,14 @@ class TransactionCoordinatorTest { @Test def shouldUseLastEpochToFenceWhenEpochsAreExhausted(): Unit = { val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - (Short.MaxValue - 1).toShort, (Short.MaxValue - 2).toShort, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0) + (Short.MaxValue - 1).toShort, (Short.MaxValue - 2).toShort, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_0) assertTrue(txnMetadata.isProducerEpochExhausted) when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt())) .thenReturn(true) val postFenceTxnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - Short.MaxValue, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, PrepareAbort, partitions, time.milliseconds(), time.milliseconds(), TV_0) + Short.MaxValue, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.PREPARE_ABORT, partitions, time.milliseconds(), time.milliseconds(), TV_0) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -1224,7 +1224,7 @@ class TransactionCoordinatorTest { producerEpoch = Short.MaxValue, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = txnTimeoutMs, - txnState = PrepareAbort, + txnState = TransactionState.PREPARE_ABORT, topicPartitions = partitions.clone, txnStartTimestamp = time.milliseconds(), txnLastUpdateTimestamp = time.milliseconds(), @@ -1257,7 +1257,7 @@ class TransactionCoordinatorTest { producerEpoch = Short.MaxValue, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = txnTimeoutMs, - txnState = PrepareAbort, + txnState = TransactionState.PREPARE_ABORT, topicPartitions = partitions.clone, txnStartTimestamp = time.milliseconds(), txnLastUpdateTimestamp = time.milliseconds(), @@ -1272,7 +1272,7 @@ class TransactionCoordinatorTest { // If the metadata doesn't include the previous producer data (for example, if it was written to the log by a broker // on an old version), the retry case should fail val txnMetadata = new TransactionMetadata(transactionalId, producerId, RecordBatch.NO_PRODUCER_ID, - RecordBatch.NO_PRODUCER_EPOCH, (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Empty, partitions, time.milliseconds, time.milliseconds, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.EMPTY, partitions, time.milliseconds, time.milliseconds, TV_0) when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt())) .thenReturn(true) @@ -1295,7 +1295,7 @@ class TransactionCoordinatorTest { def testFenceProducerWhenMappingExistsWithDifferentProducerId(): Unit = { // Existing transaction ID maps to new producer ID val txnMetadata = new TransactionMetadata(transactionalId, producerId + 1, producerId, - RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, (producerEpoch - 1).toShort, txnTimeoutMs, Empty, partitions, time.milliseconds, time.milliseconds, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, (producerEpoch - 1).toShort, txnTimeoutMs, TransactionState.EMPTY, partitions, time.milliseconds, time.milliseconds, TV_0) when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt())) .thenReturn(true) @@ -1319,7 +1319,7 @@ class TransactionCoordinatorTest { mockPidGenerator() val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_EPOCH, 10, 9, txnTimeoutMs, Empty, partitions, time.milliseconds, time.milliseconds, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, 10, 9, txnTimeoutMs, TransactionState.EMPTY, partitions, time.milliseconds, time.milliseconds, TV_0) when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt())) .thenReturn(true) @@ -1366,7 +1366,7 @@ class TransactionCoordinatorTest { mockPidGenerator() val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_EPOCH, 10, 9, txnTimeoutMs, Empty, partitions, time.milliseconds, time.milliseconds, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, 10, 9, txnTimeoutMs, TransactionState.EMPTY, partitions, time.milliseconds, time.milliseconds, TV_0) when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt())) .thenReturn(true) @@ -1415,7 +1415,7 @@ class TransactionCoordinatorTest { def testRetryInitProducerIdAfterProducerIdRotation(): Unit = { // Existing transaction ID maps to new producer ID val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_EPOCH, (Short.MaxValue - 1).toShort, (Short.MaxValue - 2).toShort, txnTimeoutMs, Empty, partitions, time.milliseconds, time.milliseconds, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, (Short.MaxValue - 1).toShort, (Short.MaxValue - 2).toShort, txnTimeoutMs, TransactionState.EMPTY, partitions, time.milliseconds, time.milliseconds, TV_0) when(pidGenerator.generateProducerId()) .thenReturn(producerId + 1) @@ -1468,7 +1468,7 @@ class TransactionCoordinatorTest { def testInitProducerIdWithInvalidEpochAfterProducerIdRotation(): Unit = { // Existing transaction ID maps to new producer ID val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_EPOCH, (Short.MaxValue - 1).toShort, (Short.MaxValue - 2).toShort, txnTimeoutMs, Empty, partitions, time.milliseconds, time.milliseconds, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, (Short.MaxValue - 1).toShort, (Short.MaxValue - 2).toShort, txnTimeoutMs, TransactionState.EMPTY, partitions, time.milliseconds, time.milliseconds, TV_0) when(pidGenerator.generateProducerId()) .thenReturn(producerId + 1) @@ -1528,7 +1528,7 @@ class TransactionCoordinatorTest { def shouldAbortExpiredTransactionsInOngoingStateAndBumpEpoch(): Unit = { val now = time.milliseconds() val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, now, now, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, now, now, TV_0) when(transactionManager.timedOutTransactions()) .thenReturn(List(TransactionalIdAndProducerIdEpoch(transactionalId, producerId, producerEpoch))) @@ -1537,7 +1537,7 @@ class TransactionCoordinatorTest { // Transaction timeouts use FenceProducerEpoch so clientTransactionVersion is 0. val expectedTransition = TxnTransitMetadata(producerId, producerId, RecordBatch.NO_PRODUCER_EPOCH, (producerEpoch + 1).toShort, - RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, PrepareAbort, partitions.clone, now, + RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.PREPARE_ABORT, partitions.clone, now, now + TransactionStateManagerConfig.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_DEFAULT, TV_0) when(transactionManager.transactionVersionLevel()).thenReturn(TV_0) @@ -1567,7 +1567,7 @@ class TransactionCoordinatorTest { def shouldNotAcceptSmallerEpochDuringTransactionExpiration(): Unit = { val now = time.milliseconds() val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, now, now, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, now, now, TV_0) when(transactionManager.timedOutTransactions()) .thenReturn(List(TransactionalIdAndProducerIdEpoch(transactionalId, producerId, producerEpoch))) @@ -1577,7 +1577,7 @@ class TransactionCoordinatorTest { when(transactionManager.transactionVersionLevel()).thenReturn(TV_0) val bumpedTxnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_EPOCH, (producerEpoch + 2).toShort, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, now, now, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, (producerEpoch + 2).toShort, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, now, now, TV_0) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, bumpedTxnMetadata)))) @@ -1593,8 +1593,8 @@ class TransactionCoordinatorTest { @Test def shouldNotAbortExpiredTransactionsThatHaveAPendingStateTransition(): Unit = { val metadata = new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0) - metadata.prepareAbortOrCommit(PrepareCommit, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false) + RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_0) + metadata.prepareAbortOrCommit(TransactionState.PREPARE_COMMIT, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false) when(transactionManager.timedOutTransactions()) .thenReturn(List(TransactionalIdAndProducerIdEpoch(transactionalId, producerId, producerEpoch))) @@ -1612,13 +1612,13 @@ class TransactionCoordinatorTest { def shouldNotBumpEpochWhenAbortingExpiredTransactionIfAppendToLogFails(): Unit = { val now = time.milliseconds() val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, now, now, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, now, now, TV_0) when(transactionManager.timedOutTransactions()) .thenReturn(List(TransactionalIdAndProducerIdEpoch(transactionalId, producerId, producerEpoch))) val txnMetadataAfterAppendFailure = new TransactionMetadata(transactionalId, producerId, producerId, - RecordBatch.NO_PRODUCER_EPOCH, (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, now, now, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, now, now, TV_0) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -1627,7 +1627,7 @@ class TransactionCoordinatorTest { // Transaction timeouts use FenceProducerEpoch so clientTransactionVersion is 0. val bumpedEpoch = (producerEpoch + 1).toShort val expectedTransition = TxnTransitMetadata(producerId, producerId, RecordBatch.NO_PRODUCER_EPOCH, bumpedEpoch, - RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, PrepareAbort, partitions.clone, now, + RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.PREPARE_ABORT, partitions.clone, now, now + TransactionStateManagerConfig.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_DEFAULT, TV_0) when(transactionManager.transactionVersionLevel()).thenReturn(TV_0) @@ -1660,8 +1660,8 @@ class TransactionCoordinatorTest { @Test def shouldNotBumpEpochWithPendingTransaction(): Unit = { val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, - RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0) - txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false) + RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_0) + txnMetadata.prepareAbortOrCommit(TransactionState.PREPARE_COMMIT, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false) when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt())) .thenReturn(true) @@ -1695,7 +1695,7 @@ class TransactionCoordinatorTest { coordinator.startup(() => transactionStatePartitionCount, enableTransactionalIdExpiration = false) val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, - RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Dead, mutable.Set.empty, time.milliseconds(), + RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.DEAD, mutable.Set.empty, time.milliseconds(), time.milliseconds(), TV_0) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -1721,7 +1721,7 @@ class TransactionCoordinatorTest { @Test def testDescribeTransactions(): Unit = { val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, - RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0) + RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, time.milliseconds(), time.milliseconds(), TV_0) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) @@ -1747,7 +1747,7 @@ class TransactionCoordinatorTest { when(transactionManager.validateTransactionTimeoutMs(anyBoolean(), anyInt())) .thenReturn(true) - // Since the clientTransactionVersion doesn't matter, use 2 since the states are PrepareCommit and PrepareAbort. + // Since the clientTransactionVersion doesn't matter, use 2 since the states are TransactionState.PREPARE_COMMIT and TransactionState.PREPARE_ABORT. val metadata = new TransactionMetadata(transactionalId, 0, 0, RecordBatch.NO_PRODUCER_EPOCH, 0, RecordBatch.NO_PRODUCER_EPOCH, 0, state, mutable.Set[TopicPartition](new TopicPartition("topic", 1)), 0, 0, TV_2) when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId))) @@ -1799,7 +1799,7 @@ class TransactionCoordinatorTest { private def mockPrepare(transactionState: TransactionState, clientTransactionVersion: TransactionVersion, runCallback: Boolean = false): TransactionMetadata = { val now = time.milliseconds() val originalMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_EPOCH, - producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, now, now, TV_0) + producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, TransactionState.ONGOING, partitions, now, now, TV_0) val transition = TxnTransitMetadata(producerId, producerId, RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, transactionState, partitions.clone, now, now, clientTransactionVersion) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala index 8a852d70cbe96..d139f1d3b7fea 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala @@ -23,6 +23,7 @@ import org.apache.kafka.common.protocol.{ByteBufferAccessor, MessageUtil} import org.apache.kafka.common.protocol.types.Field.TaggedFieldsSection import org.apache.kafka.common.protocol.types.{CompactArrayOf, Field, Schema, Struct, Type} import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord} +import org.apache.kafka.coordinator.transaction.TransactionState import org.apache.kafka.coordinator.transaction.generated.{TransactionLogKey, TransactionLogValue} import org.apache.kafka.server.common.TransactionVersion.{TV_0, TV_2} import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue, fail} @@ -49,7 +50,7 @@ class TransactionLogTest { val producerId = 23423L val txnMetadata = new TransactionMetadata(transactionalId, producerId, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_ID, producerEpoch, - RecordBatch.NO_PRODUCER_EPOCH, transactionTimeoutMs, Empty, collection.mutable.Set.empty[TopicPartition], 0, 0, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, transactionTimeoutMs, TransactionState.EMPTY, collection.mutable.Set.empty[TopicPartition], 0, 0, TV_0) txnMetadata.addPartitions(topicPartitions) assertThrows(classOf[IllegalStateException], () => TransactionLog.valueToBytes(txnMetadata.prepareNoTransit(), TV_2)) @@ -64,19 +65,19 @@ class TransactionLogTest { "four" -> 4L, "five" -> 5L) - val transactionStates = Map[Long, TransactionState](0L -> Empty, - 1L -> Ongoing, - 2L -> PrepareCommit, - 3L -> CompleteCommit, - 4L -> PrepareAbort, - 5L -> CompleteAbort) + val transactionStates = Map[Long, TransactionState](0L -> TransactionState.EMPTY, + 1L -> TransactionState.ONGOING, + 2L -> TransactionState.PREPARE_COMMIT, + 3L -> TransactionState.COMPLETE_COMMIT, + 4L -> TransactionState.PREPARE_ABORT, + 5L -> TransactionState.COMPLETE_ABORT) // generate transaction log messages val txnRecords = pidMappings.map { case (transactionalId, producerId) => val txnMetadata = new TransactionMetadata(transactionalId, producerId, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, transactionTimeoutMs, transactionStates(producerId), collection.mutable.Set.empty[TopicPartition], 0, 0, TV_0) - if (!txnMetadata.state.equals(Empty)) + if (!txnMetadata.state.equals(TransactionState.EMPTY)) txnMetadata.addPartitions(topicPartitions) val keyBytes = TransactionLog.keyToBytes(transactionalId) @@ -99,7 +100,7 @@ class TransactionLogTest { assertEquals(transactionTimeoutMs, txnMetadata.txnTimeoutMs) assertEquals(transactionStates(txnMetadata.producerId), txnMetadata.state) - if (txnMetadata.state.equals(Empty)) + if (txnMetadata.state.equals(TransactionState.EMPTY)) assertEquals(Set.empty[TopicPartition], txnMetadata.topicPartitions) else assertEquals(topicPartitions, txnMetadata.topicPartitions) @@ -113,14 +114,14 @@ class TransactionLogTest { @Test def testSerializeTransactionLogValueToHighestNonFlexibleVersion(): Unit = { - val txnTransitMetadata = TxnTransitMetadata(1, 1, 1, 1, 1, 1000, CompleteCommit, mutable.Set.empty, 500, 500, TV_0) + val txnTransitMetadata = TxnTransitMetadata(1, 1, 1, 1, 1, 1000, TransactionState.COMPLETE_COMMIT, mutable.Set.empty, 500, 500, TV_0) val txnLogValueBuffer = ByteBuffer.wrap(TransactionLog.valueToBytes(txnTransitMetadata, TV_0)) assertEquals(0, txnLogValueBuffer.getShort) } @Test def testSerializeTransactionLogValueToFlexibleVersion(): Unit = { - val txnTransitMetadata = TxnTransitMetadata(1, 1, 1, 1, 1, 1000, CompleteCommit, mutable.Set.empty, 500, 500, TV_2) + val txnTransitMetadata = TxnTransitMetadata(1, 1, 1, 1, 1, 1000, TransactionState.COMPLETE_COMMIT, mutable.Set.empty, 500, 500, TV_2) val txnLogValueBuffer = ByteBuffer.wrap(TransactionLog.valueToBytes(txnTransitMetadata, TV_2)) assertEquals(TransactionLogValue.HIGHEST_SUPPORTED_VERSION, txnLogValueBuffer.getShort) } @@ -134,7 +135,7 @@ class TransactionLogTest { val txnLogValue = new TransactionLogValue() .setProducerId(100) .setProducerEpoch(50.toShort) - .setTransactionStatus(CompleteCommit.id) + .setTransactionStatus(TransactionState.COMPLETE_COMMIT.id) .setTransactionStartTimestampMs(750L) .setTransactionLastUpdateTimestampMs(1000L) .setTransactionTimeoutMs(500) @@ -145,7 +146,7 @@ class TransactionLogTest { assertEquals(100, deserialized.producerId) assertEquals(50, deserialized.producerEpoch) - assertEquals(CompleteCommit, deserialized.state) + assertEquals(TransactionState.COMPLETE_COMMIT, deserialized.state) assertEquals(750L, deserialized.txnStartTimestamp) assertEquals(1000L, deserialized.txnLastUpdateTimestamp) assertEquals(500, deserialized.txnTimeoutMs) @@ -198,7 +199,7 @@ class TransactionLogTest { transactionLogValue.set("producer_id", 1000L) transactionLogValue.set("producer_epoch", 100.toShort) transactionLogValue.set("transaction_timeout_ms", 1000) - transactionLogValue.set("transaction_status", CompleteCommit.id) + transactionLogValue.set("transaction_status", TransactionState.COMPLETE_COMMIT.id) transactionLogValue.set("transaction_partitions", Array(txnPartitions)) transactionLogValue.set("transaction_last_update_timestamp_ms", 2000L) transactionLogValue.set("transaction_start_timestamp_ms", 3000L) @@ -227,7 +228,7 @@ class TransactionLogTest { assertEquals(1000L, txnMetadata.producerId) assertEquals(100, txnMetadata.producerEpoch) assertEquals(1000L, txnMetadata.txnTimeoutMs) - assertEquals(CompleteCommit, txnMetadata.state) + assertEquals(TransactionState.COMPLETE_COMMIT, txnMetadata.state) assertEquals(Set(new TopicPartition("topic", 1)), txnMetadata.topicPartitions) assertEquals(2000L, txnMetadata.txnLastUpdateTimestamp) assertEquals(3000L, txnMetadata.txnStartTimestamp) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala index 321e6e793f474..4131f564d6384 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala @@ -29,6 +29,7 @@ import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.{RequestHeader, TransactionResult, WriteTxnMarkersRequest, WriteTxnMarkersResponse} import org.apache.kafka.common.utils.MockTime import org.apache.kafka.common.{Node, TopicPartition} +import org.apache.kafka.coordinator.transaction.TransactionState import org.apache.kafka.metadata.MetadataCache import org.apache.kafka.server.common.{MetadataVersion, TransactionVersion} import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} @@ -68,9 +69,9 @@ class TransactionMarkerChannelManagerTest { private val txnTimeoutMs = 0 private val txnResult = TransactionResult.COMMIT private val txnMetadata1 = new TransactionMetadata(transactionalId1, producerId1, producerId1, RecordBatch.NO_PRODUCER_ID, - producerEpoch, lastProducerEpoch, txnTimeoutMs, PrepareCommit, mutable.Set[TopicPartition](partition1, partition2), 0L, 0L, TransactionVersion.TV_2) + producerEpoch, lastProducerEpoch, txnTimeoutMs, TransactionState.PREPARE_COMMIT, mutable.Set[TopicPartition](partition1, partition2), 0L, 0L, TransactionVersion.TV_2) private val txnMetadata2 = new TransactionMetadata(transactionalId2, producerId2, producerId2, RecordBatch.NO_PRODUCER_ID, - producerEpoch, lastProducerEpoch, txnTimeoutMs, PrepareCommit, mutable.Set[TopicPartition](partition1), 0L, 0L, TransactionVersion.TV_2) + producerEpoch, lastProducerEpoch, txnTimeoutMs, TransactionState.PREPARE_COMMIT, mutable.Set[TopicPartition](partition1), 0L, 0L, TransactionVersion.TV_2) private val capturedErrorsCallback: ArgumentCaptor[Errors => Unit] = ArgumentCaptor.forClass(classOf[Errors => Unit]) private val time = new MockTime @@ -480,7 +481,7 @@ class TransactionMarkerChannelManagerTest { assertEquals(0, channelManager.numTxnsWithPendingMarkers) assertEquals(0, channelManager.queueForBroker(broker1.id).get.totalNumMarkers) assertEquals(None, txnMetadata2.pendingState) - assertEquals(CompleteCommit, txnMetadata2.state) + assertEquals(TransactionState.COMPLETE_COMMIT, txnMetadata2.state) } @Test @@ -533,7 +534,7 @@ class TransactionMarkerChannelManagerTest { assertEquals(0, channelManager.numTxnsWithPendingMarkers) assertEquals(0, channelManager.queueForBroker(broker1.id).get.totalNumMarkers) assertEquals(None, txnMetadata2.pendingState) - assertEquals(PrepareCommit, txnMetadata2.state) + assertEquals(TransactionState.PREPARE_COMMIT, txnMetadata2.state) } @ParameterizedTest @@ -594,7 +595,7 @@ class TransactionMarkerChannelManagerTest { assertEquals(0, channelManager.numTxnsWithPendingMarkers) assertEquals(0, channelManager.queueForBroker(broker1.id).get.totalNumMarkers) assertEquals(None, txnMetadata2.pendingState) - assertEquals(CompleteCommit, txnMetadata2.state) + assertEquals(TransactionState.COMPLETE_COMMIT, txnMetadata2.state) } private def createPidErrorMap(errors: Errors): util.HashMap[java.lang.Long, util.Map[TopicPartition, Errors]] = { diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala index 72ffa5629c04e..b34b0725020fc 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala @@ -23,6 +23,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.{RequestHeader, TransactionResult, WriteTxnMarkersRequest, WriteTxnMarkersResponse} +import org.apache.kafka.coordinator.transaction.TransactionState import org.apache.kafka.server.common.TransactionVersion import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -44,7 +45,7 @@ class TransactionMarkerRequestCompletionHandlerTest { private val txnResult = TransactionResult.COMMIT private val topicPartition = new TopicPartition("topic1", 0) private val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, - producerEpoch, lastProducerEpoch, txnTimeoutMs, PrepareCommit, mutable.Set[TopicPartition](topicPartition), 0L, 0L, TransactionVersion.TV_2) + producerEpoch, lastProducerEpoch, txnTimeoutMs, TransactionState.PREPARE_COMMIT, mutable.Set[TopicPartition](topicPartition), 0L, 0L, TransactionVersion.TV_2) private val pendingCompleteTxnAndMarkers = asList( PendingCompleteTxnAndMarkerEntry( PendingCompleteTxn(transactionalId, coordinatorEpoch, txnMetadata, txnMetadata.prepareComplete(42)), 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 12536cddff731..56d595d24ed01 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala @@ -19,6 +19,7 @@ package kafka.coordinator.transaction import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch +import org.apache.kafka.coordinator.transaction.TransactionState import org.apache.kafka.server.common.TransactionVersion import org.apache.kafka.server.common.TransactionVersion.{TV_0, TV_2} import org.apache.kafka.server.util.MockTime @@ -27,7 +28,10 @@ import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource +import java.util.Optional + import scala.collection.mutable +import scala.jdk.CollectionConverters._ class TransactionMetadataTest { @@ -47,7 +51,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -71,7 +75,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -95,7 +99,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -117,13 +121,13 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnStartTimestamp = -1, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_2) - val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() + 1, true) + val transitMetadata = txnMetadata.prepareAbortOrCommit(TransactionState.PREPARE_ABORT, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() + 1, true) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata.producerId) assertEquals(producerEpoch + 1, txnMetadata.producerEpoch) @@ -142,13 +146,13 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = CompleteAbort, + state = TransactionState.COMPLETE_ABORT, topicPartitions = mutable.Set.empty, txnStartTimestamp = time.milliseconds() - 1, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_2) - val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() + 1, true) + val transitMetadata = txnMetadata.prepareAbortOrCommit(TransactionState.PREPARE_ABORT, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() + 1, true) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata.producerId) assertEquals(producerEpoch + 1, txnMetadata.producerEpoch) @@ -167,13 +171,13 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = CompleteCommit, + state = TransactionState.COMPLETE_COMMIT, topicPartitions = mutable.Set.empty, txnStartTimestamp = time.milliseconds() - 1, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_2) - val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() + 1, true) + val transitMetadata = txnMetadata.prepareAbortOrCommit(TransactionState.PREPARE_ABORT, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() + 1, true) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata.producerId) assertEquals(producerEpoch + 1, txnMetadata.producerEpoch) @@ -191,7 +195,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnStartTimestamp = 1L, txnLastUpdateTimestamp = time.milliseconds(), @@ -219,7 +223,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnStartTimestamp = 1L, txnLastUpdateTimestamp = time.milliseconds(), @@ -246,13 +250,13 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnStartTimestamp = time.milliseconds(), txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) - // let new time be smaller; when transiting from Empty the start time would be updated to the update-time + // let new time be smaller; when transiting from TransactionState.EMPTY the start time would be updated to the update-time var transitMetadata = txnMetadata.prepareAddPartitions(Set[TopicPartition](new TopicPartition("topic1", 0)), time.milliseconds() - 1, TV_0) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(Set[TopicPartition](new TopicPartition("topic1", 0)), txnMetadata.topicPartitions) @@ -262,7 +266,7 @@ class TransactionMetadataTest { assertEquals(time.milliseconds() - 1, txnMetadata.txnStartTimestamp) assertEquals(time.milliseconds() - 1, txnMetadata.txnLastUpdateTimestamp) - // add another partition, check that in Ongoing state the start timestamp would not change to update time + // add another partition, check that in TransactionState.ONGOING state the start timestamp would not change to update time transitMetadata = txnMetadata.prepareAddPartitions(Set[TopicPartition](new TopicPartition("topic2", 0)), time.milliseconds() - 2, TV_0) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(Set[TopicPartition](new TopicPartition("topic1", 0), new TopicPartition("topic2", 0)), txnMetadata.topicPartitions) @@ -284,16 +288,16 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Ongoing, + state = TransactionState.ONGOING, topicPartitions = mutable.Set.empty, txnStartTimestamp = 1L, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) // let new time be smaller - val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1, false) + val transitMetadata = txnMetadata.prepareAbortOrCommit(TransactionState.PREPARE_COMMIT, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1, false) txnMetadata.completeTransitionTo(transitMetadata) - assertEquals(PrepareCommit, txnMetadata.state) + assertEquals(TransactionState.PREPARE_COMMIT, txnMetadata.state) assertEquals(producerId, txnMetadata.producerId) assertEquals(RecordBatch.NO_PRODUCER_EPOCH, txnMetadata.lastProducerEpoch) assertEquals(producerEpoch, txnMetadata.producerEpoch) @@ -312,16 +316,16 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Ongoing, + state = TransactionState.ONGOING, topicPartitions = mutable.Set.empty, txnStartTimestamp = 1L, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) // let new time be smaller - val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1, false) + val transitMetadata = txnMetadata.prepareAbortOrCommit(TransactionState.PREPARE_ABORT, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1, false) txnMetadata.completeTransitionTo(transitMetadata) - assertEquals(PrepareAbort, txnMetadata.state) + assertEquals(TransactionState.PREPARE_ABORT, txnMetadata.state) assertEquals(producerId, txnMetadata.producerId) assertEquals(RecordBatch.NO_PRODUCER_EPOCH, txnMetadata.lastProducerEpoch) assertEquals(producerEpoch, txnMetadata.producerEpoch) @@ -343,7 +347,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = lastProducerEpoch, txnTimeoutMs = 30000, - state = PrepareCommit, + state = TransactionState.PREPARE_COMMIT, topicPartitions = mutable.Set.empty, txnStartTimestamp = 1L, txnLastUpdateTimestamp = time.milliseconds(), @@ -354,7 +358,7 @@ class TransactionMetadataTest { val transitMetadata = txnMetadata.prepareComplete(time.milliseconds() - 1) txnMetadata.completeTransitionTo(transitMetadata) - assertEquals(CompleteCommit, txnMetadata.state) + assertEquals(TransactionState.COMPLETE_COMMIT, txnMetadata.state) assertEquals(producerId, txnMetadata.producerId) assertEquals(lastProducerEpoch, txnMetadata.lastProducerEpoch) assertEquals(producerEpoch, txnMetadata.producerEpoch) @@ -376,7 +380,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = lastProducerEpoch, txnTimeoutMs = 30000, - state = PrepareAbort, + state = TransactionState.PREPARE_ABORT, topicPartitions = mutable.Set.empty, txnStartTimestamp = 1L, txnLastUpdateTimestamp = time.milliseconds(), @@ -387,7 +391,7 @@ class TransactionMetadataTest { val transitMetadata = txnMetadata.prepareComplete(time.milliseconds() - 1) txnMetadata.completeTransitionTo(transitMetadata) - assertEquals(CompleteAbort, txnMetadata.state) + assertEquals(TransactionState.COMPLETE_ABORT, txnMetadata.state) assertEquals(producerId, txnMetadata.producerId) assertEquals(lastProducerEpoch, txnMetadata.lastProducerEpoch) assertEquals(producerEpoch, txnMetadata.producerEpoch) @@ -407,7 +411,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Ongoing, + state = TransactionState.ONGOING, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -416,12 +420,12 @@ class TransactionMetadataTest { val fencingTransitMetadata = txnMetadata.prepareFenceProducerEpoch() assertEquals(Short.MaxValue, fencingTransitMetadata.producerEpoch) assertEquals(RecordBatch.NO_PRODUCER_EPOCH, fencingTransitMetadata.lastProducerEpoch) - assertEquals(Some(PrepareEpochFence), txnMetadata.pendingState) + assertEquals(Some(TransactionState.PREPARE_EPOCH_FENCE), txnMetadata.pendingState) // We should reset the pending state to make way for the abort transition. txnMetadata.pendingState = None - val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false) + val transitMetadata = txnMetadata.prepareAbortOrCommit(TransactionState.PREPARE_ABORT, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, transitMetadata.producerId) } @@ -438,7 +442,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = CompleteCommit, + state = TransactionState.COMPLETE_COMMIT, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -459,7 +463,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = CompleteAbort, + state = TransactionState.COMPLETE_ABORT, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -480,7 +484,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Ongoing, + state = TransactionState.ONGOING, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -500,7 +504,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -527,13 +531,13 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Ongoing, + state = TransactionState.ONGOING, topicPartitions = mutable.Set.empty, txnStartTimestamp = time.milliseconds(), txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_2) - var transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1, false) + var transitMetadata = txnMetadata.prepareAbortOrCommit(TransactionState.PREPARE_COMMIT, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1, false) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata.producerId) assertEquals((producerEpoch + 1).toShort, txnMetadata.producerEpoch) @@ -559,7 +563,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Ongoing, + state = TransactionState.ONGOING, topicPartitions = mutable.Set.empty, txnStartTimestamp = time.milliseconds(), txnLastUpdateTimestamp = time.milliseconds(), @@ -567,7 +571,7 @@ class TransactionMetadataTest { assertTrue(txnMetadata.isProducerEpochExhausted) val newProducerId = 9893L - var transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_2, newProducerId, time.milliseconds() - 1, false) + var transitMetadata = txnMetadata.prepareAbortOrCommit(TransactionState.PREPARE_COMMIT, TV_2, newProducerId, time.milliseconds() - 1, false) txnMetadata.completeTransitionTo(transitMetadata) assertEquals(producerId, txnMetadata.producerId) assertEquals(Short.MaxValue, txnMetadata.producerEpoch) @@ -584,21 +588,21 @@ class TransactionMetadataTest { @Test def testRotateProducerIdInOngoingState(): Unit = { - assertThrows(classOf[IllegalStateException], () => testRotateProducerIdInOngoingState(Ongoing, TV_0)) + assertThrows(classOf[IllegalStateException], () => testRotateProducerIdInOngoingState(TransactionState.ONGOING, TV_0)) } @ParameterizedTest @ValueSource(shorts = Array(0, 2)) def testRotateProducerIdInPrepareAbortState(transactionVersion: Short): Unit = { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(transactionVersion) - assertThrows(classOf[IllegalStateException], () => testRotateProducerIdInOngoingState(PrepareAbort, clientTransactionVersion)) + assertThrows(classOf[IllegalStateException], () => testRotateProducerIdInOngoingState(TransactionState.PREPARE_ABORT, clientTransactionVersion)) } @ParameterizedTest @ValueSource(shorts = Array(0, 2)) def testRotateProducerIdInPrepareCommitState(transactionVersion: Short): Unit = { val clientTransactionVersion = TransactionVersion.fromFeatureLevel(transactionVersion) - assertThrows(classOf[IllegalStateException], () => testRotateProducerIdInOngoingState(PrepareCommit, clientTransactionVersion)) + assertThrows(classOf[IllegalStateException], () => testRotateProducerIdInOngoingState(TransactionState.PREPARE_COMMIT, clientTransactionVersion)) } @Test @@ -613,7 +617,7 @@ class TransactionMetadataTest { producerEpoch = RecordBatch.NO_PRODUCER_EPOCH, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -637,7 +641,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -662,7 +666,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = lastProducerEpoch, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -687,7 +691,7 @@ class TransactionMetadataTest { producerEpoch = producerEpoch, lastProducerEpoch = lastProducerEpoch, txnTimeoutMs = 30000, - state = Empty, + state = TransactionState.EMPTY, topicPartitions = mutable.Set.empty, txnLastUpdateTimestamp = time.milliseconds(), clientTransactionVersion = TV_0) @@ -699,13 +703,13 @@ class TransactionMetadataTest { @Test def testTransactionStateIdAndNameMapping(): Unit = { - for (state <- TransactionState.AllStates) { + for (state <- TransactionState.ALL_STATES.asScala) { assertEquals(state, TransactionState.fromId(state.id)) - assertEquals(Some(state), TransactionState.fromName(state.name)) + assertEquals(Optional.of(state), TransactionState.fromName(state.stateName)) - if (state != Dead) { - val clientTransactionState = org.apache.kafka.clients.admin.TransactionState.parse(state.name) - assertEquals(state.name, clientTransactionState.toString) + if (state != TransactionState.DEAD) { + val clientTransactionState = org.apache.kafka.clients.admin.TransactionState.parse(state.stateName) + assertEquals(state.stateName, clientTransactionState.toString) assertNotEquals(org.apache.kafka.clients.admin.TransactionState.UNKNOWN, clientTransactionState) } } @@ -714,27 +718,27 @@ class TransactionMetadataTest { @Test def testAllTransactionStatesAreMapped(): Unit = { val unmatchedStates = mutable.Set( - Empty, - Ongoing, - PrepareCommit, - PrepareAbort, - CompleteCommit, - CompleteAbort, - PrepareEpochFence, - Dead + TransactionState.EMPTY, + TransactionState.ONGOING, + TransactionState.PREPARE_COMMIT, + TransactionState.PREPARE_ABORT, + TransactionState.COMPLETE_COMMIT, + TransactionState.COMPLETE_ABORT, + TransactionState.PREPARE_EPOCH_FENCE, + TransactionState.DEAD ) // The exhaustive match is intentional here to ensure that we are // forced to update the test case if a new state is added. - TransactionState.AllStates.foreach { - case Empty => assertTrue(unmatchedStates.remove(Empty)) - case Ongoing => assertTrue(unmatchedStates.remove(Ongoing)) - case PrepareCommit => assertTrue(unmatchedStates.remove(PrepareCommit)) - case PrepareAbort => assertTrue(unmatchedStates.remove(PrepareAbort)) - case CompleteCommit => assertTrue(unmatchedStates.remove(CompleteCommit)) - case CompleteAbort => assertTrue(unmatchedStates.remove(CompleteAbort)) - case PrepareEpochFence => assertTrue(unmatchedStates.remove(PrepareEpochFence)) - case Dead => assertTrue(unmatchedStates.remove(Dead)) + TransactionState.ALL_STATES.asScala.foreach { + case TransactionState.EMPTY => assertTrue(unmatchedStates.remove(TransactionState.EMPTY)) + case TransactionState.ONGOING => assertTrue(unmatchedStates.remove(TransactionState.ONGOING)) + case TransactionState.PREPARE_COMMIT => assertTrue(unmatchedStates.remove(TransactionState.PREPARE_COMMIT)) + case TransactionState.PREPARE_ABORT => assertTrue(unmatchedStates.remove(TransactionState.PREPARE_ABORT)) + case TransactionState.COMPLETE_COMMIT => assertTrue(unmatchedStates.remove(TransactionState.COMPLETE_COMMIT)) + case TransactionState.COMPLETE_ABORT => assertTrue(unmatchedStates.remove(TransactionState.COMPLETE_ABORT)) + case TransactionState.PREPARE_EPOCH_FENCE => assertTrue(unmatchedStates.remove(TransactionState.PREPARE_EPOCH_FENCE)) + case TransactionState.DEAD => assertTrue(unmatchedStates.remove(TransactionState.DEAD)) } assertEquals(Set.empty, unmatchedStates) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index d3544f2bb1a3c..2eb0cbf630eb7 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -25,6 +25,7 @@ import kafka.server.ReplicaManager import kafka.utils.TestUtils import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.compress.Compression +import org.apache.kafka.common.errors.InvalidRegularExpression import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME import org.apache.kafka.common.metrics.{JmxReporter, KafkaMetricsContext, Metrics} import org.apache.kafka.common.protocol.{Errors, MessageUtil} @@ -32,6 +33,7 @@ import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.MockTime +import org.apache.kafka.coordinator.transaction.TransactionState import org.apache.kafka.metadata.MetadataCache import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion, RequestLocal, TransactionVersion} import org.apache.kafka.server.common.TransactionVersion.{TV_0, TV_2} @@ -180,7 +182,7 @@ class TransactionStateManagerTest { ).thenReturn(new FetchDataInfo(new LogOffsetMetadata(startOffset), fileRecordsMock)) when(replicaManager.getLogEndOffset(topicPartition)).thenReturn(Some(endOffset)) - txnMetadata1.state = PrepareCommit + txnMetadata1.state = TransactionState.PREPARE_COMMIT txnMetadata1.addPartitions(Set[TopicPartition]( new TopicPartition("topic1", 0), new TopicPartition("topic1", 1))) @@ -239,7 +241,7 @@ class TransactionStateManagerTest { ).thenReturn(new FetchDataInfo(new LogOffsetMetadata(startOffset), fileRecordsMock)) when(replicaManager.getLogEndOffset(topicPartition)).thenReturn(Some(endOffset)) - txnMetadata1.state = PrepareCommit + txnMetadata1.state = TransactionState.PREPARE_COMMIT txnMetadata1.addPartitions(Set[TopicPartition]( new TopicPartition("topic1", 0), new TopicPartition("topic1", 1))) @@ -284,7 +286,7 @@ class TransactionStateManagerTest { // generate transaction log messages for two pids traces: // pid1's transaction started with two partitions - txnMetadata1.state = Ongoing + txnMetadata1.state = TransactionState.ONGOING txnMetadata1.addPartitions(Set[TopicPartition](new TopicPartition("topic1", 0), new TopicPartition("topic1", 1))) @@ -298,12 +300,12 @@ class TransactionStateManagerTest { txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit(), TV_2)) // pid1's transaction is preparing to commit - txnMetadata1.state = PrepareCommit + txnMetadata1.state = TransactionState.PREPARE_COMMIT txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit(), TV_2)) // pid2's transaction started with three partitions - txnMetadata2.state = Ongoing + txnMetadata2.state = TransactionState.ONGOING txnMetadata2.addPartitions(Set[TopicPartition](new TopicPartition("topic3", 0), new TopicPartition("topic3", 1), new TopicPartition("topic3", 2))) @@ -311,17 +313,17 @@ class TransactionStateManagerTest { txnRecords += new SimpleRecord(txnMessageKeyBytes2, TransactionLog.valueToBytes(txnMetadata2.prepareNoTransit(), TV_2)) // pid2's transaction is preparing to abort - txnMetadata2.state = PrepareAbort + txnMetadata2.state = TransactionState.PREPARE_ABORT txnRecords += new SimpleRecord(txnMessageKeyBytes2, TransactionLog.valueToBytes(txnMetadata2.prepareNoTransit(), TV_2)) // pid2's transaction has aborted - txnMetadata2.state = CompleteAbort + txnMetadata2.state = TransactionState.COMPLETE_ABORT txnRecords += new SimpleRecord(txnMessageKeyBytes2, TransactionLog.valueToBytes(txnMetadata2.prepareNoTransit(), TV_2)) // pid2's epoch has advanced, with no ongoing transaction yet - txnMetadata2.state = Empty + txnMetadata2.state = TransactionState.EMPTY txnMetadata2.topicPartitions.clear() txnRecords += new SimpleRecord(txnMessageKeyBytes2, TransactionLog.valueToBytes(txnMetadata2.prepareNoTransit(), TV_2)) @@ -510,7 +512,7 @@ class TransactionStateManagerTest { prepareForTxnMessageAppend(Errors.UNKNOWN_TOPIC_OR_PARTITION) transactionManager.appendTransactionToLog(transactionalId1, coordinatorEpoch = 10, failedMetadata, assertCallback, _ => true, RequestLocal.withThreadConfinedCaching) assertEquals(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata1))), transactionManager.getTransactionState(transactionalId1)) - assertEquals(Some(Ongoing), txnMetadata1.pendingState) + assertEquals(Some(TransactionState.ONGOING), txnMetadata1.pendingState) } @Test @@ -567,7 +569,8 @@ class TransactionStateManagerTest { val listResponse = transactionManager.listTransactionStates( filterProducerIds = Set.empty, filterStateNames = Set.empty, - -1L + -1L, + null ) assertEquals(Errors.COORDINATOR_LOAD_IN_PROGRESS, Errors.forCode(listResponse.errorCode)) } @@ -589,42 +592,46 @@ class TransactionStateManagerTest { } } - putTransaction(transactionalId = "t0", producerId = 0, state = Ongoing) - putTransaction(transactionalId = "t1", producerId = 1, state = Ongoing) + putTransaction(transactionalId = "t0", producerId = 0, state = TransactionState.ONGOING) + putTransaction(transactionalId = "t1", producerId = 1, state = TransactionState.ONGOING) + putTransaction(transactionalId = "my-special-0", producerId = 0, state = TransactionState.ONGOING) // update time to create transactions with various durations time.sleep(1000) - putTransaction(transactionalId = "t2", producerId = 2, state = PrepareCommit) - putTransaction(transactionalId = "t3", producerId = 3, state = PrepareAbort) + putTransaction(transactionalId = "t2", producerId = 2, state = TransactionState.PREPARE_COMMIT) + putTransaction(transactionalId = "t3", producerId = 3, state = TransactionState.PREPARE_ABORT) + putTransaction(transactionalId = "your-special-1", producerId = 0, state = TransactionState.PREPARE_ABORT) time.sleep(1000) - putTransaction(transactionalId = "t4", producerId = 4, state = CompleteCommit) - putTransaction(transactionalId = "t5", producerId = 5, state = CompleteAbort) - putTransaction(transactionalId = "t6", producerId = 6, state = CompleteAbort) - putTransaction(transactionalId = "t7", producerId = 7, state = PrepareEpochFence) + putTransaction(transactionalId = "t4", producerId = 4, state = TransactionState.COMPLETE_COMMIT) + putTransaction(transactionalId = "t5", producerId = 5, state = TransactionState.COMPLETE_ABORT) + putTransaction(transactionalId = "t6", producerId = 6, state = TransactionState.COMPLETE_ABORT) + putTransaction(transactionalId = "t7", producerId = 7, state = TransactionState.PREPARE_EPOCH_FENCE) + putTransaction(transactionalId = "their-special-2", producerId = 7, state = TransactionState.COMPLETE_ABORT) time.sleep(1000) - // Note that `Dead` transactions are never returned. This is a transient state + // Note that `TransactionState.DEAD` transactions are never returned. This is a transient state // which is used when the transaction state is in the process of being deleted // (whether though expiration or coordinator unloading). - putTransaction(transactionalId = "t8", producerId = 8, state = Dead) + putTransaction(transactionalId = "t8", producerId = 8, state = TransactionState.DEAD) def assertListTransactions( expectedTransactionalIds: Set[String], filterProducerIds: Set[Long] = Set.empty, filterStates: Set[String] = Set.empty, - filterDuration: Long = -1L + filterDuration: Long = -1L, + filteredTransactionalIdPattern: String = null ): Unit = { - val listResponse = transactionManager.listTransactionStates(filterProducerIds, filterStates, filterDuration) + val listResponse = transactionManager.listTransactionStates(filterProducerIds, filterStates, filterDuration, filteredTransactionalIdPattern) assertEquals(Errors.NONE, Errors.forCode(listResponse.errorCode)) assertEquals(expectedTransactionalIds, listResponse.transactionStates.asScala.map(_.transactionalId).toSet) val expectedUnknownStates = filterStates.filter(state => TransactionState.fromName(state).isEmpty) assertEquals(expectedUnknownStates, listResponse.unknownStateFilters.asScala.toSet) } - assertListTransactions(Set("t0", "t1", "t2", "t3", "t4", "t5", "t6", "t7")) - assertListTransactions(Set("t0", "t1", "t2", "t3", "t4", "t5", "t6", "t7"), filterDuration = 0L) - assertListTransactions(Set("t0", "t1", "t2", "t3"), filterDuration = 1000L) - assertListTransactions(Set("t0", "t1"), filterDuration = 2000L) + assertListTransactions(Set("t0", "t1", "t2", "t3", "t4", "t5", "t6", "t7", "my-special-0", "your-special-1", "their-special-2")) + assertListTransactions(Set("t0", "t1", "t2", "t3", "t4", "t5", "t6", "t7", "my-special-0", "your-special-1", "their-special-2"), filterDuration = 0L) + assertListTransactions(Set("t0", "t1", "t2", "t3", "my-special-0", "your-special-1"), filterDuration = 1000L) + assertListTransactions(Set("t0", "t1", "my-special-0"), filterDuration = 2000L) assertListTransactions(Set(), filterDuration = 3000L) - assertListTransactions(Set("t0", "t1"), filterStates = Set("Ongoing")) - assertListTransactions(Set("t0", "t1"), filterStates = Set("Ongoing", "UnknownState")) + assertListTransactions(Set("t0", "t1", "my-special-0"), filterStates = Set("Ongoing")) + assertListTransactions(Set("t0", "t1", "my-special-0"), filterStates = Set("Ongoing", "UnknownState")) assertListTransactions(Set("t2", "t4"), filterStates = Set("PrepareCommit", "CompleteCommit")) assertListTransactions(Set(), filterStates = Set("UnknownState")) assertListTransactions(Set("t5"), filterProducerIds = Set(5L)) @@ -634,6 +641,15 @@ class TransactionStateManagerTest { assertListTransactions(Set(), filterProducerIds = Set(3L, 6L), filterStates = Set("UnknownState")) assertListTransactions(Set(), filterProducerIds = Set(10L), filterStates = Set("CompleteCommit")) assertListTransactions(Set(), filterStates = Set("Dead")) + assertListTransactions(Set("my-special-0", "your-special-1", "their-special-2"), filteredTransactionalIdPattern = ".*special-.*") + assertListTransactions(Set(), filteredTransactionalIdPattern = "nothing") + assertListTransactions(Set("my-special-0", "your-special-1"), filterDuration = 1000L, filteredTransactionalIdPattern = ".*special-.*") + assertListTransactions(Set("their-special-2"), filterProducerIds = Set(7L), filterStates = Set("CompleteCommit", "CompleteAbort"), filteredTransactionalIdPattern = ".*special-.*") + } + + @Test + def testListTransactionsFilteringWithInvalidPattern(): Unit = { + assertThrows(classOf[InvalidRegularExpression], () => transactionManager.listTransactionStates(Set.empty, Set.empty, -1L, "(ab(cd")) } @Test @@ -642,12 +658,12 @@ class TransactionStateManagerTest { transactionManager.addLoadedTransactionsToCache(partitionId, 0, new ConcurrentHashMap[String, TransactionMetadata]()) } - transactionManager.putTransactionStateIfNotExists(transactionMetadata("ongoing", producerId = 0, state = Ongoing)) - transactionManager.putTransactionStateIfNotExists(transactionMetadata("not-expiring", producerId = 1, state = Ongoing, txnTimeout = 10000)) - transactionManager.putTransactionStateIfNotExists(transactionMetadata("prepare-commit", producerId = 2, state = PrepareCommit)) - transactionManager.putTransactionStateIfNotExists(transactionMetadata("prepare-abort", producerId = 3, state = PrepareAbort)) - transactionManager.putTransactionStateIfNotExists(transactionMetadata("complete-commit", producerId = 4, state = CompleteCommit)) - transactionManager.putTransactionStateIfNotExists(transactionMetadata("complete-abort", producerId = 5, state = CompleteAbort)) + transactionManager.putTransactionStateIfNotExists(transactionMetadata("ongoing", producerId = 0, state = TransactionState.ONGOING)) + transactionManager.putTransactionStateIfNotExists(transactionMetadata("not-expiring", producerId = 1, state = TransactionState.ONGOING, txnTimeout = 10000)) + transactionManager.putTransactionStateIfNotExists(transactionMetadata("prepare-commit", producerId = 2, state = TransactionState.PREPARE_COMMIT)) + transactionManager.putTransactionStateIfNotExists(transactionMetadata("prepare-abort", producerId = 3, state = TransactionState.PREPARE_ABORT)) + transactionManager.putTransactionStateIfNotExists(transactionMetadata("complete-commit", producerId = 4, state = TransactionState.COMPLETE_COMMIT)) + transactionManager.putTransactionStateIfNotExists(transactionMetadata("complete-abort", producerId = 5, state = TransactionState.COMPLETE_ABORT)) time.sleep(2000) val expiring = transactionManager.timedOutTransactions() @@ -656,59 +672,59 @@ class TransactionStateManagerTest { @Test def shouldWriteTxnMarkersForTransactionInPreparedCommitState(): Unit = { - verifyWritesTxnMarkersInPrepareState(PrepareCommit) + verifyWritesTxnMarkersInPrepareState(TransactionState.PREPARE_COMMIT) } @Test def shouldWriteTxnMarkersForTransactionInPreparedAbortState(): Unit = { - verifyWritesTxnMarkersInPrepareState(PrepareAbort) + verifyWritesTxnMarkersInPrepareState(TransactionState.PREPARE_ABORT) } @Test def shouldRemoveCompleteCommitExpiredTransactionalIds(): Unit = { - setupAndRunTransactionalIdExpiration(Errors.NONE, CompleteCommit) + setupAndRunTransactionalIdExpiration(Errors.NONE, TransactionState.COMPLETE_COMMIT) verifyMetadataDoesntExist(transactionalId1) verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldRemoveCompleteAbortExpiredTransactionalIds(): Unit = { - setupAndRunTransactionalIdExpiration(Errors.NONE, CompleteAbort) + setupAndRunTransactionalIdExpiration(Errors.NONE, TransactionState.COMPLETE_ABORT) verifyMetadataDoesntExist(transactionalId1) verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldRemoveEmptyExpiredTransactionalIds(): Unit = { - setupAndRunTransactionalIdExpiration(Errors.NONE, Empty) + setupAndRunTransactionalIdExpiration(Errors.NONE, TransactionState.EMPTY) verifyMetadataDoesntExist(transactionalId1) verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldNotRemoveExpiredTransactionalIdsIfLogAppendFails(): Unit = { - setupAndRunTransactionalIdExpiration(Errors.NOT_ENOUGH_REPLICAS, CompleteAbort) + setupAndRunTransactionalIdExpiration(Errors.NOT_ENOUGH_REPLICAS, TransactionState.COMPLETE_ABORT) verifyMetadataDoesExistAndIsUsable(transactionalId1) verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldNotRemoveOngoingTransactionalIds(): Unit = { - setupAndRunTransactionalIdExpiration(Errors.NONE, Ongoing) + setupAndRunTransactionalIdExpiration(Errors.NONE, TransactionState.ONGOING) verifyMetadataDoesExistAndIsUsable(transactionalId1) verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldNotRemovePrepareAbortTransactionalIds(): Unit = { - setupAndRunTransactionalIdExpiration(Errors.NONE, PrepareAbort) + setupAndRunTransactionalIdExpiration(Errors.NONE, TransactionState.PREPARE_ABORT) verifyMetadataDoesExistAndIsUsable(transactionalId1) verifyMetadataDoesExistAndIsUsable(transactionalId2) } @Test def shouldNotRemovePrepareCommitTransactionalIds(): Unit = { - setupAndRunTransactionalIdExpiration(Errors.NONE, PrepareCommit) + setupAndRunTransactionalIdExpiration(Errors.NONE, TransactionState.PREPARE_COMMIT) verifyMetadataDoesExistAndIsUsable(transactionalId1) verifyMetadataDoesExistAndIsUsable(transactionalId2) } @@ -864,7 +880,7 @@ class TransactionStateManagerTest { // will be expired and it should succeed. val timestamp = time.milliseconds() val txnMetadata = new TransactionMetadata(transactionalId, 1, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, - RecordBatch.NO_PRODUCER_EPOCH, transactionTimeoutMs, Empty, collection.mutable.Set.empty[TopicPartition], timestamp, timestamp, TV_0) + RecordBatch.NO_PRODUCER_EPOCH, transactionTimeoutMs, TransactionState.EMPTY, collection.mutable.Set.empty[TopicPartition], timestamp, timestamp, TV_0) transactionManager.putTransactionStateIfNotExists(txnMetadata) time.sleep(txnConfig.transactionalIdExpirationMs + 1) @@ -931,7 +947,7 @@ class TransactionStateManagerTest { } private def listExpirableTransactionalIds(): Set[String] = { - val activeTransactionalIds = transactionManager.listTransactionStates(Set.empty, Set.empty, -1L) + val activeTransactionalIds = transactionManager.listTransactionStates(Set.empty, Set.empty, -1L, null) .transactionStates .asScala .map(_.transactionalId) @@ -951,7 +967,7 @@ class TransactionStateManagerTest { @Test def testSuccessfulReimmigration(): Unit = { - txnMetadata1.state = PrepareCommit + txnMetadata1.state = TransactionState.PREPARE_COMMIT txnMetadata1.addPartitions(Set[TopicPartition](new TopicPartition("topic1", 0), new TopicPartition("topic1", 1))) @@ -1018,9 +1034,9 @@ class TransactionStateManagerTest { @Test def testLoadTransactionMetadataContainingSegmentEndingWithEmptyBatch(): Unit = { // Simulate a case where a log contains two segments and the first segment ending with an empty batch. - txnMetadata1.state = PrepareCommit + txnMetadata1.state = TransactionState.PREPARE_COMMIT txnMetadata1.addPartitions(Set[TopicPartition](new TopicPartition("topic1", 0))) - txnMetadata2.state = Ongoing + txnMetadata2.state = TransactionState.ONGOING txnMetadata2.addPartitions(Set[TopicPartition](new TopicPartition("topic2", 0))) // Create the first segment which contains two batches. @@ -1161,7 +1177,7 @@ class TransactionStateManagerTest { transactionManager.removeExpiredTransactionalIds() val stateAllowsExpiration = txnState match { - case Empty | CompleteCommit | CompleteAbort => true + case TransactionState.EMPTY | TransactionState.COMPLETE_COMMIT | TransactionState.COMPLETE_ABORT => true case _ => false } @@ -1208,7 +1224,7 @@ class TransactionStateManagerTest { private def transactionMetadata(transactionalId: String, producerId: Long, - state: TransactionState = Empty, + state: TransactionState = TransactionState.EMPTY, txnTimeout: Int = transactionTimeoutMs): TransactionMetadata = { val timestamp = time.milliseconds() new TransactionMetadata(transactionalId, producerId, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_ID, 0.toShort, @@ -1285,7 +1301,7 @@ class TransactionStateManagerTest { assertEquals(Double.NaN, partitionLoadTime("partition-load-time-avg"), 0) assertTrue(reporter.containsMbean(mBeanName)) - txnMetadata1.state = Ongoing + txnMetadata1.state = TransactionState.ONGOING txnMetadata1.addPartitions(Set[TopicPartition](new TopicPartition("topic1", 1), new TopicPartition("topic1", 1))) @@ -1304,7 +1320,7 @@ class TransactionStateManagerTest { @Test def testIgnoreUnknownRecordType(): Unit = { - txnMetadata1.state = PrepareCommit + txnMetadata1.state = TransactionState.PREPARE_COMMIT txnMetadata1.addPartitions(Set[TopicPartition](new TopicPartition("topic1", 0), new TopicPartition("topic1", 1))) diff --git a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala deleted file mode 100644 index 4bfb5a7104b00..0000000000000 --- a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala +++ /dev/null @@ -1,156 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * 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 - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.integration - -import java.util.Properties -import kafka.server.KafkaConfig -import kafka.utils.{Logging, TestUtils} - -import scala.jdk.CollectionConverters._ -import org.junit.jupiter.api.{BeforeEach, TestInfo} -import com.yammer.metrics.core.Gauge -import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs} -import org.apache.kafka.server.metrics.KafkaYammerMetrics -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource - -class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with Logging { - - private val nodesNum = 3 - private val topicName = "topic" - private val topicNum = 2 - private val replicationFactor = 3 - private val partitionNum = 3 - private val createDeleteIterations = 3 - - private val overridingProps = new Properties - overridingProps.put(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "true") - overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false") - // speed up the test for UnderReplicatedPartitions, which relies on the ISR expiry thread to execute concurrently with topic creation - // But the replica.lag.time.max.ms value still need to consider the slow Jenkins testing environment - overridingProps.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, "4000") - - private val testedMetrics = List("OfflinePartitionsCount","PreferredReplicaImbalanceCount","UnderReplicatedPartitions") - private val topics = List.tabulate(topicNum) (n => topicName + n) - - @volatile private var running = true - - override def generateConfigs = TestUtils.createBrokerConfigs(nodesNum) - .map(KafkaConfig.fromProps(_, overridingProps)) - - @BeforeEach - override def setUp(testInfo: TestInfo): Unit = { - // Do some Metrics Registry cleanup by removing the metrics that this test checks. - // This is a test workaround to the issue that prior harness runs may have left a populated registry. - // see https://issues.apache.org/jira/browse/KAFKA-4605 - for (m <- testedMetrics) { - val metricName = KafkaYammerMetrics.defaultRegistry.allMetrics.asScala.keys.find(_.getName.endsWith(m)) - metricName.foreach(KafkaYammerMetrics.defaultRegistry.removeMetric) - } - - super.setUp(testInfo) - } - - /* - * checking all metrics we care in a single test is faster though it would be more elegant to have 3 @Test methods - */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testMetricsDuringTopicCreateDelete(quorum: String): Unit = { - - // For UnderReplicatedPartitions, because of https://issues.apache.org/jira/browse/KAFKA-4605 - // we can't access the metrics value of each server. So instead we directly invoke the method - // replicaManager.underReplicatedPartitionCount() that defines the metrics value. - @volatile var underReplicatedPartitionCount = 0 - - // For OfflinePartitionsCount and PreferredReplicaImbalanceCount even with https://issues.apache.org/jira/browse/KAFKA-4605 - // the test has worked reliably because the metric that gets triggered is the one generated by the first started server (controller) - val offlinePartitionsCountGauge = getGauge("OfflinePartitionsCount") - @volatile var offlinePartitionsCount = offlinePartitionsCountGauge.value - assert(offlinePartitionsCount == 0) - - val preferredReplicaImbalanceCountGauge = getGauge("PreferredReplicaImbalanceCount") - @volatile var preferredReplicaImbalanceCount = preferredReplicaImbalanceCountGauge.value - assert(preferredReplicaImbalanceCount == 0) - - // Thread checking the metric continuously - running = true - val thread = new Thread(() => { - while (running) { - for (s <- servers if running) { - underReplicatedPartitionCount = s.replicaManager.underReplicatedPartitionCount - if (underReplicatedPartitionCount > 0) { - running = false - } - } - - preferredReplicaImbalanceCount = preferredReplicaImbalanceCountGauge.value - if (preferredReplicaImbalanceCount > 0) { - running = false - } - - offlinePartitionsCount = offlinePartitionsCountGauge.value - if (offlinePartitionsCount > 0) { - running = false - } - } - }) - thread.start() - - // breakable loop that creates and deletes topics - createDeleteTopics() - - // if the thread checking the gauge is still run, stop it - running = false - thread.join() - - assert(offlinePartitionsCount==0, s"Expect offlinePartitionsCount to be 0, but got: $offlinePartitionsCount") - assert(preferredReplicaImbalanceCount==0, s"Expect PreferredReplicaImbalanceCount to be 0, but got: $preferredReplicaImbalanceCount") - assert(underReplicatedPartitionCount==0, s"Expect UnderReplicatedPartitionCount to be 0, but got: $underReplicatedPartitionCount") - } - - private def getGauge(metricName: String) = { - KafkaYammerMetrics.defaultRegistry.allMetrics.asScala - .find { case (k, _) => k.getName.endsWith(metricName) } - .getOrElse(throw new AssertionError( "Unable to find metric " + metricName)) - ._2.asInstanceOf[Gauge[Int]] - } - - private def createDeleteTopics(): Unit = { - for (l <- 1 to createDeleteIterations if running) { - // Create topics - for (t <- topics if running) { - try { - createTopic(t, partitionNum, replicationFactor) - } catch { - case e: Exception => e.printStackTrace() - } - } - - // Delete topics - for (t <- topics if running) { - try { - deleteTopic(t) - TestUtils.verifyTopicDeletion(t, partitionNum, servers) - } catch { - case e: Exception => e.printStackTrace() - } - } - } - } -} diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index b1e161b975340..acf21e69ec377 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -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(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentSize: java.lang.Integer) + props.put(TopicConfig.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) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 8445baa7719fb..f93d703f07777 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -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(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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) @@ -370,7 +370,7 @@ class LogCleanerManagerTest extends Logging { // change cleanup policy from delete to compact val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, log.config.segmentSize(): Integer) + logProps.put(TopicConfig.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) @@ -548,7 +548,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsForNone(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -570,7 +570,7 @@ class LogCleanerManagerTest extends Logging { @Test def testCleanableOffsetsActiveSegment(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -592,7 +592,7 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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)) @@ -625,7 +625,7 @@ class LogCleanerManagerTest extends Logging { def testCleanableOffsetsForShortTime(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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)) @@ -667,7 +667,7 @@ class LogCleanerManagerTest extends Logging { def testUndecidedTransactionalDataNotCleanable(): Unit = { val compactionLag = 60 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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)) @@ -711,7 +711,7 @@ class LogCleanerManagerTest extends Logging { @Test def testDoneCleaning(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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) @@ -830,7 +830,7 @@ class LogCleanerManagerTest extends Logging { private def createLowRetentionLogConfig(segmentSize: Int, cleanupPolicy: String): LogConfig = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentSize: Integer) + logProps.put(TopicConfig.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 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 6c3688f497794..09a9d1c40f140 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -26,7 +26,6 @@ import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanedTransactionMetadata, Cleaner, CleanerConfig, CleanerStats, LocalLog, LogAppendInfo, LogCleaner, LogCleanerManager, LogCleaningAbortedException, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetMap, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} @@ -57,7 +56,7 @@ class LogCleanerTest extends Logging { val tmpdir = TestUtils.tempDir() val dir = TestUtils.randomPartitionLogDir(tmpdir) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.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 = new LogConfig(logProps) @@ -149,7 +148,7 @@ class LogCleanerTest extends Logging { def testCleanSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -182,7 +181,7 @@ class LogCleanerTest extends Logging { // Construct a log instance. The replaceSegments() method of the log instance is overridden so that // it waits for another thread to execute deleteOldSegments() val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024 : java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) @@ -272,7 +271,7 @@ class LogCleanerTest extends Logging { val originalMaxFileSize = 1024 val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact": java.lang.String) logProps.put(TopicConfig.PREALLOCATE_CONFIG, "true": java.lang.String) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -296,7 +295,7 @@ class LogCleanerTest extends Logging { def testDuplicateCheckAfterCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) var log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -465,7 +464,7 @@ class LogCleanerTest extends Logging { def testBasicTransactionAwareCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -498,7 +497,7 @@ class LogCleanerTest extends Logging { def testCleanWithTransactionsSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -544,7 +543,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -592,7 +591,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(capacity = Int.MaxValue, maxMessageSize = 100) val logProps = new Properties() logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, 100: java.lang.Integer) - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -620,7 +619,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -683,7 +682,7 @@ class LogCleanerTest extends Logging { def testCleanEmptyControlBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -715,7 +714,7 @@ class LogCleanerTest extends Logging { def testCommittedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -737,7 +736,7 @@ class LogCleanerTest extends Logging { def testAbortedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -766,7 +765,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -800,7 +799,7 @@ class LogCleanerTest extends Logging { val producerId = 1L val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, 0, AppendOrigin.REPLICATION) @@ -833,7 +832,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -897,7 +896,7 @@ class LogCleanerTest extends Logging { // Create cleaner with very small default max message size val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -969,7 +968,7 @@ class LogCleanerTest extends Logging { def createLogWithMessagesLargerThanMaxSize(largeMessageSize: Int): (UnifiedLog, FakeOffsetMap) = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -995,7 +994,7 @@ class LogCleanerTest extends Logging { def testCleaningWithDeletes(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1022,7 +1021,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 3 messages in the map val cleaner = makeCleaner(4) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1051,7 +1050,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsProducerLastSequence(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) log.appendAsLeader(record(0, 0), 0) // offset 0 @@ -1074,7 +1073,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsLastSequenceEvenIfTransactionAborted(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -1108,7 +1107,7 @@ class LogCleanerTest extends Logging { def testCleaningWithKeysConflictingWithTxnMarkerKeys(): Unit = { val cleaner = makeCleaner(10) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val leaderEpoch = 5 val producerEpoch = 0.toShort @@ -1152,7 +1151,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 1 message in the map val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1183,7 +1182,7 @@ class LogCleanerTest extends Logging { def testCleaningWithUncleanableSection(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1226,7 +1225,7 @@ class LogCleanerTest extends Logging { def testLogToClean(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1244,7 +1243,7 @@ class LogCleanerTest extends Logging { def testLogToCleanWithUncleanableSection(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1277,7 +1276,7 @@ class LogCleanerTest extends Logging { // create a log with compaction turned off so we can append unkeyed messages val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1335,7 +1334,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsWithAbort(): Unit = { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1357,7 +1356,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsRetainingLastEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1389,7 +1388,7 @@ class LogCleanerTest extends Logging { def testSegmentGrouping(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1490,7 +1489,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1542,7 +1541,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) //mimic the effect of loading an empty index file logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 400: java.lang.Integer) @@ -1667,7 +1666,7 @@ class LogCleanerTest extends Logging { def testRecoveryAfterCrash(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 10: java.lang.Integer) @@ -1798,7 +1797,7 @@ class LogCleanerTest extends Logging { def testBuildOffsetMapFakeLarge(): Unit = { val map = new FakeOffsetMap(1000) val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) @@ -1946,7 +1945,7 @@ class LogCleanerTest extends Logging { @Test def testCleaningBeyondMissingOffsets(): Unit = { val logProps = new Properties() - logProps.put(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) val cleaner = makeCleaner(Int.MaxValue) diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 8c49978e6bd9f..0da8366f443ee 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -60,7 +60,7 @@ class LogConcurrencyTest { @Test def testUncommittedDataNotConsumedFrequentSegmentRolls(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 237: Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 237: Integer) val logConfig = new LogConfig(logProps) testUncommittedDataNotConsumed(createLog(logConfig)) } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 1ef21ad81f407..8e417a695ee36 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -245,7 +245,7 @@ class LogLoaderTest { @Test def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "640") + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "640") val logConfig = new LogConfig(logProps) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 04f4acca5debd..67880e0ced54c 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -60,7 +60,7 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 60 * 1000 val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) val logConfig = new LogConfig(logProps) @@ -391,7 +391,7 @@ class LogManagerTest { logManager.shutdown() val segmentBytes = 10 * setSize val properties = new Properties() - properties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes.toString) + properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes.toString) properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString) val configRepository = MockConfigRepository.forTopic(name, properties) diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index 0ff68988d76fb..04c91741037bb 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -75,7 +75,7 @@ object LogTestUtils { remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs: java.lang.Long) logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 42c813074aa9a..3f73f8f731a71 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -2734,7 +2734,7 @@ class UnifiedLogTest { @Test def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { val logProps = new Properties() - logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000") + logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000") logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") val logConfig = new LogConfig(logProps) diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index 78f5335caf42d..8e2698b0842cf 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AlterReplicaLogDirsRequest, AlterReplicaLogDirsResponse} import org.apache.kafka.server.config.ServerLogConfigs -import org.apache.kafka.storage.internals.log.{LogConfig, LogFileUtils} +import org.apache.kafka.storage.internals.log.LogFileUtils import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -148,7 +148,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { // We don't want files with `.deleted` suffix are removed too fast, // so we can validate there will be orphan files and orphan files will be removed eventually. topicProperties.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, "10000") - topicProperties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1024") + topicProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1024") createTopic(topic, partitionNum, 1, topicProperties) assertEquals(logDir1, brokers.head.logManager.getLog(tp).get.dir.getParent) diff --git a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala index 45b470f402cf8..2d4c4455ad63d 100644 --- a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala @@ -114,7 +114,7 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - Some(shareCoordinator)) + shareCoordinator) val topicsCollection = new CreateTopicsRequestData.CreatableTopicCollection topicsCollection.add(getNewTopic(topicName, numPartitions, replicationFactor)) @@ -217,7 +217,7 @@ class AutoTopicCreationManagerTest { @Test def testCreateStreamsInternalTopics(): Unit = { val topicConfig = new CreatableTopicConfigCollection() - topicConfig.add(new CreatableTopicConfig().setName("cleanup.policy").setValue("compact")); + topicConfig.add(new CreatableTopicConfig().setName("cleanup.policy").setValue("compact")) val topics = Map( "stream-topic-1" -> new CreatableTopic().setName("stream-topic-1").setNumPartitions(3).setReplicationFactor(2).setConfigs(topicConfig), @@ -230,7 +230,7 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - Some(shareCoordinator)) + shareCoordinator) autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) @@ -251,7 +251,7 @@ class AutoTopicCreationManagerTest { .build(ApiKeys.CREATE_TOPICS.latestVersion()) val forwardedRequestBuffer = capturedRequest.requestData().duplicate() - assertEquals(requestHeader, RequestHeader.parse(forwardedRequestBuffer)); + assertEquals(requestHeader, RequestHeader.parse(forwardedRequestBuffer)) assertEquals(requestBody.data(), CreateTopicsRequest.parse(new ByteBufferAccessor(forwardedRequestBuffer), ApiKeys.CREATE_TOPICS.latestVersion()).data()) } @@ -266,7 +266,7 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - Some(shareCoordinator)) + shareCoordinator) autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) @@ -287,9 +287,9 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - Some(shareCoordinator)) + shareCoordinator) - autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext); + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) val argumentCaptor = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]) Mockito.verify(brokerToController).sendRequest( @@ -307,7 +307,7 @@ class AutoTopicCreationManagerTest { .setTimeoutMs(requestTimeout)) .build(ApiKeys.CREATE_TOPICS.latestVersion()) val forwardedRequestBuffer = capturedRequest.requestData().duplicate() - assertEquals(requestHeader, RequestHeader.parse(forwardedRequestBuffer)); + assertEquals(requestHeader, RequestHeader.parse(forwardedRequestBuffer)) assertEquals(requestBody.data(), CreateTopicsRequest.parse(new ByteBufferAccessor(forwardedRequestBuffer), ApiKeys.CREATE_TOPICS.latestVersion()).data()) } @@ -324,9 +324,9 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - Some(shareCoordinator)) + shareCoordinator) - autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext); + autoTopicCreationManager.createStreamsInternalTopics(topics, requestContext) val argumentCaptor = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]) Mockito.verify(brokerToController).sendRequest( @@ -355,7 +355,7 @@ class AutoTopicCreationManagerTest { brokerToController, groupCoordinator, transactionCoordinator, - Some(shareCoordinator)) + shareCoordinator) val createTopicApiVersion = new ApiVersionsResponseData.ApiVersion() .setApiKey(ApiKeys.CREATE_TOPICS.id) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 8bf29020eca30..5f87b20d1f579 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -670,6 +670,16 @@ class DynamicBrokerConfigTest { assertTrue(m.currentReporters.isEmpty) } + @Test + def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = { + val props = TestUtils.createBrokerConfig(0, port = 8181) + props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, "1024") + val config = new KafkaConfig(props) + assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) + config.updateCurrentConfig(new KafkaConfig(props)) + assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) + } + @Test def testDynamicLogLocalRetentionMsConfig(): Unit = { val props = TestUtils.createBrokerConfig(0, port = 8181) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index cb3a40b9f89cc..28c8d694f9d51 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -106,14 +106,14 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { TestUtils.retry(10000) { val logOpt = this.brokers.head.logManager.getLog(tp) assertTrue(logOpt.isDefined) - assertEquals(oldSegmentSize, logOpt.get.config.segmentSize()) + assertEquals(oldSegmentSize, logOpt.get.config.segmentSize) } val newSegmentSize = 2000 val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), + val op = new AlterConfigOp(new ConfigEntry(TopicConfig.SEGMENT_BYTES_CONFIG, newSegmentSize.toString), OpType.SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get } finally { @@ -121,7 +121,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } val log = brokers.head.logManager.getLog(tp).get TestUtils.retry(10000) { - assertEquals(newSegmentSize, log.config.segmentSize()) + assertEquals(newSegmentSize, log.config.segmentSize) } (1 to 50).foreach(i => TestUtils.produceMessage(brokers, tp.topic, i.toString)) diff --git a/core/src/test/scala/unit/kafka/server/ForwardingManagerMetricsTest.scala b/core/src/test/scala/unit/kafka/server/ForwardingManagerMetricsTest.scala deleted file mode 100644 index 2c10decb3ed5c..0000000000000 --- a/core/src/test/scala/unit/kafka/server/ForwardingManagerMetricsTest.scala +++ /dev/null @@ -1,114 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * 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 - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package unit.kafka.server - -import kafka.server.ForwardingManagerMetrics -import org.apache.kafka.common.MetricName -import org.apache.kafka.common.metrics.Metrics -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} -import org.junit.jupiter.api.Test - -import java.util.Collections -import scala.jdk.CollectionConverters._ - -final class ForwardingManagerMetricsTest { - @Test - def testMetricsNames(): Unit = { - val metrics = new Metrics() - val expectedGroup = "ForwardingManager" - - val expectedMetrics = Set( - new MetricName("QueueTimeMs.p99", expectedGroup, "", Collections.emptyMap()), - new MetricName("QueueTimeMs.p999", expectedGroup, "", Collections.emptyMap()), - new MetricName("QueueLength", expectedGroup, "", Collections.emptyMap()), - new MetricName("RemoteTimeMs.p99", expectedGroup, "", Collections.emptyMap()), - new MetricName("RemoteTimeMs.p999", expectedGroup, "", Collections.emptyMap()) - ) - - var metricsMap = metrics.metrics().asScala.filter { case (name, _) => name.group == expectedGroup } - assertEquals(0, metricsMap.size) - - ForwardingManagerMetrics(metrics, 1000) - metricsMap = metrics.metrics().asScala.filter { case (name, _) => name.group == expectedGroup } - assertEquals(metricsMap.size, expectedMetrics.size) - metricsMap.foreach { case (name, _) => - assertTrue(expectedMetrics.contains(name)) - } - } - - @Test - def testQueueTimeMs(): Unit = { - val metrics = new Metrics() - - val forwardingManagerMetrics = ForwardingManagerMetrics(metrics, 1000) - val queueTimeMsP99 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist.latencyP99Name) - val queueTimeMsP999 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist.latencyP999Name) - assertEquals(Double.NaN, queueTimeMsP99.metricValue.asInstanceOf[Double]) - assertEquals(Double.NaN, queueTimeMsP999.metricValue.asInstanceOf[Double]) - for(i <- 0 to 999) { - forwardingManagerMetrics.queueTimeMsHist.record(i) - } - assertEquals(990.0, queueTimeMsP99.metricValue.asInstanceOf[Double]) - assertEquals(999.0, queueTimeMsP999.metricValue.asInstanceOf[Double]) - } - - @Test - def testQueueLength(): Unit = { - val metrics = new Metrics() - - val forwardingManagerMetrics = ForwardingManagerMetrics(metrics, 1000) - val queueLength = metrics.metrics().get(forwardingManagerMetrics.queueLengthName) - assertEquals(0, queueLength.metricValue.asInstanceOf[Int]) - forwardingManagerMetrics.queueLength.getAndIncrement() - assertEquals(1, queueLength.metricValue.asInstanceOf[Int]) - } - - @Test - def testRemoteTimeMs(): Unit = { - val metrics = new Metrics() - - val forwardingManagerMetrics = ForwardingManagerMetrics(metrics, 1000) - val remoteTimeMsP99 = metrics.metrics().get(forwardingManagerMetrics.remoteTimeMsHist.latencyP99Name) - val remoteTimeMsP999 = metrics.metrics().get(forwardingManagerMetrics.remoteTimeMsHist.latencyP999Name) - assertEquals(Double.NaN, remoteTimeMsP99.metricValue.asInstanceOf[Double]) - assertEquals(Double.NaN, remoteTimeMsP999.metricValue.asInstanceOf[Double]) - for (i <- 0 to 999) { - forwardingManagerMetrics.remoteTimeMsHist.record(i) - } - assertEquals(990.0, remoteTimeMsP99.metricValue.asInstanceOf[Double]) - assertEquals(999.0, remoteTimeMsP999.metricValue.asInstanceOf[Double]) - } - - @Test - def testTimeoutMs(): Unit = { - val metrics = new Metrics() - val timeoutMs = 500 - val forwardingManagerMetrics = ForwardingManagerMetrics(metrics, timeoutMs) - val queueTimeMsP99 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist.latencyP99Name) - val queueTimeMsP999 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist.latencyP999Name) - assertEquals(Double.NaN, queueTimeMsP99.metricValue.asInstanceOf[Double]) - assertEquals(Double.NaN, queueTimeMsP999.metricValue.asInstanceOf[Double]) - for(i <- 0 to 99) { - forwardingManagerMetrics.queueTimeMsHist.record(i) - } - forwardingManagerMetrics.queueTimeMsHist.record(1000) - - assertEquals(99, queueTimeMsP99.metricValue.asInstanceOf[Double]) - assertEquals(timeoutMs * 0.999, queueTimeMsP999.metricValue.asInstanceOf[Double]) - } -} diff --git a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala index d2d8d3e0382c3..16e4b2bcb66f4 100644 --- a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala @@ -53,7 +53,7 @@ class ForwardingManagerTest { private val forwardingManager = new ForwardingManagerImpl(brokerToController, metrics) private val principalBuilder = new DefaultKafkaPrincipalBuilder(null, null) private val queueTimeMsP999 = metrics.metrics().get(forwardingManager.forwardingManagerMetrics.queueTimeMsHist.latencyP999Name) - private val queueLength = metrics.metrics().get(forwardingManager.forwardingManagerMetrics.queueLengthName) + private val queueLength = metrics.metrics().get(forwardingManager.forwardingManagerMetrics.queueLengthName()) private val remoteTimeMsP999 = metrics.metrics().get(forwardingManager.forwardingManagerMetrics.remoteTimeMsHist.latencyP999Name) private def controllerApiVersions: NodeApiVersions = { diff --git a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala index df939c29ffb45..f7675651cd6dd 100644 --- a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala @@ -33,9 +33,11 @@ import org.apache.kafka.common.utils.ProducerIdAndEpoch import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT import org.junit.jupiter.api.Assertions.{assertEquals, fail} +import java.net.Socket import java.util.{Comparator, Properties} import java.util.stream.Collectors import scala.collection.Seq +import scala.collection.mutable.ListBuffer import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag @@ -46,6 +48,8 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { protected var producer: KafkaProducer[String, String] = _ + protected var openSockets: ListBuffer[Socket] = ListBuffer[Socket]() + protected def createOffsetsTopic(): Unit = { val admin = cluster.admin() try { @@ -140,6 +144,14 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { keySerializer = new StringSerializer, valueSerializer = new StringSerializer) } + protected def closeSockets(): Unit = { + while (openSockets.nonEmpty) { + val socket = openSockets.head + socket.close() + openSockets.remove(0) + } + } + protected def closeProducer(): Unit = { if(producer != null) producer.close() @@ -901,6 +913,24 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { assertEquals(expectedResponseData.results.asScala.toSet, deleteGroupsResponse.data.results.asScala.toSet) } + protected def connectAny(): Socket = { + val socket: Socket = IntegrationTestUtils.connect( + cluster.anyBrokerSocketServer(), + cluster.clientListener() + ) + openSockets += socket + socket + } + + protected def connect(destination: Int): Socket = { + val socket: Socket = IntegrationTestUtils.connect( + brokerSocketServer(destination), + cluster.clientListener() + ) + openSockets += socket + socket + } + protected def connectAndReceive[T <: AbstractResponse]( request: AbstractRequest )(implicit classTag: ClassTag[T]): T = { diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 0d311435a2d82..84e4546ff34b1 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -88,7 +88,7 @@ import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.{ClientMetricsManager, SimpleApiVersionManager} import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authorizer} -import org.apache.kafka.server.common.{FeatureVersion, FinalizedFeatures, GroupVersion, KRaftVersion, MetadataVersion, RequestLocal, StreamsVersion, TransactionVersion} +import org.apache.kafka.server.common.{FeatureVersion, FinalizedFeatures, GroupVersion, KRaftVersion, MetadataVersion, RequestLocal, ShareVersion, StreamsVersion, TransactionVersion} import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs} import org.apache.kafka.server.metrics.ClientMetricsTestUtils import org.apache.kafka.server.share.{CachedSharePartition, ErroneousAndValidPartitionData, SharePartitionKey} @@ -192,7 +192,7 @@ class KafkaApisTest extends Logging { replicaManager = replicaManager, groupCoordinator = groupCoordinator, txnCoordinator = txnCoordinator, - shareCoordinator = Some(shareCoordinator), + shareCoordinator = shareCoordinator, autoTopicCreationManager = autoTopicCreationManager, brokerId = brokerId, config = config, @@ -225,6 +225,28 @@ class KafkaApisTest extends Logging { } } + def initializeMetadataCacheWithShareGroupsEnabled(enableShareGroups: Boolean = true): MetadataCache = { + val cache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) + val delta = new MetadataDelta(MetadataImage.EMPTY); + delta.replay(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) + ) + if (enableShareGroups) { + delta.replay(new FeatureLevelRecord() + .setName(ShareVersion.FEATURE_NAME) + .setFeatureLevel(ShareVersion.SV_1.featureLevel()) + ) + } else { + delta.replay(new FeatureLevelRecord() + .setName(ShareVersion.FEATURE_NAME) + .setFeatureLevel(ShareVersion.SV_0.featureLevel()) + ) + } + cache.setImage(delta.apply(MetadataProvenance.EMPTY)) + cache + } + @Test def testDescribeConfigsWithAuthorizer(): Unit = { val authorizer: Authorizer = mock(classOf[Authorizer]) @@ -4142,7 +4164,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4166,7 +4188,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, shareSessionEpoch), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName))) ) @@ -4186,10 +4208,7 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -4209,7 +4228,7 @@ class KafkaApisTest extends Logging { def testHandleShareFetchRequestInvalidRequestOnInitialEpoch(): Unit = { val topicName = "foo" val topicId = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4238,10 +4257,11 @@ class KafkaApisTest extends Logging { cachedSharePartitions.mustAdd(new CachedSharePartition( new TopicIdPartition(topicId, partitionIndex, topicName), false)) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenThrow( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenThrow( Errors.INVALID_REQUEST.exception() - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), - new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( + new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2 + ))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4266,10 +4286,7 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -4311,7 +4328,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4335,7 +4352,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) @@ -4365,10 +4382,7 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -4415,7 +4429,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4423,7 +4437,7 @@ class KafkaApisTest extends Logging { FutureUtils.failedFuture[util.Map[TopicIdPartition, ShareFetchResponseData.PartitionData]](Errors.UNKNOWN_SERVER_ERROR.exception()) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) @@ -4444,10 +4458,7 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -4460,7 +4471,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4491,9 +4502,10 @@ class KafkaApisTest extends Logging { cachedSharePartitions.mustAdd(new CachedSharePartition( new TopicIdPartition(topicId, partitionIndex, topicName), false)) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())) - .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), - new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())) + .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( + new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) + ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4518,10 +4530,7 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -4534,7 +4543,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4552,9 +4561,10 @@ class KafkaApisTest extends Logging { cachedSharePartitions.mustAdd(new CachedSharePartition( new TopicIdPartition(topicId, partitionIndex, topicName), false)) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())) - .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), - new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())) + .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( + new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) + ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4579,10 +4589,7 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -4595,7 +4602,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4611,7 +4618,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) @@ -4632,10 +4639,7 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -4656,7 +4660,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4678,7 +4682,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) @@ -4700,10 +4704,7 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -4745,7 +4746,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -4767,7 +4768,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) @@ -4789,10 +4790,7 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -4831,7 +4829,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -4905,14 +4903,14 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId, partitionIndex, topicName), false) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), - new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), - new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 3)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( + new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), new ShareSession( + new ShareSessionKey(groupId, memberId), cachedSharePartitions, 3)) ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( @@ -4932,10 +4930,7 @@ class KafkaApisTest extends Logging { var request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -5044,7 +5039,7 @@ class KafkaApisTest extends Logging { val topicName4 = "foo4" val topicId4 = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 2, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) addTopicToMetadataCache(topicName3, 1, topicId = topicId3) @@ -5170,17 +5165,17 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId4, 0, topicName4), false )) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId1, new TopicPartition(topicName1, 0)), new TopicIdPartition(topicId1, new TopicPartition(topicName1, 1)), new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)), new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), - new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions1, 2)) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), - new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions2, 3)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( + new ShareSessionKey(groupId, memberId), cachedSharePartitions1, 2)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), new ShareSession( + new ShareSessionKey(groupId, memberId), cachedSharePartitions2, 3)) ).thenReturn(new FinalContext()) when(sharePartitionManager.releaseSession(any(), any())).thenReturn( @@ -5254,10 +5249,7 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -5507,7 +5499,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -5595,10 +5587,7 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = kafkaApis.handleFetchFromShareFetchRequest( request, @@ -5658,7 +5647,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -5734,10 +5723,7 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = kafkaApis.handleFetchFromShareFetchRequest( request, @@ -5791,7 +5777,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -5862,10 +5848,7 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) // First share fetch request is to establish the share session with the broker. - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = kafkaApis.handleFetchFromShareFetchRequest( request, @@ -5927,7 +5910,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid() val topicId3 = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) // topicName3 is not in the metadataCache. @@ -6011,10 +5994,7 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = kafkaApis.handleFetchFromShareFetchRequest( request, @@ -6106,7 +6086,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -6150,12 +6130,13 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId, 0, topicName), false )) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), - new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( + new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) + ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -6181,10 +6162,7 @@ class KafkaApisTest extends Logging { var shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) var request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) var response = verifyNoThrottling[ShareFetchResponse](request) var responseData = response.data() @@ -6241,7 +6219,20 @@ class KafkaApisTest extends Logging { val memberId: Uuid = Uuid.randomUuid() val groupId = "group" - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = { + val cache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) + val delta = new MetadataDelta(MetadataImage.EMPTY); + delta.replay(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) + ) + delta.replay(new FeatureLevelRecord() + .setName(ShareVersion.FEATURE_NAME) + .setFeatureLevel(ShareVersion.SV_0.featureLevel()) + ) + cache.setImage(delta.apply(MetadataProvenance.EMPTY)) + cache + } val shareFetchRequestData = new ShareFetchRequestData(). setGroupId(groupId). @@ -6264,10 +6255,7 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "false"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) @@ -6281,7 +6269,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6315,8 +6303,6 @@ class KafkaApisTest extends Logging { .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Option(authorizer), ) kafkaApis.handleShareFetchRequest(request) @@ -6332,7 +6318,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6347,7 +6333,7 @@ class KafkaApisTest extends Logging { ).asJava) ) - when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenReturn( + when(sharePartitionManager.newContext(any(), any(), any(), any(), any(), any())).thenReturn( new FinalContext() ) @@ -6377,10 +6363,7 @@ class KafkaApisTest extends Logging { val shareFetchRequest = new ShareFetchRequest.Builder(shareFetchRequestData).build(ApiKeys.SHARE_FETCH.latestVersion) val request = buildRequest(shareFetchRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareFetchRequest(request) val response = verifyNoThrottling[ShareFetchResponse](request) val responseData = response.data() @@ -6402,7 +6385,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -6443,10 +6426,7 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) val responseData = response.data() @@ -6466,7 +6446,20 @@ class KafkaApisTest extends Logging { val memberId: Uuid = Uuid.randomUuid() val groupId = "group" - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = { + val cache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_1) + val delta = new MetadataDelta(MetadataImage.EMPTY); + delta.replay(new FeatureLevelRecord() + .setName(MetadataVersion.FEATURE_NAME) + .setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()) + ) + delta.replay(new FeatureLevelRecord() + .setName(ShareVersion.FEATURE_NAME) + .setFeatureLevel(ShareVersion.SV_0.featureLevel()) + ) + cache.setImage(delta.apply(MetadataProvenance.EMPTY)) + cache + } val shareAcknowledgeRequestData = new ShareAcknowledgeRequestData() .setGroupId(groupId) @@ -6489,10 +6482,7 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData).build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "false"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) @@ -6506,7 +6496,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6540,8 +6530,6 @@ class KafkaApisTest extends Logging { .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Option(authorizer), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6557,7 +6545,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6590,10 +6578,7 @@ class KafkaApisTest extends Logging { .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) @@ -6607,7 +6592,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6640,10 +6625,7 @@ class KafkaApisTest extends Logging { .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) @@ -6657,7 +6639,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val groupId: String = "group" val memberId: Uuid = Uuid.ZERO_UUID @@ -6688,10 +6670,7 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) @@ -6713,7 +6692,7 @@ class KafkaApisTest extends Logging { val partitionIndex = 0 val topicIdPartition = new TopicIdPartition(topicId, new TopicPartition(topicName, partitionIndex)) val topicPartition = topicIdPartition.topicPartition - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicPartition.topic, numPartitions = 1, numBrokers = 3, topicId) val memberId: Uuid = Uuid.ZERO_UUID @@ -6762,10 +6741,7 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) @@ -6788,7 +6764,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -6824,10 +6800,7 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) val responseData = response.data() @@ -6840,7 +6813,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -6890,10 +6863,7 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) val responseData = response.data() @@ -6912,7 +6882,7 @@ class KafkaApisTest extends Logging { val topicName = "foo" val topicId = Uuid.randomUuid() val partitionIndex = 0 - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName, 1, topicId = topicId) val memberId: Uuid = Uuid.randomUuid() @@ -6957,10 +6927,7 @@ class KafkaApisTest extends Logging { val shareAcknowledgeRequest = new ShareAcknowledgeRequest.Builder(shareAcknowledgeRequestData) .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handleShareAcknowledgeRequest(request) val response = verifyNoThrottling[ShareAcknowledgeResponse](request) val responseData = response.data() @@ -6987,7 +6954,7 @@ class KafkaApisTest extends Logging { def testGetAcknowledgeBatchesFromShareFetchRequest(): Unit = { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() val shareFetchRequestData = new ShareFetchRequestData(). setGroupId("group"). setMemberId(Uuid.randomUuid().toString). @@ -7038,10 +7005,7 @@ class KafkaApisTest extends Logging { topicNames.put(topicId2, "foo2") val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareFetchRequest(shareFetchRequest, topicNames, erroneous) assertEquals(4, acknowledgeBatches.size) @@ -7059,7 +7023,7 @@ class KafkaApisTest extends Logging { def testGetAcknowledgeBatchesFromShareFetchRequestError(): Unit = { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() val shareFetchRequestData = new ShareFetchRequestData(). setGroupId("group"). setMemberId(Uuid.randomUuid().toString). @@ -7103,10 +7067,7 @@ class KafkaApisTest extends Logging { topicIdNames.put(topicId1, "foo1") // topicId2 is not present in topicIdNames val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareFetchRequest(shareFetchRequest, topicIdNames, erroneous) val erroneousTopicIdPartitions = kafkaApis.validateAcknowledgementBatches(acknowledgeBatches, erroneous) @@ -7125,7 +7086,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() val shareAcknowledgeRequestData = new ShareAcknowledgeRequestData(). setGroupId("group"). setMemberId(Uuid.randomUuid().toString). @@ -7175,10 +7136,7 @@ class KafkaApisTest extends Logging { topicNames.put(topicId2, "foo2") val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareAcknowledgeRequest(shareAcknowledgeRequest, topicNames, erroneous) assertEquals(3, acknowledgeBatches.size) @@ -7196,7 +7154,7 @@ class KafkaApisTest extends Logging { def testGetAcknowledgeBatchesFromShareAcknowledgeRequestError(): Unit = { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() val shareAcknowledgeRequestData = new ShareAcknowledgeRequestData(). setGroupId("group"). setMemberId(Uuid.randomUuid().toString). @@ -7241,10 +7199,7 @@ class KafkaApisTest extends Logging { topicIdNames.put(topicId1, "foo1") // topicId2 not present in topicIdNames val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareAcknowledgeRequest(shareAcknowledgeRequest, topicIdNames, erroneous) val erroneousTopicIdPartitions = kafkaApis.validateAcknowledgementBatches(acknowledgeBatches, erroneous) @@ -7271,7 +7226,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid() val memberId = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -7312,10 +7267,7 @@ class KafkaApisTest extends Logging { val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val ackResult = kafkaApis.handleAcknowledgements( acknowledgementData, erroneous, @@ -7349,7 +7301,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid() val memberId = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -7390,10 +7342,7 @@ class KafkaApisTest extends Logging { val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val ackResult = kafkaApis.handleAcknowledgements( acknowledgementData, erroneous, @@ -7427,7 +7376,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid() val memberId = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() // Topic with id topicId1 is not present in Metadata Cache addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -7469,10 +7418,7 @@ class KafkaApisTest extends Logging { val erroneous = mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData]() - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val ackResult = kafkaApis.handleAcknowledgements( acknowledgementData, erroneous, @@ -7506,7 +7452,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid() val memberId = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -7542,10 +7488,7 @@ class KafkaApisTest extends Logging { erroneous += (tp3 -> ShareAcknowledgeResponse.partitionResponse(tp3, Errors.UNKNOWN_TOPIC_ID)) - kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() val ackResult = kafkaApis.handleAcknowledgements( acknowledgementData, erroneous, @@ -7577,7 +7520,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid() val topicId2 = Uuid.randomUuid() - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() val responseAcknowledgeData: mutable.Map[TopicIdPartition, ShareAcknowledgeResponseData.PartitionData] = mutable.Map() responseAcknowledgeData += (new TopicIdPartition(topicId1, new TopicPartition("foo", 0)) -> new ShareAcknowledgeResponseData.PartitionData().setPartitionIndex(0).setErrorCode(Errors.NONE.code)) @@ -7639,9 +7582,7 @@ class KafkaApisTest extends Logging { .build(ApiKeys.SHARE_ACKNOWLEDGE.latestVersion) val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( - overrideProperties = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + ) val response = kafkaApis.processShareAcknowledgeResponse(responseAcknowledgeData, request) val responseData = response.data() val topicResponses = responseData.responses() @@ -9714,7 +9655,7 @@ class KafkaApisTest extends Logging { when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), any[Long])).thenReturn(0) - when(txnCoordinator.handleListTransactions(Set.empty[Long], Set.empty[String], -1L)) + when(txnCoordinator.handleListTransactions(Set.empty[Long], Set.empty[String], -1L, null)) .thenReturn(new ListTransactionsResponseData() .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code)) kafkaApis = createKafkaApis() @@ -9728,7 +9669,7 @@ class KafkaApisTest extends Logging { @Test def testListTransactionsAuthorization(): Unit = { val authorizer: Authorizer = mock(classOf[Authorizer]) - val data = new ListTransactionsRequestData() + val data = new ListTransactionsRequestData().setTransactionalIdPattern("my.*") val listTransactionsRequest = new ListTransactionsRequest.Builder(data).build() val request = buildRequest(listTransactionsRequest) when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), @@ -9736,15 +9677,15 @@ class KafkaApisTest extends Logging { val transactionStates = new util.ArrayList[ListTransactionsResponseData.TransactionState]() transactionStates.add(new ListTransactionsResponseData.TransactionState() - .setTransactionalId("foo") + .setTransactionalId("myFoo") .setProducerId(12345L) .setTransactionState("Ongoing")) transactionStates.add(new ListTransactionsResponseData.TransactionState() - .setTransactionalId("bar") + .setTransactionalId("myBar") .setProducerId(98765) .setTransactionState("PrepareAbort")) - when(txnCoordinator.handleListTransactions(Set.empty[Long], Set.empty[String], -1L)) + when(txnCoordinator.handleListTransactions(Set.empty[Long], Set.empty[String], -1L, "my.*")) .thenReturn(new ListTransactionsResponseData() .setErrorCode(Errors.NONE.code) .setTransactionStates(transactionStates)) @@ -9755,10 +9696,10 @@ class KafkaApisTest extends Logging { Collections.singletonList(action) } - when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActions("foo")))) + when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActions("myFoo")))) .thenReturn(Seq(AuthorizationResult.ALLOWED).asJava) - when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActions("bar")))) + when(authorizer.authorize(any[RequestContext], ArgumentMatchers.eq(buildExpectedActions("myBar")))) .thenReturn(Seq(AuthorizationResult.DENIED).asJava) kafkaApis = createKafkaApis(authorizer = Some(authorizer)) kafkaApis.handleListTransactionsRequest(request) @@ -9766,7 +9707,7 @@ class KafkaApisTest extends Logging { val response = verifyNoThrottling[ListTransactionsResponse](request) assertEquals(1, response.data.transactionStates.size()) val transactionState = response.data.transactionStates.get(0) - assertEquals("foo", transactionState.transactionalId) + assertEquals("myFoo", transactionState.transactionalId) assertEquals(12345L, transactionState.producerId) assertEquals("Ongoing", transactionState.transactionState) } @@ -11037,8 +10978,10 @@ class KafkaApisTest extends Logging { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequestData().setGroupId("group") val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest).build()) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) - kafkaApis = createKafkaApis() + metadataCache = mock(classOf[KRaftMetadataCache]) + kafkaApis = createKafkaApis( + featureVersions = Seq(ShareVersion.SV_0) + ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val expectedHeartbeatResponse = new ShareGroupHeartbeatResponseData() @@ -11058,10 +11001,8 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, shareGroupHeartbeatRequest )).thenReturn(future) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) - kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val shareGroupHeartbeatResponse = new ShareGroupHeartbeatResponseData() @@ -11081,9 +11022,8 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11094,7 +11034,7 @@ class KafkaApisTest extends Logging { @Test def testShareGroupHeartbeatRequestTopicAuthorizationFailed(): Unit = { - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() val groupId = "group" val fooTopicName = "foo" val barTopicName = "bar" @@ -11123,7 +11063,6 @@ class KafkaApisTest extends Logging { } kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11143,10 +11082,8 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, shareGroupHeartbeatRequest )).thenReturn(future) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) - kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() + kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) future.completeExceptionally(Errors.FENCED_MEMBER_EPOCH.exception) @@ -11185,8 +11122,7 @@ class KafkaApisTest extends Logging { new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(1)).setMembers(List(member1).asJava), new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(2)).setMembers(List(member2).asJava) ).asJava - getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") - , true, null, describedGroups) + getShareGroupDescribeResponse(groupIds, enableShareGroups = true, verifyNoErr = true, null, describedGroups) } @Test @@ -11196,7 +11132,7 @@ class KafkaApisTest extends Logging { new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(0)), new ShareGroupDescribeResponseData.DescribedGroup().setGroupId(groupIds.get(1)) ).asJava - val response = getShareGroupDescribeResponse(groupIds, Map.empty, false, null, describedGroups) + val response = getShareGroupDescribeResponse(groupIds, enableShareGroups = false, verifyNoErr = false, null, describedGroups) assertNotNull(response.data) assertEquals(2, response.data.groups.size) response.data.groups.forEach(group => assertEquals(Errors.UNSUPPORTED_VERSION.code(), group.errorCode())) @@ -11209,8 +11145,7 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava) - val response = getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") - , false, authorizer, describedGroups) + val response = getShareGroupDescribeResponse(groupIds, enableShareGroups = true, verifyNoErr = false, authorizer, describedGroups) assertNotNull(response.data) assertEquals(2, response.data.groups.size) response.data.groups.forEach(group => assertEquals(Errors.GROUP_AUTHORIZATION_FAILED.code(), group.errorCode())) @@ -11227,8 +11162,7 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) - val response = getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") - , false, authorizer, describedGroups) + val response = getShareGroupDescribeResponse(groupIds, enableShareGroups = true, verifyNoErr = false, authorizer, describedGroups) assertNotNull(response.data) assertEquals(2, response.data.groups.size) @@ -11291,8 +11225,7 @@ class KafkaApisTest extends Logging { .setGroupId(groupIds.get(2)) .setMembers(List(member2).asJava)).asJava - val response = getShareGroupDescribeResponse(groupIds, Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true") - , false, authorizer, describedGroups) + val response = getShareGroupDescribeResponse(groupIds, enableShareGroups = true, verifyNoErr = false, authorizer, describedGroups) assertNotNull(response.data) assertEquals(3, response.data.groups.size) @@ -11338,13 +11271,9 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava - val config = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", - ) - val response = getReadShareGroupStateResponse( readRequestData, - config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, + ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, null, readStateResultData @@ -11393,13 +11322,9 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) - val config = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", - ) - val response = getReadShareGroupStateResponse( readRequestData, - config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, + ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, authorizer, readStateResultData @@ -11441,13 +11366,9 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava - val config = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", - ) - val response = getReadShareGroupStateSummaryResponse( readSummaryRequestData, - config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, + ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, null, readStateSummaryResultData @@ -11489,13 +11410,9 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) - val config = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", - ) - val response = getReadShareGroupStateSummaryResponse( readSummaryRequestData, - config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, + ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, authorizer, readStateSummaryResultData @@ -11539,9 +11456,8 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(util.List.of(AuthorizationResult.DENIED)) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11567,9 +11483,8 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(util.List.of(AuthorizationResult.DENIED)) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11592,7 +11507,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid val topicName3 = "topic-3" val topicId3 = Uuid.randomUuid - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 1, topicId = topicId2) addTopicToMetadataCache(topicName3, 1, topicId = topicId3) @@ -11625,9 +11540,7 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, describeShareGroupOffsetsRequestGroup2 )).thenReturn(futureGroup2) - kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val describeShareGroupOffsetsResponseGroup1 = new DescribeShareGroupOffsetsResponseGroup() @@ -11708,7 +11621,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid val topicName3 = "topic-3" val topicId3 = Uuid.randomUuid - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 1, topicId = topicId2) addTopicToMetadataCache(topicName3, 1, topicId = topicId3) @@ -11769,7 +11682,6 @@ class KafkaApisTest extends Logging { }.asJava } kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer) ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11886,7 +11798,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid val topicName3 = "topic-3" val topicId3 = Uuid.randomUuid - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 1, topicId = topicId2) addTopicToMetadataCache(topicName3, 1, topicId = topicId3) @@ -11931,7 +11843,6 @@ class KafkaApisTest extends Logging { }.asJava } kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer) ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -12048,7 +11959,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid val topicName3 = "topic-3" val topicId3 = Uuid.randomUuid - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 1, topicId = topicId2) addTopicToMetadataCache(topicName3, 1, topicId = topicId3) @@ -12072,9 +11983,7 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, describeShareGroupOffsetsRequestGroup2 )).thenReturn(futureGroup2) - kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val describeShareGroupOffsetsResponseGroup1 = new DescribeShareGroupOffsetsResponseGroup() @@ -12149,16 +12058,14 @@ class KafkaApisTest extends Logging { @Test def testDescribeShareGroupOffsetsRequestEmptyGroupsSuccess(): Unit = { - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val future = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] - kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val describeShareGroupOffsetsResponseGroup = new DescribeShareGroupOffsetsResponseGroup() @@ -12172,7 +12079,7 @@ class KafkaApisTest extends Logging { @Test def testDescribeShareGroupOffsetsRequestEmptyTopicsSuccess(): Unit = { - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() val describeShareGroupOffsetsRequestGroup = new DescribeShareGroupOffsetsRequestGroup().setGroupId("group") @@ -12185,9 +12092,7 @@ class KafkaApisTest extends Logging { requestChannelRequest.context, describeShareGroupOffsetsRequestGroup )).thenReturn(future) - kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val describeShareGroupOffsetsResponseGroup = new DescribeShareGroupOffsetsResponseGroup() @@ -12227,9 +12132,8 @@ class KafkaApisTest extends Logging { val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(util.List.of(AuthorizationResult.DENIED)) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -12259,7 +12163,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid val topicName2 = "topic-2" val topicId2 = Uuid.randomUuid - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 2, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -12294,7 +12198,6 @@ class KafkaApisTest extends Logging { .thenReturn(util.List.of(AuthorizationResult.ALLOWED)) kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Some(authorizer) ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -12342,7 +12245,7 @@ class KafkaApisTest extends Logging { val topicId2 = Uuid.randomUuid val topicName3 = "topic-3" val topicId3 = Uuid.randomUuid - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) addTopicToMetadataCache(topicName3, 3, topicId = topicId3) @@ -12368,9 +12271,7 @@ class KafkaApisTest extends Logging { deleteShareGroupOffsetsRequestData )).thenReturn(resultFuture) - kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val deleteShareGroupOffsetsResponseData = new DeleteShareGroupOffsetsResponseData() @@ -12405,7 +12306,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid val topicName2 = "topic-2" val topicId2 = Uuid.randomUuid - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -12426,9 +12327,7 @@ class KafkaApisTest extends Logging { deleteShareGroupOffsetsRequestData )).thenReturn(CompletableFuture.failedFuture(Errors.UNKNOWN_SERVER_ERROR.exception)) - kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val deleteShareGroupOffsetsResponseData = new DeleteShareGroupOffsetsResponseData() @@ -12445,7 +12344,7 @@ class KafkaApisTest extends Logging { val topicId1 = Uuid.randomUuid val topicName2 = "topic-2" val topicId2 = Uuid.randomUuid - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() addTopicToMetadataCache(topicName1, 1, topicId = topicId1) addTopicToMetadataCache(topicName2, 2, topicId = topicId2) @@ -12470,9 +12369,7 @@ class KafkaApisTest extends Logging { deleteShareGroupOffsetsRequestData )).thenReturn(CompletableFuture.completedFuture(groupCoordinatorResponse)) - kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val deleteShareGroupOffsetsResponseData = new DeleteShareGroupOffsetsResponseData() @@ -12485,7 +12382,7 @@ class KafkaApisTest extends Logging { @Test def testDeleteShareGroupOffsetsRequestEmptyTopicsSuccess(): Unit = { - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() val deleteShareGroupOffsetsRequest = new DeleteShareGroupOffsetsRequestData() .setGroupId("group") @@ -12493,9 +12390,7 @@ class KafkaApisTest extends Logging { val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] - kafkaApis = createKafkaApis( - overrideProperties = Map(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), - ) + kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) val deleteShareGroupOffsetsResponse = new DeleteShareGroupOffsetsResponseData() @@ -12540,13 +12435,9 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava - val config = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", - ) - val response = getWriteShareGroupStateResponse( writeRequestData, - config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, + ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, null, writeStateResultData @@ -12591,17 +12482,13 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava - val config = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", - ) - val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) val response = getWriteShareGroupStateResponse( writeRequestData, - config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, + ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, authorizer, writeStateResultData @@ -12640,13 +12527,9 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava - val config = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", - ) - val response = getDeleteShareGroupStateResponse( deleteRequestData, - config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, + ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, null, deleteStateResultData @@ -12685,13 +12568,9 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) - val config = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", - ) - val response = getDeleteShareGroupStateResponse( deleteRequestData, - config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, + ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, authorizer, deleteStateResultData @@ -12731,13 +12610,9 @@ class KafkaApisTest extends Logging { ).asJava) ).asJava - val config = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", - ) - val response = getInitializeShareGroupStateResponse( initRequestData, - config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, + ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, null, initStateResultData @@ -12777,13 +12652,9 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) - val config = Map( - ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", - ) - val response = getInitializeShareGroupStateResponse( initRequestData, - config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, + ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, authorizer, initStateResultData @@ -12797,7 +12668,7 @@ class KafkaApisTest extends Logging { }) } - def getShareGroupDescribeResponse(groupIds: util.List[String], configOverrides: Map[String, String] = Map.empty, + def getShareGroupDescribeResponse(groupIds: util.List[String], enableShareGroups: Boolean = true, verifyNoErr: Boolean = true, authorizer: Authorizer = null, describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup]): ShareGroupDescribeResponse = { val shareGroupDescribeRequestData = new ShareGroupDescribeRequestData() @@ -12809,9 +12680,8 @@ class KafkaApisTest extends Logging { any[RequestContext], any[util.List[String]] )).thenReturn(future) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled(enableShareGroups) kafkaApis = createKafkaApis( - overrideProperties = configOverrides, authorizer = Option(authorizer), ) kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -12837,7 +12707,7 @@ class KafkaApisTest extends Logging { any[RequestContext], any[ReadShareGroupStateRequestData] )).thenReturn(future) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() kafkaApis = createKafkaApis( overrideProperties = configOverrides, authorizer = Option(authorizer), @@ -12866,7 +12736,7 @@ class KafkaApisTest extends Logging { any[RequestContext], any[ReadShareGroupStateSummaryRequestData] )).thenReturn(future) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() kafkaApis = createKafkaApis( overrideProperties = configOverrides, authorizer = Option(authorizer), @@ -12895,7 +12765,7 @@ class KafkaApisTest extends Logging { any[RequestContext], any[WriteShareGroupStateRequestData] )).thenReturn(future) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() kafkaApis = createKafkaApis( overrideProperties = configOverrides, authorizer = Option(authorizer), @@ -12924,7 +12794,7 @@ class KafkaApisTest extends Logging { any[RequestContext], any[DeleteShareGroupStateRequestData] )).thenReturn(future) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() kafkaApis = createKafkaApis( overrideProperties = configOverrides, authorizer = Option(authorizer), @@ -12953,7 +12823,7 @@ class KafkaApisTest extends Logging { any[RequestContext], any[InitializeShareGroupStateRequestData] )).thenReturn(future) - metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) + metadataCache = initializeMetadataCacheWithShareGroupsEnabled() kafkaApis = createKafkaApis( overrideProperties = configOverrides, authorizer = Option(authorizer), diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 55c88666c70b7..7f78f2ba59047 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1742,13 +1742,11 @@ class KafkaConfigTest { props.put(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer") var config = KafkaConfig.fromProps(props) assertEquals(Set(GroupType.CLASSIC, GroupType.CONSUMER), config.groupCoordinatorRebalanceProtocols) - assertFalse(config.shareGroupConfig.isShareGroupEnabled) // This is OK. props.put(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer,share") config = KafkaConfig.fromProps(props) assertEquals(Set(GroupType.CLASSIC, GroupType.CONSUMER, GroupType.SHARE), config.groupCoordinatorRebalanceProtocols) - assertTrue(config.shareGroupConfig.isShareGroupEnabled) props.put(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,streams") val config2 = KafkaConfig.fromProps(props) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index c578ed1543d6e..5f524d34a5b53 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -6128,7 +6128,7 @@ class ReplicaManagerTest { topicPartitions.forEach((topicIdPartition: TopicIdPartition) => delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId, topicIdPartition.partition))) // You cannot acquire records for sp1, so request will be stored in purgatory waiting for timeout. - when(sp1.maybeAcquireFetchLock).thenReturn(false) + when(sp1.maybeAcquireFetchLock(any())).thenReturn(false) rm.addDelayedShareFetchRequest(delayedShareFetch = delayedShareFetch, delayedShareFetchKeys = delayedShareFetchWatchKeys) verify(delayedShareFetch, times(0)).forceComplete() diff --git a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala index 45afae3a0dc13..aef8d49aa099e 100644 --- a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala @@ -17,16 +17,18 @@ package kafka.server import kafka.utils.TestUtils -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, ClusterTests, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, ClusterTests, Type} import org.apache.kafka.common.message.ShareFetchResponseData.AcquiredRecords import org.apache.kafka.common.message.{ShareAcknowledgeRequestData, ShareAcknowledgeResponseData, ShareFetchRequestData, ShareFetchResponseData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.requests.{ShareAcknowledgeRequest, ShareAcknowledgeResponse, ShareFetchRequest, ShareFetchResponse, ShareRequestMetadata} import org.apache.kafka.common.test.ClusterInstance +import org.apache.kafka.server.common.Feature import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, Timeout} +import java.net.Socket import java.util import java.util.Collections import scala.jdk.CollectionConverters._ @@ -42,9 +44,14 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo @AfterEach def tearDown(): Unit = { closeProducer + closeSockets } - @ClusterTest + @ClusterTest( + features = Array( + new ClusterFeature(feature = Feature.SHARE_VERSION, version = 0) + ) + ) def testShareFetchRequestIsInAccessibleWhenConfigsDisabled(): Unit = { val groupId: String = "group" val metadata: ShareRequestMetadata = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH) @@ -53,20 +60,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("topic1", 1)) ) + val socket: Socket = connectAny() + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) assertEquals(Errors.UNSUPPORTED_VERSION.code, shareFetchResponse.data.errorCode) assertEquals(0, shareFetchResponse.data.acquisitionLockTimeoutMs) } - @ClusterTest + @ClusterTest( + features = Array( + new ClusterFeature(feature = Feature.SHARE_VERSION, version = 0) + ) + ) def testShareAcknowledgeRequestIsInAccessibleWhenConfigsDisabled(): Unit = { val groupId: String = "group" val metadata: ShareRequestMetadata = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH) + val socket: Socket = connectAny() + val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, Map.empty) - val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = IntegrationTestUtils.sendAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest, socket) assertEquals(Errors.UNSUPPORTED_VERSION.code, shareAcknowledgeResponse.data.errorCode) } @@ -75,8 +90,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ), @@ -84,8 +97,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -116,11 +127,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connect(nonReplicaId) + // Send the share fetch request to the non-replica and verify the error code val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest, nonReplicaId) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) assertEquals(30000, shareFetchResponse.data.acquisitionLockTimeoutMs) - val partitionData = shareFetchResponse.responseData(topicNames).get(topicIdPartition) assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.code, partitionData.errorCode) assertEquals(leader, partitionData.currentLeader().leaderId()) @@ -130,16 +142,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -162,8 +170,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -173,7 +183,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val metadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)) val acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMap) - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -196,16 +206,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -230,8 +236,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition1, topicIdPartition2, topicIdPartition3) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partitions - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic partitions created above @@ -248,7 +256,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // as the share partitions might not be initialized yet. So, we retry until we get the response. var responses = Seq[ShareFetchResponseData.PartitionData]() TestUtils.waitUntilTrue(() => { - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) assertEquals(30000, shareFetchResponseData.acquisitionLockTimeoutMs) @@ -296,8 +304,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ), @@ -305,8 +311,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -341,15 +345,19 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val metadata: ShareRequestMetadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH) val acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty + val socket1: Socket = connect(leader1) + val socket2: Socket = connect(leader2) + val socket3: Socket = connect(leader3) + // Send the first share fetch request to initialize the share partitions // Create different share fetch requests for different partitions as they may have leaders on separate brokers var shareFetchRequest1 = createShareFetchRequest(groupId, metadata, send1, Seq.empty, acknowledgementsMap) var shareFetchRequest2 = createShareFetchRequest(groupId, metadata, send2, Seq.empty, acknowledgementsMap) var shareFetchRequest3 = createShareFetchRequest(groupId, metadata, send3, Seq.empty, acknowledgementsMap) - var shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1, destination = leader1) - var shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2, destination = leader2) - var shareFetchResponse3 = connectAndReceive[ShareFetchResponse](shareFetchRequest3, destination = leader3) + var shareFetchResponse1 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest1, socket1) + var shareFetchResponse2 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest2, socket2) + var shareFetchResponse3 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest3, socket3) initProducer() // Producing 10 records to the topic partitions created above @@ -363,9 +371,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareFetchRequest2 = createShareFetchRequest(groupId, metadata, send2, Seq.empty, acknowledgementsMap) shareFetchRequest3 = createShareFetchRequest(groupId, metadata, send3, Seq.empty, acknowledgementsMap) - shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1, destination = leader1) - shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2, destination = leader2) - shareFetchResponse3 = connectAndReceive[ShareFetchResponse](shareFetchRequest3, destination = leader3) + shareFetchResponse1 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest1, socket1) + shareFetchResponse2 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest2, socket2) + shareFetchResponse3 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest3, socket3) val shareFetchResponseData1 = shareFetchResponse1.data() assertEquals(Errors.NONE.code, shareFetchResponseData1.errorCode) @@ -417,16 +425,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -450,8 +454,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize share partitions - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -462,7 +468,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -489,7 +495,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = IntegrationTestUtils.sendAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest, socket) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -511,7 +517,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -534,8 +540,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000") @@ -543,8 +547,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -569,8 +571,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send, 15000) + sendFirstShareFetchRequest(memberId, groupId, send, socket, 15000) initProducer() // Producing 10 records to the topic created above @@ -581,7 +585,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -610,7 +614,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -635,7 +639,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -658,16 +662,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -691,8 +691,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partiion - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -703,7 +705,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -730,7 +732,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(2.toByte))).asJava) // Release the records val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = IntegrationTestUtils.sendAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest, socket) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -749,7 +751,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -772,16 +774,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -805,8 +803,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -817,7 +817,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -862,7 +862,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo releaseAcknowledgementSent = true } shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -890,16 +890,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -923,8 +919,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -935,7 +933,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -962,7 +960,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(3.toByte))).asJava) // Reject the records val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = IntegrationTestUtils.sendAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest, socket) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -984,7 +982,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1007,16 +1005,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1040,8 +1034,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -1052,7 +1048,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1081,7 +1077,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(3.toByte))).asJava) // Reject the records shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1106,7 +1102,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1129,8 +1125,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.delivery.count.limit", value = "2") // Setting max delivery count config to 2 @@ -1138,8 +1132,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1164,8 +1156,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the shar partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -1176,7 +1170,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1203,7 +1197,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(2.toByte))).asJava) // Release the records var shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - var shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) + var shareAcknowledgeResponse = IntegrationTestUtils.sendAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest, socket) var shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -1222,7 +1216,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1248,7 +1242,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(2.toByte))).asJava) // Release the records again shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) + shareAcknowledgeResponse = IntegrationTestUtils.sendAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest, socket) shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -1270,7 +1264,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1293,16 +1287,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1315,7 +1305,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo def testShareFetchRequestSuccessfulSharingBetweenMultipleConsumers(): Unit = { val groupId: String = "group" - val memberId = Uuid.randomUuid() val memberId1 = Uuid.randomUuid() val memberId2 = Uuid.randomUuid() val memberId3 = Uuid.randomUuid() @@ -1330,8 +1319,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket1: Socket = connectAny() + val socket2: Socket = connectAny() + val socket3: Socket = connectAny() + // Sending a dummy share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId1, groupId, send, socket1) initProducer() // Producing 10000 records to the topic created above @@ -1351,9 +1344,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val acknowledgementsMap3: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty val shareFetchRequest3 = createShareFetchRequest(groupId, metadata3, send, Seq.empty, acknowledgementsMap3, minBytes = 100, maxBytes = 1500) - val shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1) - val shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2) - val shareFetchResponse3 = connectAndReceive[ShareFetchResponse](shareFetchRequest3) + val shareFetchResponse1 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest1, socket1) + val shareFetchResponse2 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest2, socket2) + val shareFetchResponse3 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest3, socket3) val shareFetchResponseData1 = shareFetchResponse1.data() @@ -1389,16 +1382,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1427,10 +1416,14 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket1: Socket = connectAny() + val socket2: Socket = connectAny() + val socket3: Socket = connectAny() + // Sending 3 dummy share Fetch Requests with to inititlaize the share partitions for each share group\ - sendFirstShareFetchRequest(memberId1, groupId1, send) - sendFirstShareFetchRequest(memberId2, groupId2, send) - sendFirstShareFetchRequest(memberId3, groupId3, send) + sendFirstShareFetchRequest(memberId1, groupId1, send, socket1) + sendFirstShareFetchRequest(memberId2, groupId2, send, socket2) + sendFirstShareFetchRequest(memberId3, groupId3, send, socket3) initProducer() // Producing 10 records to the topic created above @@ -1450,9 +1443,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val acknowledgementsMap3: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty val shareFetchRequest3 = createShareFetchRequest(groupId3, metadata3, send, Seq.empty, acknowledgementsMap3) - val shareFetchResponse1 = connectAndReceive[ShareFetchResponse](shareFetchRequest1) - val shareFetchResponse2 = connectAndReceive[ShareFetchResponse](shareFetchRequest2) - val shareFetchResponse3 = connectAndReceive[ShareFetchResponse](shareFetchRequest3) + val shareFetchResponse1 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest1, socket1) + val shareFetchResponse2 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest2, socket2) + val shareFetchResponse3 = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest3, socket3) val shareFetchResponseData1 = shareFetchResponse1.data() @@ -1489,16 +1482,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1522,8 +1511,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -1534,7 +1525,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1563,7 +1554,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1589,7 +1580,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(19) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1601,16 +1592,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1634,8 +1621,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -1646,7 +1635,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var acknowledgementsMapForFetch: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1675,7 +1664,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMapForFetch) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1702,7 +1691,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(19) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Accept the records val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMapForAcknowledge) - val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = IntegrationTestUtils.sendAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest, socket) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.NONE.code, shareAcknowledgeResponseData.errorCode) @@ -1722,16 +1711,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1759,6 +1744,8 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + val metadata: ShareRequestMetadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH) val acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map(topicIdPartition -> List(new ShareFetchRequestData.AcknowledgementBatch() @@ -1766,7 +1753,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) // Acknowledgements in the Initial Fetch Request val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMap) - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) val shareFetchResponseData = shareFetchResponse.data() // The response will have a top level error code because this is an Initial Fetch request with acknowledgement data present @@ -1778,16 +1765,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1809,6 +1792,8 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val topicId = topicIds.get(topic) val topicIdPartition = new TopicIdPartition(topicId, new TopicPartition(topic, partition)) + val socket: Socket = connectAny() + // Send the share fetch request to fetch the records produced above val metadata: ShareRequestMetadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH) val acknowledgementsMap: Map[TopicIdPartition, util.List[ShareAcknowledgeRequestData.AcknowledgementBatch]] = @@ -1818,7 +1803,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMap) - val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = IntegrationTestUtils.sendAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest, socket) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.INVALID_SHARE_SESSION_EPOCH.code, shareAcknowledgeResponseData.errorCode) @@ -1828,16 +1813,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1861,8 +1842,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -1872,7 +1855,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1894,7 +1877,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.nextEpoch(shareSessionEpoch)) metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.INVALID_SHARE_SESSION_EPOCH.code, shareFetchResponseData.errorCode) @@ -1904,16 +1887,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -1937,8 +1916,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -1948,7 +1929,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -1975,7 +1956,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMap) - val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = IntegrationTestUtils.sendAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest, socket) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.INVALID_SHARE_SESSION_EPOCH.code, shareAcknowledgeResponseData.errorCode) @@ -1985,16 +1966,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -2019,8 +1996,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -2030,7 +2009,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) var shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - var shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + var shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) var shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -2052,7 +2031,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo shareSessionEpoch = ShareRequestMetadata.nextEpoch(shareSessionEpoch) metadata = new ShareRequestMetadata(wrongMemberId, shareSessionEpoch) shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.SHARE_SESSION_NOT_FOUND.code, shareFetchResponseData.errorCode) @@ -2062,16 +2041,100 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.max.share.sessions", value="2"), + new ClusterConfigProperty(key = "group.share.max.size", value="2") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "group.share.max.share.sessions", value="2"), + new ClusterConfigProperty(key = "group.share.max.size", value="2") + ) + ), + ) + ) + def testShareSessionEvictedOnConnectionDrop(): Unit = { + val groupId: String = "group" + val memberId1 = Uuid.randomUuid() + val memberId2 = Uuid.randomUuid() + val memberId3 = Uuid.randomUuid() + + val topic = "topic" + val partition = 0 + + createTopicAndReturnLeaders(topic, numPartitions = 3) + val topicIds = getTopicIds.asJava + val topicId = topicIds.get(topic) + val topicIdPartition = new TopicIdPartition(topicId, new TopicPartition(topic, partition)) + + val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + + val socket1: Socket = connectAny() + val socket2: Socket = connectAny() + val socket3: Socket = connectAny() + + // member1 sends share fetch request to register it's share session. Note it does not close the socket connection after. + TestUtils.waitUntilTrue(() => { + val metadata = new ShareRequestMetadata(memberId1, ShareRequestMetadata.INITIAL_EPOCH) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket1) + val shareFetchResponseData = shareFetchResponse.data() + shareFetchResponseData.errorCode == Errors.NONE.code + }, "Share fetch request failed", 5000) + + // member2 sends share fetch request to register it's share session. Note it does not close the socket connection after. + TestUtils.waitUntilTrue(() => { + val metadata = new ShareRequestMetadata(memberId2, ShareRequestMetadata.INITIAL_EPOCH) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket2) + val shareFetchResponseData = shareFetchResponse.data() + shareFetchResponseData.errorCode == Errors.NONE.code + }, "Share fetch request failed", 5000) + + // member3 sends share fetch request to register it's share session. Since the maximum number of share sessions that could + // exist in the share session cache is 2 (group.share.max.share.sessions), the attempt to register a third + // share session with the ShareSessionCache would throw SHARE_SESSION_LIMIT_REACHED + TestUtils.waitUntilTrue(() => { + val metadata = new ShareRequestMetadata(memberId3, ShareRequestMetadata.INITIAL_EPOCH) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket3) + val shareFetchResponseData = shareFetchResponse.data() + shareFetchResponseData.errorCode == Errors.SHARE_SESSION_NOT_FOUND.code + }, "Share fetch request failed", 5000) + + // Now we will close the socket connections for the members, mimicking a client disconnection + closeSockets() + + val socket4: Socket = connectAny() + + // Since one of the socket connections was closed before, the corresponding share session was dropped from the ShareSessionCache + // on the broker. Now, since the cache is not full, new share sessions can be registered + TestUtils.waitUntilTrue(() => { + val metadata = new ShareRequestMetadata(memberId3, ShareRequestMetadata.INITIAL_EPOCH) + val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket4) + val shareFetchResponseData = shareFetchResponse.data() + shareFetchResponseData.errorCode == Errors.NONE.code + }, "Share fetch request failed", 5000) + } + + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -2096,8 +2159,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -2107,7 +2172,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo var shareSessionEpoch = ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH) var metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -2134,7 +2199,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo .setLastOffset(9) .setAcknowledgeTypes(Collections.singletonList(1.toByte))).asJava) val shareAcknowledgeRequest = createShareAcknowledgeRequest(groupId, metadata, acknowledgementsMap) - val shareAcknowledgeResponse = connectAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest) + val shareAcknowledgeResponse = IntegrationTestUtils.sendAndReceive[ShareAcknowledgeResponse](shareAcknowledgeRequest, socket) val shareAcknowledgeResponseData = shareAcknowledgeResponse.data() assertEquals(Errors.SHARE_SESSION_NOT_FOUND.code, shareAcknowledgeResponseData.errorCode) @@ -2144,16 +2209,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -2179,8 +2240,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition1, topicIdPartition2) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic partitions created above @@ -2197,7 +2260,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // as the share partitions might not be initialized yet. So, we retry until we get the response. var responses = Seq[ShareFetchResponseData.PartitionData]() TestUtils.waitUntilTrue(() => { - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) assertEquals(30000, shareFetchResponseData.acquisitionLockTimeoutMs) @@ -2223,7 +2286,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo metadata = new ShareRequestMetadata(memberId, shareSessionEpoch) val forget: Seq[TopicIdPartition] = Seq(topicIdPartition1) shareFetchRequest = createShareFetchRequest(groupId, metadata, Seq.empty, forget, acknowledgementsMap) - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -2246,16 +2309,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -2278,8 +2337,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -2289,7 +2350,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val metadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)) val acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMap, maxRecords = 1, batchSize = 1) - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) val shareFetchResponseData = shareFetchResponse.data assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -2312,16 +2373,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo Array( new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") ) ), new ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), @@ -2344,8 +2401,10 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val send: Seq[TopicIdPartition] = Seq(topicIdPartition) + val socket: Socket = connectAny() + // Send the first share fetch request to initialize the share partition - sendFirstShareFetchRequest(memberId, groupId, send) + sendFirstShareFetchRequest(memberId, groupId, send, socket) initProducer() // Producing 10 records to the topic created above @@ -2355,7 +2414,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val metadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)) val acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]] = Map.empty val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, acknowledgementsMap, maxRecords = 5, batchSize = 1) - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) val shareFetchResponseData = shareFetchResponse.data assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) @@ -2376,12 +2435,12 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo // For initial fetch request, the response may not be available in the first attempt when the share // partition is not initialized yet. Hence, wait for response from all partitions before proceeding. - private def sendFirstShareFetchRequest(memberId: Uuid, groupId: String, topicIdPartitions: Seq[TopicIdPartition], lockTimeout: Int = 30000): Unit = { + private def sendFirstShareFetchRequest(memberId: Uuid, groupId: String, topicIdPartitions: Seq[TopicIdPartition], socket: Socket, lockTimeout: Int = 30000): Unit = { val partitions: util.Set[Integer] = new util.HashSet() TestUtils.waitUntilTrue(() => { val metadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH) val shareFetchRequest = createShareFetchRequest(groupId, metadata, topicIdPartitions, Seq.empty, Map.empty) - val shareFetchResponse = connectAndReceive[ShareFetchResponse](shareFetchRequest) + val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) val shareFetchResponseData = shareFetchResponse.data() assertEquals(Errors.NONE.code, shareFetchResponseData.errorCode) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala index 6c707c7bae13f..408f31db8d15f 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala @@ -29,6 +29,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig import org.apache.kafka.security.authorizer.AclEntry +import org.apache.kafka.server.common.Feature import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Timeout @@ -41,7 +42,11 @@ import scala.jdk.CollectionConverters._ )) class ShareGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest + @ClusterTest( + features = Array( + new ClusterFeature(feature = Feature.SHARE_VERSION, version = 0) + ) + ) def testShareGroupDescribeIsInAccessibleWhenConfigsDisabled(): Unit = { val shareGroupDescribeRequest = new ShareGroupDescribeRequest.Builder( new ShareGroupDescribeRequestData().setGroupIds(List("grp-1", "grp-2").asJava) @@ -65,10 +70,8 @@ class ShareGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCoord @ClusterTest( serverProperties = Array( - new ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer,share"), new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"), - new ClusterConfigProperty(key = ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG, value = "true"), ) ) def testShareGroupDescribe(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala index 77aae6c4d5731..865870eef3beb 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala @@ -16,7 +16,7 @@ */ package kafka.server -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, NewPartitions} import org.apache.kafka.common.Uuid @@ -24,6 +24,7 @@ import org.apache.kafka.common.message.{ShareGroupHeartbeatRequestData, ShareGro import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse} import org.apache.kafka.common.test.ClusterInstance +import org.apache.kafka.server.common.Feature import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertNotNull, assertNull, assertTrue} import org.junit.jupiter.api.Timeout @@ -37,7 +38,9 @@ import scala.jdk.CollectionConverters._ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { @ClusterTest( - types = Array(Type.KRAFT) + features = Array( + new ClusterFeature(feature = Feature.SHARE_VERSION, version = 0) + ) ) def testShareGroupHeartbeatIsInAccessibleWhenConfigsDisabled(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( @@ -50,9 +53,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } @ClusterTest( - types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") )) @@ -142,9 +143,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } @ClusterTest( - types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") )) @@ -298,9 +297,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } @ClusterTest( - types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") )) @@ -409,9 +406,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } @ClusterTest( - types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") )) @@ -591,10 +586,82 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } } + @ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.max.size", value = "2") + )) + def testShareGroupMaxSizeConfigExceeded(): Unit = { + val groupId: String = "group" + val memberId1 = Uuid.randomUuid() + val memberId2 = Uuid.randomUuid() + val memberId3 = Uuid.randomUuid() + + val admin = cluster.admin() + + // Creates the __consumer_offsets topics because it won't be created automatically + // in this test because it does not use FindCoordinator API. + try { + TestUtils.createOffsetsTopicWithAdmin( + admin = admin, + brokers = cluster.brokers.values().asScala.toSeq, + controllers = cluster.controllers().values().asScala.toSeq + ) + + // Heartbeat request to join the group by the first member (memberId1). + var shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( + new ShareGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1.toString) + .setMemberEpoch(0) + .setSubscribedTopicNames(List("foo").asJava) + ).build() + + // Send the request until receiving a successful response. There is a delay + // here because the group coordinator is loaded in the background. + var shareGroupHeartbeatResponse: ShareGroupHeartbeatResponse = null + TestUtils.waitUntilTrue(() => { + shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) + shareGroupHeartbeatResponse.data.errorCode == Errors.NONE.code + }, msg = s"Could not join the group successfully. Last response $shareGroupHeartbeatResponse.") + + // Heartbeat request to join the group by the second member (memberId2). + shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( + new ShareGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2.toString) + .setMemberEpoch(0) + .setSubscribedTopicNames(List("foo").asJava) + ).build() + + // Send the request until receiving a successful response + TestUtils.waitUntilTrue(() => { + shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) + shareGroupHeartbeatResponse.data.errorCode == Errors.NONE.code + }, msg = s"Could not join the group successfully. Last response $shareGroupHeartbeatResponse.") + + // Heartbeat request to join the group by the third member (memberId3). + shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( + new ShareGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3.toString) + .setMemberEpoch(0) + .setSubscribedTopicNames(List("foo").asJava) + ).build() + + shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) + // Since the group.share.max.size config is set to 2, a third member cannot join the same group. + assertEquals(shareGroupHeartbeatResponse.data.errorCode, Errors.GROUP_MAX_SIZE_REACHED.code) + + } finally { + admin.close() + } + } + @ClusterTest( types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), new ClusterConfigProperty(key = "group.share.heartbeat.interval.ms", value = "500"), @@ -773,9 +840,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { } @ClusterTest( - types = Array(Type.KRAFT), serverProperties = Array( - new ClusterConfigProperty(key = "group.share.enable", value = "true"), new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") )) 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 1e297fc33c994..3abea688468b1 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala @@ -86,7 +86,7 @@ class OffsetsForLeaderEpochTest { //Then assertEquals( - Seq(newOffsetForLeaderTopicResult(tp, Errors.NONE, offsetAndEpoch.leaderEpoch, offsetAndEpoch.offset)), + Seq(newOffsetForLeaderTopicResult(tp, Errors.NONE, offsetAndEpoch.epoch(), offsetAndEpoch.offset)), response) } diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala index 575d3855dcde5..20e5072cdd898 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -196,7 +196,7 @@ class BrokerMetadataPublisherTest { replicaManager, groupCoordinator, mock(classOf[TransactionCoordinator]), - Some(mock(classOf[ShareCoordinator])), + mock(classOf[ShareCoordinator]), mock(classOf[DynamicConfigPublisher]), mock(classOf[DynamicClientQuotaPublisher]), mock(classOf[DynamicTopicClusterQuotaPublisher]), diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 7227c72b9b94b..4320e46901661 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -43,8 +43,8 @@ import org.apache.kafka.coordinator.share.generated.{ShareSnapshotKey, ShareSnap import org.apache.kafka.coordinator.transaction.generated.{TransactionLogKey, TransactionLogValue} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.MetadataRecordSerde -import org.apache.kafka.raft.{KafkaRaftClient, MetadataLogConfig, OffsetAndEpoch, VoterSetTest} -import org.apache.kafka.server.common.{ApiMessageAndVersion, KRaftVersion} +import org.apache.kafka.raft.{KafkaRaftClient, MetadataLogConfig, VoterSetTest} +import org.apache.kafka.server.common.{ApiMessageAndVersion, KRaftVersion, OffsetAndEpoch} import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteLogSegmentMetadataUpdate, RemoteLogSegmentState, RemotePartitionDeleteMetadata, RemotePartitionDeleteState} @@ -545,6 +545,7 @@ class DumpLogSegmentsTest { time, time.scheduler, new MetadataLogConfig( + 100 * 1024, 100 * 1024, 10 * 1000, 100 * 1024, diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala deleted file mode 100644 index 8518a469c7d78..0000000000000 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ /dev/null @@ -1,234 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * 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 - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.utils - -import java.util.{Optional, Properties} -import java.util.concurrent.atomic._ -import java.util.concurrent.{ConcurrentHashMap, CountDownLatch, Executors, TimeUnit} -import kafka.utils.TestUtils.retry -import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.server.util.{KafkaScheduler, MockTime} -import org.apache.kafka.storage.internals.log.{LocalLog, LogConfig, LogDirFailureChannel, LogLoader, LogOffsetsListener, LogSegments, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog} -import org.apache.kafka.storage.log.metrics.BrokerTopicStats -import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, Timeout} - - -class SchedulerTest { - - val scheduler = new KafkaScheduler(1) - val mockTime = new MockTime - val counter1 = new AtomicInteger(0) - val counter2 = new AtomicInteger(0) - - @BeforeEach - def setup(): Unit = { - scheduler.startup() - } - - @AfterEach - def teardown(): Unit = { - scheduler.shutdown() - } - - @Test - def testMockSchedulerNonPeriodicTask(): Unit = { - mockTime.scheduler.scheduleOnce("test1", () => counter1.getAndIncrement(), 1) - mockTime.scheduler.scheduleOnce("test2", () => counter2.getAndIncrement(), 100) - assertEquals(0, counter1.get, "Counter1 should not be incremented prior to task running.") - assertEquals(0, counter2.get, "Counter2 should not be incremented prior to task running.") - mockTime.sleep(1) - assertEquals(1, counter1.get, "Counter1 should be incremented") - assertEquals(0, counter2.get, "Counter2 should not be incremented") - mockTime.sleep(100000) - assertEquals(1, counter1.get, "More sleeping should not result in more incrementing on counter1.") - assertEquals(1, counter2.get, "Counter2 should now be incremented.") - } - - @Test - def testMockSchedulerPeriodicTask(): Unit = { - mockTime.scheduler.schedule("test1", () => counter1.getAndIncrement(), 1, 1) - mockTime.scheduler.schedule("test2", () => counter2.getAndIncrement(), 100, 100) - assertEquals(0, counter1.get, "Counter1 should not be incremented prior to task running.") - assertEquals(0, counter2.get, "Counter2 should not be incremented prior to task running.") - mockTime.sleep(1) - assertEquals(1, counter1.get, "Counter1 should be incremented") - assertEquals(0, counter2.get, "Counter2 should not be incremented") - mockTime.sleep(100) - assertEquals(101, counter1.get, "Counter1 should be incremented 101 times") - assertEquals(1, counter2.get, "Counter2 should not be incremented once") - } - - @Test - def testReentrantTaskInMockScheduler(): Unit = { - mockTime.scheduler.scheduleOnce("test1", () => mockTime.scheduler.scheduleOnce("test2", () => counter2.getAndIncrement(), 0), 1) - mockTime.sleep(1) - assertEquals(1, counter2.get) - } - - @Test - def testNonPeriodicTask(): Unit = { - scheduler.scheduleOnce("test", () => counter1.getAndIncrement()) - retry(30000) { - assertEquals(counter1.get, 1) - } - Thread.sleep(5) - assertEquals(1, counter1.get, "Should only run once") - } - - @Test - def testNonPeriodicTaskWhenPeriodIsZero(): Unit = { - scheduler.schedule("test", () => counter1.getAndIncrement(), 0, 0) - retry(30000) { - assertEquals(counter1.get, 1) - } - Thread.sleep(5) - assertEquals(1, counter1.get, "Should only run once") - } - - @Test - def testPeriodicTask(): Unit = { - scheduler.schedule("test", () => counter1.getAndIncrement(), 0, 5) - retry(30000) { - assertTrue(counter1.get >= 20, "Should count to 20") - } - } - - @Test - def testRestart(): Unit = { - // schedule a task to increment a counter - mockTime.scheduler.scheduleOnce("test1", () => counter1.getAndIncrement(), 1) - mockTime.sleep(1) - assertEquals(1, counter1.get()) - - // restart the scheduler - mockTime.scheduler.shutdown() - mockTime.scheduler.startup() - - // schedule another task to increment the counter - mockTime.scheduler.scheduleOnce("test1", () => counter1.getAndIncrement(), 1) - mockTime.sleep(1) - assertEquals(2, counter1.get()) - } - - @Test - def testUnscheduleProducerTask(): Unit = { - val tmpDir = TestUtils.tempDir() - val logDir = TestUtils.randomPartitionLogDir(tmpDir) - val logConfig = new LogConfig(new Properties()) - val brokerTopicStats = new BrokerTopicStats - val maxTransactionTimeoutMs = 5 * 60 * 1000 - val maxProducerIdExpirationMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT - val producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT - val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) - val logDirFailureChannel = new LogDirFailureChannel(10) - val segments = new LogSegments(topicPartition) - val leaderEpochCache = UnifiedLog.createLeaderEpochCache( - logDir, topicPartition, logDirFailureChannel, Optional.empty, mockTime.scheduler) - val producerStateManagerConfig = new ProducerStateManagerConfig(maxProducerIdExpirationMs, false) - val producerStateManager = new ProducerStateManager(topicPartition, logDir, - maxTransactionTimeoutMs, producerStateManagerConfig, mockTime) - val offsets = new LogLoader( - logDir, - topicPartition, - logConfig, - scheduler, - mockTime, - logDirFailureChannel, - true, - segments, - 0L, - 0L, - leaderEpochCache, - producerStateManager, - new ConcurrentHashMap[String, Integer], - false - ).load() - val localLog = new LocalLog(logDir, logConfig, segments, offsets.recoveryPoint, - offsets.nextOffsetMetadata, scheduler, mockTime, topicPartition, logDirFailureChannel) - val log = new UnifiedLog(offsets.logStartOffset, - localLog, - brokerTopicStats, - producerIdExpirationCheckIntervalMs, - leaderEpochCache, - producerStateManager, - Optional.empty, - false, - LogOffsetsListener.NO_OP_OFFSETS_LISTENER) - assertTrue(scheduler.taskRunning(log.producerExpireCheck)) - log.close() - assertFalse(scheduler.taskRunning(log.producerExpireCheck)) - } - - /** - * Verify that scheduler lock is not held when invoking task method, allowing new tasks to be scheduled - * when another is being executed. This is required to avoid deadlocks when: - * a) Thread1 executes a task which attempts to acquire LockA - * b) Thread2 holding LockA attempts to schedule a new task - */ - @Timeout(15) - @Test - def testMockSchedulerLocking(): Unit = { - val initLatch = new CountDownLatch(1) - val completionLatch = new CountDownLatch(2) - val taskLatches = List(new CountDownLatch(1), new CountDownLatch(1)) - def scheduledTask(taskLatch: CountDownLatch): Unit = { - initLatch.countDown() - assertTrue(taskLatch.await(30, TimeUnit.SECONDS), "Timed out waiting for latch") - completionLatch.countDown() - } - mockTime.scheduler.scheduleOnce("test1", () => scheduledTask(taskLatches.head), 1) - val tickExecutor = Executors.newSingleThreadScheduledExecutor() - try { - tickExecutor.scheduleWithFixedDelay(() => mockTime.sleep(1), 0, 1, TimeUnit.MILLISECONDS) - - // wait for first task to execute and then schedule the next task while the first one is running - assertTrue(initLatch.await(10, TimeUnit.SECONDS)) - mockTime.scheduler.scheduleOnce("test2", () => scheduledTask(taskLatches(1)), 1) - - taskLatches.foreach(_.countDown()) - assertTrue(completionLatch.await(10, TimeUnit.SECONDS), "Tasks did not complete") - - } finally { - tickExecutor.shutdownNow() - } - } - - @Test - def testPendingTaskSize(): Unit = { - val latch1 = new CountDownLatch(1) - val latch2 = new CountDownLatch(2) - val task1 = new Runnable { - override def run(): Unit = { - latch1.await() - } - } - scheduler.scheduleOnce("task1", task1, 0) - scheduler.scheduleOnce("task2", () => latch2.countDown(), 5) - scheduler.scheduleOnce("task3", () => latch2.countDown(), 5) - retry(30000) { - assertEquals(2, scheduler.pendingTaskSize()) - } - latch1.countDown() - latch2.await() - retry(30000) { - assertEquals(0, scheduler.pendingTaskSize()) - } - scheduler.shutdown() - assertEquals(0, scheduler.pendingTaskSize()) - } -} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index ebccd66359b8a..cc2566b492e96 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -1702,6 +1702,7 @@ private CompletableFuture new DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponsePartition() .setPartitionIndex(partitionData.partition()) .setStartOffset(partitionData.errorCode() == Errors.NONE.code() ? partitionData.startOffset() : PartitionFactory.UNINITIALIZED_START_OFFSET) + .setLeaderEpoch(partitionData.errorCode() == Errors.NONE.code() ? partitionData.leaderEpoch() : PartitionFactory.DEFAULT_LEADER_EPOCH) ).toList()) )); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index fb2c7c4c6af45..c3387859bc774 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -2684,7 +2684,12 @@ Map> subscribedTopicsChangeMap(String groupId, Map> topicPartitionChangeMap = new HashMap<>(); ShareGroupStatePartitionMetadataInfo info = shareGroupPartitionMetadata.get(groupId); - Map> alreadyInitialized = info == null ? new HashMap<>() : mergeShareGroupInitMaps(info.initializedTopics(), info.initializingTopics()); + + // We are only considering initialized TPs here. This is because it could happen + // that some topics have been moved to initializing but the corresponding persister request + // could not be made/failed (invoked by the group coordinator). Then there would be no way to try + // the persister call. This way we get the opportunity to retry. + Map> alreadyInitialized = info == null ? new HashMap<>() : info.initializedTopics(); subscriptionMetadata.forEach((topicName, topicMetadata) -> { Set alreadyInitializedPartSet = alreadyInitialized.getOrDefault(topicMetadata.id(), Set.of()); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java index d686ba1a3427c..faa9b92dff6c4 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.coordinator.group.modern.share; -import org.apache.kafka.common.GroupType; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.utils.Utils; @@ -24,8 +23,6 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; @@ -72,7 +69,7 @@ public class ShareGroupConfig { public static final String SHARE_GROUP_PERSISTER_CLASS_NAME_CONFIG = "group.share.persister.class.name"; public static final String SHARE_GROUP_PERSISTER_CLASS_NAME_DEFAULT = "org.apache.kafka.server.share.persister.DefaultStatePersister"; - public static final String SHARE_GROUP_PERSISTER_CLASS_NAME_DOC = "The class name of share persister for share group. The class should implement " + + public static final String SHARE_GROUP_PERSISTER_CLASS_NAME_DOC = "The fully qualified name of a class which implements " + "the org.apache.kafka.server.share.Persister interface."; public static final ConfigDef CONFIG_DEF = new ConfigDef() @@ -99,13 +96,8 @@ public class ShareGroupConfig { public ShareGroupConfig(AbstractConfig config) { this.config = config; - // Share groups are enabled in either of the two following cases: - // 1. The internal configuration to enable it is explicitly set; or - // 2. the share rebalance protocol is enabled. - Set protocols = config.getList(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG) - .stream().map(String::toUpperCase).collect(Collectors.toSet()); - isShareGroupEnabled = config.getBoolean(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG) || - protocols.contains(GroupType.SHARE.name()); + // The proper way to enable share groups is to use the share.version feature with v1 or later. + isShareGroupEnabled = config.getBoolean(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG); shareGroupPartitionMaxRecordLocks = config.getInt(ShareGroupConfig.SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG); shareGroupDeliveryCountLimit = config.getInt(ShareGroupConfig.SHARE_GROUP_DELIVERY_COUNT_LIMIT_CONFIG); shareGroupRecordLockDurationMs = config.getInt(ShareGroupConfig.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index f84fcd627c9ac..0fd4d1270e282 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -109,7 +109,6 @@ import org.apache.kafka.server.share.persister.TopicData; import org.apache.kafka.server.util.FutureUtils; import org.apache.kafka.server.util.timer.MockTimer; -import org.apache.kafka.storage.internals.log.LogConfig; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -901,7 +900,7 @@ public void testGroupMetadataTopicConfigs() { Properties expectedProperties = new Properties(); expectedProperties.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); expectedProperties.put(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.PRODUCER.name); - expectedProperties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000"); + expectedProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000"); assertEquals(expectedProperties, service.groupMetadataTopicConfigs()); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 250b10fec34e2..065dbd6aa9810 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -21260,16 +21260,21 @@ public void testShareGroupHeartbeatInitializeOnPartitionUpdate() { .setMemberEpoch(0) .setSubscribedTopicNames(List.of(t1Name, t2Name))); - assertTrue(result.records().contains( - newShareGroupStatePartitionMetadataRecord(groupId, mkShareGroupStateMap(List.of( - mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(0, 1)), - mkShareGroupStateMetadataEntry(t2Uuid, t2Name, List.of(0, 1)) - )), - Map.of(), - Map.of() - )) + CoordinatorRecord expected = newShareGroupStatePartitionMetadataRecord( + groupId, + mkShareGroupStateMap(List.of( + mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(0, 1)), + mkShareGroupStateMetadataEntry(t2Uuid, t2Name, List.of(0, 1)) + )), + Map.of(), + Map.of() ); + Optional actual = result.records().stream().filter(record -> record.key() instanceof ShareGroupStatePartitionMetadataKey) + .findAny(); + assertTrue(actual.isPresent()); + assertRecordEquals(expected, actual.get()); + verifyShareGroupHeartbeatInitializeRequest( result.response().getValue(), Map.of( @@ -21330,18 +21335,23 @@ public void testShareGroupHeartbeatInitializeOnPartitionUpdate() { .setMemberEpoch(1) .setSubscribedTopicNames(null)); - assertTrue(result.records().contains( - newShareGroupStatePartitionMetadataRecord(groupId, mkShareGroupStateMap(List.of( - mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(2, 3)) - )), - mkShareGroupStateMap(List.of( - mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(0, 1)), - mkShareGroupStateMetadataEntry(t2Uuid, t2Name, List.of(0, 1)) - )), - Map.of() - )) + expected = newShareGroupStatePartitionMetadataRecord( + groupId, + mkShareGroupStateMap(List.of( + mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(2, 3)) + )), + mkShareGroupStateMap(List.of( + mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(0, 1)), + mkShareGroupStateMetadataEntry(t2Uuid, t2Name, List.of(0, 1)) + )), + Map.of() ); + actual = result.records().stream().filter(record -> record.key() instanceof ShareGroupStatePartitionMetadataKey) + .findAny(); + assertTrue(actual.isPresent()); + assertRecordEquals(expected, actual.get()); + verifyShareGroupHeartbeatInitializeRequest( result.response().getValue(), Map.of( @@ -21357,7 +21367,7 @@ public void testShareGroupHeartbeatInitializeOnPartitionUpdate() { } @Test - public void testShareGroupHeartbeatNoPersisterRequestWithInitializing() { + public void testShareGroupHeartbeatPersisterRequestWithInitializing() { MockPartitionAssignor assignor = new MockPartitionAssignor("range"); assignor.prepareGroupAssignment(new GroupAssignment(Map.of())); GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() @@ -21401,7 +21411,7 @@ public void testShareGroupHeartbeatNoPersisterRequestWithInitializing() { .setMemberEpoch(0) .setSubscribedTopicNames(List.of(t1Name))); - assertFalse(result.records().contains( + assertTrue(result.records().contains( newShareGroupStatePartitionMetadataRecord(groupId, mkShareGroupStateMap(List.of( mkShareGroupStateMetadataEntry(t1Uuid, t1Name, List.of(0, 1)) )), @@ -21412,10 +21422,10 @@ public void testShareGroupHeartbeatNoPersisterRequestWithInitializing() { verifyShareGroupHeartbeatInitializeRequest( result.response().getValue(), - Map.of(), + Map.of(t1Uuid, Set.of(0, 1)), groupId, - 0, - false + 1, + true ); } @@ -21620,6 +21630,7 @@ topicName, new TopicMetadata(topicId, topicName, partitions) // Since t1 is initializing and t2 is initialized due to replay above. assertEquals( Map.of( + t1Id, Set.of(0, 1), t3Id, Set.of(0, 1, 2) ), context.groupMetadataManager.subscribedTopicsChangeMap(groupId, Map.of( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java index 698dd59380c67..73c0c0cc10e68 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java @@ -122,7 +122,6 @@ public static ShareGroupConfig createShareGroupConfig( configs.put(ShareGroupConfig.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG, shareGroupRecordLockDurationsMs); configs.put(ShareGroupConfig.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG, shareGroupMinRecordLockDurationMs); configs.put(ShareGroupConfig.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG, shareGroupMaxRecordLockDurationMs); - configs.put(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer,share"); configs.put(GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG, 1); configs.put(GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, 10); diff --git a/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java index 5c49687774710..9fad1d0d88bad 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/OffsetControlManager.java @@ -24,8 +24,8 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.controller.metrics.QuorumControllerMetrics; import org.apache.kafka.raft.Batch; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.Snapshots; import org.apache.kafka.timeline.SnapshotRegistry; diff --git a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java index 2fe49220018d9..ea196dfc741b8 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java +++ b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java @@ -334,11 +334,11 @@ private void tryElection(PartitionChangeRecord record) { targetIsr = List.of(electionResult.node); targetElr = targetElr.stream().filter(replica -> replica != electionResult.node) .collect(Collectors.toList()); - log.trace("Setting new leader for topicId {}, partition {} to {} using ELR", - topicId, partitionId, electionResult.node); + log.info("Setting new leader for topicId {}, partition {} to {} using ELR. Previous partition: {}, change record: {}", + topicId, partitionId, electionResult.node, partition, record); } else if (electionResult.unclean) { - log.info("Setting new leader for topicId {}, partition {} to {} using an unclean election", - topicId, partitionId, electionResult.node); + log.info("Setting new leader for topicId {}, partition {} to {} using an unclean election. Previous partition: {}, change record: {}", + topicId, partitionId, electionResult.node, partition, record); } else { log.trace("Setting new leader for topicId {}, partition {} to {} using a clean election", topicId, partitionId, electionResult.node); diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index b03501eb7bb67..ded837ec6a34a 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -111,12 +111,12 @@ import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.LeaderAndEpoch; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RaftClient; import org.apache.kafka.server.authorizer.AclCreateResult; import org.apache.kafka.server.authorizer.AclDeleteResult; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.fault.FaultHandler; import org.apache.kafka.server.fault.FaultHandlerException; import org.apache.kafka.server.policy.AlterConfigPolicy; diff --git a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java index 843f779826eb9..53c77af57a8f3 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java +++ b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java @@ -48,7 +48,7 @@ static int delta(boolean prev, boolean next) { private int offlinePartitionsChange = 0; private int partitionsWithoutPreferredLeaderChange = 0; private int uncleanLeaderElection = 0; - private int electionFromElrCounter = 0; + private int electionFromElr = 0; public int fencedBrokersChange() { return fencedBrokersChange; @@ -70,6 +70,14 @@ public int offlinePartitionsChange() { return offlinePartitionsChange; } + public int uncleanLeaderElection() { + return uncleanLeaderElection; + } + + public int electionFromElr() { + return electionFromElr; + } + public int partitionsWithoutPreferredLeaderChange() { return partitionsWithoutPreferredLeaderChange; } @@ -105,10 +113,13 @@ void handleTopicChange(TopicImage prev, TopicDelta topicDelta) { } else { for (Entry entry : topicDelta.partitionChanges().entrySet()) { int partitionId = entry.getKey(); + PartitionRegistration prevPartition = prev.partitions().get(partitionId); PartitionRegistration nextPartition = entry.getValue(); - handlePartitionChange(prev.partitions().get(partitionId), nextPartition); + handlePartitionChange(prevPartition, nextPartition); } } + topicDelta.partitionToUncleanLeaderElectionCount().forEach((partitionId, count) -> uncleanLeaderElection += count); + topicDelta.partitionToElrElectionCount().forEach((partitionId, count) -> electionFromElr += count); } void handlePartitionChange(PartitionRegistration prev, PartitionRegistration next) { @@ -127,15 +138,6 @@ void handlePartitionChange(PartitionRegistration prev, PartitionRegistration nex isPresent = true; isOffline = !next.hasLeader(); isWithoutPreferredLeader = !next.hasPreferredLeader(); - // take current all replicas as ISR if prev is null (new created partition), so we won't treat it as unclean election. - int[] prevIsr = prev != null ? prev.isr : next.replicas; - int[] prevElr = prev != null ? prev.elr : new int[]{}; - if (!PartitionRegistration.electionWasClean(next.leader, prevIsr, prevElr)) { - uncleanLeaderElection++; - } - if (PartitionRegistration.electionFromElr(next.leader, prevElr)) { - electionFromElrCounter++; - } } globalPartitionsChange += delta(wasPresent, isPresent); offlinePartitionsChange += delta(wasOffline, isOffline); @@ -168,9 +170,9 @@ void apply(ControllerMetadataMetrics metrics) { metrics.updateUncleanLeaderElection(uncleanLeaderElection); uncleanLeaderElection = 0; } - if (electionFromElrCounter > 0) { - metrics.updateElectionFromEligibleLeaderReplicasCount(electionFromElrCounter); - electionFromElrCounter = 0; + if (electionFromElr > 0) { + metrics.updateElectionFromEligibleLeaderReplicasCount(electionFromElr); + electionFromElr = 0; } } } diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java b/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java index fceca8f564e8d..fc14eb1c5b135 100644 --- a/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/MetadataImage.java @@ -20,7 +20,7 @@ import org.apache.kafka.image.node.MetadataImageNode; import org.apache.kafka.image.writer.ImageWriter; import org.apache.kafka.image.writer.ImageWriterOptions; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import java.util.Objects; diff --git a/metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java b/metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java index 9e0414d8c8273..e9cef3e70e6a1 100644 --- a/metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java +++ b/metadata/src/main/java/org/apache/kafka/image/MetadataProvenance.java @@ -17,7 +17,7 @@ package org.apache.kafka.image; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.Snapshots; import java.util.Objects; diff --git a/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java b/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java index fdb26e8e7df76..8fc84c5e0062d 100644 --- a/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java @@ -40,6 +40,8 @@ public final class TopicDelta { private final TopicImage image; private final Map partitionChanges = new HashMap<>(); + private Map partitionToUncleanLeaderElectionCount = new HashMap<>(); + private Map partitionToElrElectionCount = new HashMap<>(); public TopicDelta(TopicImage image) { this.image = image; @@ -69,20 +71,46 @@ public Uuid id() { return image.id(); } + public Map partitionToElrElectionCount() { + return partitionToElrElectionCount; + } + public Map partitionToUncleanLeaderElectionCount() { + return partitionToUncleanLeaderElectionCount; + } + public void replay(PartitionRecord record) { + int partitionId = record.partitionId(); + PartitionRegistration prevPartition = partitionChanges.get(partitionId); + if (prevPartition == null) { + prevPartition = image.partitions().get(partitionId); + } + if (prevPartition != null) { + updateElectionStats(partitionId, prevPartition, record.leader(), record.leaderRecoveryState()); + } partitionChanges.put(record.partitionId(), new PartitionRegistration(record)); } public void replay(PartitionChangeRecord record) { - PartitionRegistration partition = partitionChanges.get(record.partitionId()); - if (partition == null) { - partition = image.partitions().get(record.partitionId()); - if (partition == null) { + int partitionId = record.partitionId(); + PartitionRegistration prevPartition = partitionChanges.get(partitionId); + if (prevPartition == null) { + prevPartition = image.partitions().get(partitionId); + if (prevPartition == null) { throw new RuntimeException("Unable to find partition " + - record.topicId() + ":" + record.partitionId()); + record.topicId() + ":" + partitionId); } } - partitionChanges.put(record.partitionId(), partition.merge(record)); + updateElectionStats(partitionId, prevPartition, record.leader(), record.leaderRecoveryState()); + partitionChanges.put(record.partitionId(), prevPartition.merge(record)); + } + + private void updateElectionStats(int partitionId, PartitionRegistration prevPartition, int newLeader, byte newLeaderRecoveryState) { + if (PartitionRegistration.electionWasUnclean(newLeaderRecoveryState)) { + partitionToUncleanLeaderElectionCount.put(partitionId, partitionToUncleanLeaderElectionCount.getOrDefault(partitionId, 0) + 1); + } + if (Replicas.contains(prevPartition.elr, newLeader)) { + partitionToElrElectionCount.put(partitionId, partitionToElrElectionCount.getOrDefault(partitionId, 0) + 1); + } } public void replay(ClearElrRecord record) { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java index 13c1743ee5560..f61c4c70e0680 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java @@ -166,9 +166,6 @@ public Map resolveEffectiveTopicConfigs( ConfigDef configDef = configDefs.getOrDefault(ConfigResource.Type.TOPIC, EMPTY_CONFIG_DEF); HashMap effectiveConfigs = new HashMap<>(); for (ConfigDef.ConfigKey configKey : configDef.configKeys().values()) { - if (configKey.internalConfig && !dynamicTopicConfigs.containsKey(configKey.name)) { - continue; - } ConfigEntry entry = resolveEffectiveTopicConfig(configKey, staticNodeConfig, dynamicClusterConfigs, dynamicNodeConfigs, dynamicTopicConfigs); effectiveConfigs.put(entry.name(), entry); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java b/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java index a8f9e166e1f70..f74ceded9c155 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/PartitionRegistration.java @@ -33,7 +33,6 @@ import java.util.List; import java.util.Objects; -import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER; import static org.apache.kafka.metadata.LeaderConstants.NO_LEADER_CHANGE; @@ -165,12 +164,8 @@ public PartitionRegistration build() { public final int leaderEpoch; public final int partitionEpoch; - public static boolean electionWasClean(int newLeader, int[] isr, int[] elr) { - return newLeader == NO_LEADER || Replicas.contains(isr, newLeader) || Replicas.contains(elr, newLeader); - } - - public static boolean electionFromElr(int newLeader, int[] elr) { - return Replicas.contains(elr, newLeader); + public static boolean electionWasUnclean(byte leaderRecoveryState) { + return leaderRecoveryState == LeaderRecoveryState.RECOVERING.value(); } private static List checkDirectories(PartitionRecord record) { @@ -351,7 +346,7 @@ public String diff(PartitionRegistration prev) { } public void maybeLogPartitionChange(Logger log, String description, PartitionRegistration prev) { - if (!electionWasClean(leader, prev.isr, prev.elr)) { + if (electionWasUnclean(this.leaderRecoveryState.value())) { log.info("UNCLEAN partition change for {}: {}", description, diff(prev)); } else if (log.isDebugEnabled()) { log.debug("partition change for {}: {}", description, diff(prev)); diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java index 483defcc6af4e..eac143209dd6b 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java @@ -396,7 +396,7 @@ public void testCannotDowngradeBeforeMinimumKraftVersion() { build(); manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())); assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, - "Invalid update version 6 for feature metadata.version. Local controller 0 only supports versions 7-29")), + "Invalid update version 6 for feature metadata.version. Local controller 0 only supports versions 7-28")), manager.updateFeatures( Map.of(MetadataVersion.FEATURE_NAME, MetadataVersionTestUtils.IBP_3_3_IV2_FEATURE_LEVEL), Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), diff --git a/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java index 57f0df436c11d..aebdbca1730e1 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java @@ -23,8 +23,8 @@ import org.apache.kafka.common.metadata.NoOpRecord; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.Batch; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.timeline.TrackingSnapshotRegistry; import org.junit.jupiter.api.Test; diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java index e37a9634a8d99..0c094f19ca10b 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java @@ -88,10 +88,10 @@ static BrokerRegistrationRequestData.FeatureCollection brokerFeaturesPlusFeature .setName(MetadataVersion.FEATURE_NAME) .setMinSupportedVersion(minVersion.featureLevel()) .setMaxSupportedVersion(maxVersion.featureLevel())); - featureMaxVersions.entrySet().forEach(entry -> { + featureMaxVersions.forEach((key, value) -> { features.add(new BrokerRegistrationRequestData.Feature() - .setName(entry.getKey()) - .setMaxSupportedVersion(entry.getValue()) + .setName(key) + .setMaxSupportedVersion(value) .setMinSupportedVersion((short) 0)); }); return features; diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 39444fb738936..4a2e823242566 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -102,12 +102,12 @@ import org.apache.kafka.metalog.LocalLogManager; import org.apache.kafka.metalog.LocalLogManagerTestEnv; import org.apache.kafka.raft.Batch; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.EligibleLeaderReplicasVersion; import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.common.TopicIdPartition; import org.apache.kafka.snapshot.FileRawSnapshotReader; import org.apache.kafka.snapshot.Snapshots; diff --git a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java index 1964c93c8b36b..25bb5d852233a 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java @@ -24,12 +24,14 @@ import org.apache.kafka.image.TopicImage; import org.apache.kafka.image.writer.ImageWriterOptions; import org.apache.kafka.metadata.BrokerRegistration; +import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.apache.kafka.controller.metrics.ControllerMetricsTestUtils.FakePartitionRegistrationType.NON_PREFERRED_LEADER; @@ -170,4 +172,19 @@ public void testTopicChange() { assertEquals(0, changes.offlinePartitionsChange()); assertEquals(1, changes.partitionsWithoutPreferredLeaderChange()); } + + @Test + public void testTopicElectionResult() { + ControllerMetricsChanges changes = new ControllerMetricsChanges(); + TopicImage image = new TopicImage("foo", FOO_ID, Map.of()); + TopicDelta delta = new TopicDelta(image); + delta.replay(new PartitionRecord().setPartitionId(0).setLeader(0).setIsr(List.of(0, 1)).setReplicas(List.of(0, 1, 2))); + delta.replay(new PartitionChangeRecord().setPartitionId(0).setLeader(2).setIsr(List.of(2)).setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value())); + + delta.replay(new PartitionRecord().setPartitionId(1).setLeader(-1).setIsr(List.of()).setEligibleLeaderReplicas(List.of(0, 1)).setReplicas(List.of(0, 1, 2))); + delta.replay(new PartitionChangeRecord().setPartitionId(1).setLeader(1).setIsr(List.of(1)).setEligibleLeaderReplicas(List.of(0, 1))); + changes.handleTopicChange(image, delta); + assertEquals(1, changes.uncleanLeaderElection()); + assertEquals(1, changes.electionFromElr()); + } } diff --git a/metadata/src/test/java/org/apache/kafka/image/FakeSnapshotWriter.java b/metadata/src/test/java/org/apache/kafka/image/FakeSnapshotWriter.java index 43c999c04abe5..a524214bf0470 100644 --- a/metadata/src/test/java/org/apache/kafka/image/FakeSnapshotWriter.java +++ b/metadata/src/test/java/org/apache/kafka/image/FakeSnapshotWriter.java @@ -17,8 +17,8 @@ package org.apache.kafka.image; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.SnapshotWriter; import java.util.ArrayList; diff --git a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java index b91612d70ba45..d50c5a4bcac7e 100644 --- a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java @@ -39,6 +39,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -443,6 +444,43 @@ public void testClearElrRecordForNonExistTopic() { assertThrows(RuntimeException.class, () -> RecordTestUtils.replayAll(delta, topicRecords)); } + @Test + public void testTopicDeltaElectionStatsWithEmptyImage() { + TopicImage image = new TopicImage("topic", Uuid.randomUuid(), Collections.EMPTY_MAP); + TopicDelta delta = new TopicDelta(image); + delta.replay(new PartitionRecord().setPartitionId(0).setLeader(0).setIsr(List.of(0, 1)).setReplicas(List.of(0, 1, 2))); + delta.replay(new PartitionChangeRecord().setPartitionId(0).setLeader(2).setIsr(List.of(2)).setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value())); + assertEquals(1, delta.partitionToUncleanLeaderElectionCount().get(0)); + delta.replay(new PartitionChangeRecord().setPartitionId(0).setIsr(List.of(1, 2)).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value())); + assertEquals(1, delta.partitionToUncleanLeaderElectionCount().get(0)); + delta.replay(new PartitionChangeRecord().setPartitionId(0).setLeader(0).setIsr(List.of(0)).setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value())); + assertEquals(2, delta.partitionToUncleanLeaderElectionCount().get(0)); + delta.replay(new PartitionChangeRecord().setPartitionId(0).setLeader(1).setIsr(List.of(1)).setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value())); + assertEquals(3, delta.partitionToUncleanLeaderElectionCount().get(0)); + assertTrue(delta.partitionToElrElectionCount().isEmpty()); + + delta.replay(new PartitionRecord().setPartitionId(1).setLeader(0).setIsr(List.of(0, 1)).setReplicas(List.of(0, 1, 2))); + delta.replay(new PartitionChangeRecord().setPartitionId(1).setLeader(-1).setIsr(List.of()).setEligibleLeaderReplicas(List.of(0, 1))); + assertTrue(delta.partitionToElrElectionCount().isEmpty()); + delta.replay(new PartitionChangeRecord().setPartitionId(1).setLeader(1).setIsr(List.of(1)).setEligibleLeaderReplicas(List.of(0, 1))); + assertEquals(1, delta.partitionToElrElectionCount().get(1)); + } + + @Test + public void testTopicDeltaElectionStatsWithNonEmptyImage() { + TopicImage image = new TopicImage("topic", Uuid.randomUuid(), Map.of( + 0, new PartitionRegistration(new PartitionRecord().setPartitionId(0).setLeader(0).setIsr(List.of(0, 1)).setReplicas(List.of(0, 1, 2))), + 1, new PartitionRegistration(new PartitionRecord().setPartitionId(1).setLeader(-1).setIsr(List.of()).setEligibleLeaderReplicas(List.of(0, 1)).setReplicas(List.of(0, 1, 2))) + )); + TopicDelta delta = new TopicDelta(image); + delta.replay(new PartitionRecord().setPartitionId(0).setLeader(2).setIsr(List.of(2)).setReplicas(List.of(0, 1, 2)).setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value())); + assertEquals(1, delta.partitionToUncleanLeaderElectionCount().get(0)); + assertTrue(delta.partitionToElrElectionCount().isEmpty()); + + delta.replay(new PartitionChangeRecord().setPartitionId(1).setLeader(1).setIsr(List.of(1)).setEligibleLeaderReplicas(List.of(0, 1))); + assertEquals(1, delta.partitionToElrElectionCount().get(1)); + } + @Test public void testLocalReassignmentChanges() { int localId = 3; diff --git a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java index 8c4526de56210..beaeefdc38c2a 100644 --- a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java @@ -36,9 +36,9 @@ import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.ControlRecord; import org.apache.kafka.raft.LeaderAndEpoch; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.fault.MockFaultHandler; import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.test.TestUtils; diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java index 9647c4b031f67..cf123078eae39 100644 --- a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java @@ -21,10 +21,10 @@ import org.apache.kafka.image.FakeSnapshotWriter; import org.apache.kafka.image.MetadataImageTest; import org.apache.kafka.raft.LeaderAndEpoch; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RaftClient; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.SnapshotWriter; import org.junit.jupiter.api.Test; diff --git a/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java index c0de7d2125d35..8089812358765 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java @@ -55,18 +55,9 @@ @Timeout(40) public class PartitionRegistrationTest { @Test - public void testElectionWasClean() { - assertTrue(PartitionRegistration.electionWasClean(1, new int[]{1, 2}, new int[]{})); - assertFalse(PartitionRegistration.electionWasClean(1, new int[]{0, 2}, new int[]{})); - assertFalse(PartitionRegistration.electionWasClean(1, new int[]{}, new int[]{3, 4})); - assertTrue(PartitionRegistration.electionWasClean(3, new int[]{1, 2, 3, 4, 5, 6}, new int[]{})); - assertTrue(PartitionRegistration.electionWasClean(3, new int[]{}, new int[]{1, 2, 3})); - } - - @Test - public void testEligibleLeaderReplicasElection() { - assertTrue(PartitionRegistration.electionFromElr(1, new int[]{1, 2})); - assertFalse(PartitionRegistration.electionFromElr(1, new int[]{0, 2})); + public void testElectionWasUnclean() { + assertFalse(PartitionRegistration.electionWasUnclean(LeaderRecoveryState.RECOVERED.value())); + assertTrue(PartitionRegistration.electionWasUnclean(LeaderRecoveryState.RECOVERING.value())); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java index ebb6f5869690f..2eeeab2259a21 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java @@ -192,6 +192,39 @@ public void testIgnoreFormatted() throws Exception { } } + @Test + public void testStandaloneWithIgnoreFormatted() throws Exception { + try (TestEnv testEnv = new TestEnv(1)) { + FormatterContext formatter1 = testEnv.newFormatter(); + String originalDirectoryId = Uuid.randomUuid().toString(); + String newDirectoryId = Uuid.randomUuid().toString(); + formatter1.formatter + .setInitialControllers(DynamicVoters.parse("1@localhost:8020:" + originalDirectoryId)) + .run(); + assertEquals("Formatting dynamic metadata voter directory " + testEnv.directory(0) + + " with metadata.version " + MetadataVersion.latestProduction() + ".", + formatter1.output().trim()); + assertMetadataDirectoryId(testEnv, Uuid.fromString(originalDirectoryId)); + + FormatterContext formatter2 = testEnv.newFormatter(); + formatter2.formatter + .setIgnoreFormatted(true) + .setInitialControllers(DynamicVoters.parse("1@localhost:8020:" + newDirectoryId)) + .run(); + assertEquals("All of the log directories are already formatted.", + formatter2.output().trim()); + assertMetadataDirectoryId(testEnv, Uuid.fromString(originalDirectoryId)); + } + } + + private void assertMetadataDirectoryId(TestEnv testEnv, Uuid expectedDirectoryId) throws Exception { + MetaPropertiesEnsemble ensemble = new MetaPropertiesEnsemble.Loader(). + addLogDirs(testEnv.directories). + load(); + MetaProperties logDirProps0 = ensemble.logDirProps().get(testEnv.directory(0)); + assertEquals(expectedDirectoryId, logDirProps0.directoryId().get()); + } + @Test public void testOneDirectoryFormattedAndOthersNotFormatted() throws Exception { try (TestEnv testEnv = new TestEnv(2)) { diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java index 0d5eb8dba0761..c13f90f57faa2 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java @@ -27,13 +27,13 @@ import org.apache.kafka.queue.KafkaEventQueue; import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.LeaderAndEpoch; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RaftClient; import org.apache.kafka.raft.errors.BufferAllocationException; import org.apache.kafka.raft.errors.NotLeaderException; import org.apache.kafka.raft.internals.MemoryBatchReader; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.MockRawSnapshotReader; import org.apache.kafka.snapshot.MockRawSnapshotWriter; import org.apache.kafka.snapshot.RawSnapshotReader; @@ -395,6 +395,8 @@ void beginShutdown() { } } + private final LogContext logContext; + private final Logger log; /** @@ -448,6 +450,7 @@ public LocalLogManager(LogContext logContext, SharedLogData shared, String threadNamePrefix, KRaftVersion lastKRaftVersion) { + this.logContext = logContext; this.log = logContext.logger(LocalLogManager.class); this.nodeId = nodeId; this.shared = shared; @@ -477,7 +480,8 @@ private void scheduleLogCheck() { new MetadataRecordSerde(), BufferSupplier.create(), Integer.MAX_VALUE, - true + true, + logContext ) ); } diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 5005f509cbd1f..40864bb08c998 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -86,6 +86,7 @@ import org.apache.kafka.raft.internals.ThresholdPurgatory; import org.apache.kafka.raft.internals.UpdateVoterHandler; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.NotifyingRawSnapshotWriter; import org.apache.kafka.snapshot.RawSnapshotReader; @@ -443,7 +444,8 @@ private Optional> latestSnapshot() { serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES, - true /* Validate batch CRC*/ + true, /* Validate batch CRC*/ + logContext ) ); } @@ -878,8 +880,9 @@ private VoteResponseData handleVoteRequest( Optional voterKey = RaftUtil.voteRequestVoterKey(request, partitionRequest); if (!isValidVoterKey(voterKey)) { logger.info( - "A replica sent a voter key ({}) in the VOTE request that doesn't match the " + + "A replica {} sent a voter key ({}) in the VOTE request that doesn't match the " + "local key ({}, {}); rejecting the vote", + replicaId, voterKey, nodeId, nodeDirectoryId @@ -3896,7 +3899,8 @@ private void fireHandleCommit(long baseOffset, Records records) { BufferSupplier.create(), MAX_BATCH_SIZE_BYTES, this, - true /* Validate batch CRC*/ + true, /* Validate batch CRC*/ + logContext ) ); } diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index 805d16d361c7b..b0a6f9f045ad6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -52,6 +52,10 @@ public class MetadataLogConfig { "configuration. The Kafka node will generate a snapshot when either the maximum time interval is reached or the " + "maximum bytes limit is reached."; + public static final String METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG = "metadata.log.segment.min.bytes"; + public static final String METADATA_LOG_SEGMENT_MIN_BYTES_DOC = "Override the minimum size for a single metadata log file. This should be used for testing only."; + public static final int METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT = 8 * 1024 * 1024; + public static final String METADATA_LOG_SEGMENT_BYTES_CONFIG = "metadata.log.segment.bytes"; public static final String METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file."; public static final int METADATA_LOG_SEGMENT_BYTES_DEFAULT = 1024 * 1024 * 1024; @@ -81,13 +85,14 @@ public class MetadataLogConfig { .define(METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC) .define(METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, METADATA_LOG_DIR_DOC) .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) + .defineInternal(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_MIN_BYTES_DOC) .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); private final int logSegmentBytes; - private final Integer internalLogSegmentBytes; + private final int logSegmentMinBytes; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -98,6 +103,7 @@ public class MetadataLogConfig { /** * Configuration for the metadata log * @param logSegmentBytes The maximum size of a single metadata log file + * @param logSegmentMinBytes The minimum size of a single metadata log file * @param logSegmentMillis The maximum time before a new metadata log file is rolled out * @param retentionMaxBytes The size of the metadata log and snapshots before deleting old snapshots and log files * @param retentionMillis The time to keep a metadata log file or snapshot before deleting it @@ -106,6 +112,7 @@ public class MetadataLogConfig { * @param deleteDelayMillis The amount of time to wait before deleting a file from the filesystem */ public MetadataLogConfig(int logSegmentBytes, + int logSegmentMinBytes, long logSegmentMillis, long retentionMaxBytes, long retentionMillis, @@ -113,7 +120,7 @@ public MetadataLogConfig(int logSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = logSegmentBytes; - this.internalLogSegmentBytes = logSegmentBytes; + this.logSegmentMinBytes = logSegmentMinBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; @@ -124,7 +131,7 @@ public MetadataLogConfig(int logSegmentBytes, public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); - this.internalLogSegmentBytes = config.getInt(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG); + this.logSegmentMinBytes = config.getInt(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); @@ -134,12 +141,11 @@ public MetadataLogConfig(AbstractConfig config) { } public int logSegmentBytes() { - if (internalLogSegmentBytes == null) return logSegmentBytes; - return internalLogSegmentBytes; + return logSegmentBytes; } - - public Integer internalLogSegmentBytes() { - return internalLogSegmentBytes; + + public int logSegmentMinBytes() { + return logSegmentMinBytes; } public long logSegmentMillis() { diff --git a/raft/src/main/java/org/apache/kafka/raft/OffsetAndEpoch.java b/raft/src/main/java/org/apache/kafka/raft/OffsetAndEpoch.java deleted file mode 100644 index cba6108fd5d8a..0000000000000 --- a/raft/src/main/java/org/apache/kafka/raft/OffsetAndEpoch.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * 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 - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.raft; - -public record OffsetAndEpoch(long offset, int epoch) implements Comparable { - - @Override - public int compareTo(OffsetAndEpoch o) { - if (epoch == o.epoch) - return Long.compare(offset, o.offset); - return Integer.compare(epoch, o.epoch); - } -} diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index d0450a9b0470c..d4b81bd3ce7e6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -23,6 +23,7 @@ import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.KRaftControlRecordStateMachine; import org.apache.kafka.raft.internals.KafkaRaftMetrics; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.slf4j.Logger; diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java index 90131ce457573..273a2ec56c57c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java @@ -19,6 +19,7 @@ import org.apache.kafka.raft.errors.BufferAllocationException; import org.apache.kafka.raft.errors.NotLeaderException; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java index 80ad513437017..caa087378c568 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java @@ -42,6 +42,7 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.server.common.OffsetAndEpoch; import java.net.InetSocketAddress; import java.util.Collection; diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java index 8f5ba31a45d30..0bbd351ff4db4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.record.Records; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; diff --git a/raft/src/main/java/org/apache/kafka/raft/ValidOffsetAndEpoch.java b/raft/src/main/java/org/apache/kafka/raft/ValidOffsetAndEpoch.java index 1de7bffb9ab9c..96baf3b963644 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ValidOffsetAndEpoch.java +++ b/raft/src/main/java/org/apache/kafka/raft/ValidOffsetAndEpoch.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.server.common.OffsetAndEpoch; + import java.util.Objects; public final class ValidOffsetAndEpoch { diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java index a8768f902a822..d9cd47a3a2790 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachine.java @@ -52,6 +52,7 @@ public final class KRaftControlRecordStateMachine { private static final long STARTING_NEXT_OFFSET = -1; private static final long SMALLEST_LOG_OFFSET = 0; + private final LogContext logContext; private final ReplicatedLog log; private final RecordSerde serde; private final BufferSupplier bufferSupplier; @@ -95,12 +96,13 @@ public KRaftControlRecordStateMachine( KafkaRaftMetrics kafkaRaftMetrics, ExternalKRaftMetrics externalKRaftMetrics ) { + this.logContext = logContext; this.log = log; this.voterSetHistory = new VoterSetHistory(staticVoterSet, logContext); this.serde = serde; this.bufferSupplier = bufferSupplier; this.maxBatchSizeBytes = maxBatchSizeBytes; - this.logger = logContext.logger(this.getClass()); + this.logger = logContext.logger(getClass()); this.kafkaRaftMetrics = kafkaRaftMetrics; this.externalKRaftMetrics = externalKRaftMetrics; this.staticVoterSet = staticVoterSet; @@ -237,7 +239,8 @@ private void maybeLoadLog() { serde, bufferSupplier, maxBatchSizeBytes, - true // Validate batch CRC + true, // Validate batch CRC + logContext ) ) { while (iterator.hasNext()) { @@ -266,7 +269,8 @@ private void maybeLoadSnapshot() { serde, bufferSupplier, maxBatchSizeBytes, - true // Validate batch CRC + true, // Validate batch CRC + logContext ) ) { logger.info( diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java index 124c1a307fc1c..87c5b217d8ed4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/KafkaRaftMetrics.java @@ -26,9 +26,9 @@ import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.metrics.stats.WindowedSum; import org.apache.kafka.raft.LogOffsetMetadata; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.QuorumState; import org.apache.kafka.raft.ReplicaKey; +import org.apache.kafka.server.common.OffsetAndEpoch; import java.util.List; import java.util.OptionalLong; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java index 64572b6bc4991..404c56a8b8f81 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.apache.kafka.server.common.serialization.RecordSerde; @@ -102,11 +103,12 @@ public static RecordsBatchReader of( BufferSupplier bufferSupplier, int maxBatchSize, CloseListener> closeListener, - boolean doCrcValidation + boolean doCrcValidation, + LogContext logContext ) { return new RecordsBatchReader<>( baseOffset, - new RecordsIterator<>(records, serde, bufferSupplier, maxBatchSize, doCrcValidation), + new RecordsIterator<>(records, serde, bufferSupplier, maxBatchSize, doCrcValidation, logContext), closeListener ); } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java index 60c140fe2ded0..b912d748bcad3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java @@ -27,11 +27,14 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.ByteUtils; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.ControlRecord; import org.apache.kafka.server.common.serialization.RecordSerde; +import org.slf4j.Logger; + import java.io.IOException; import java.io.InputStream; import java.io.UncheckedIOException; @@ -45,6 +48,7 @@ import java.util.function.BiFunction; public final class RecordsIterator implements Iterator>, AutoCloseable { + private final Logger logger; private final Records records; private final RecordSerde serde; private final BufferSupplier bufferSupplier; @@ -73,13 +77,15 @@ public RecordsIterator( RecordSerde serde, BufferSupplier bufferSupplier, int batchSize, - boolean doCrcValidation + boolean doCrcValidation, + LogContext logContext ) { this.records = records; this.serde = serde; this.bufferSupplier = bufferSupplier; this.batchSize = Math.max(batchSize, Records.HEADER_SIZE_UP_TO_MAGIC); this.doCrcValidation = doCrcValidation; + this.logger = logContext.logger(getClass()); } @Override @@ -143,9 +149,15 @@ private MemoryRecords createMemoryRecords(FileRecords fileRecords) { MemoryRecords memoryRecords = readFileRecords(fileRecords, buffer); // firstBatchSize() is always non-null because the minimum buffer is HEADER_SIZE_UP_TO_MAGIC. - if (memoryRecords.firstBatchSize() <= buffer.remaining()) { + int firstBatchSize = memoryRecords.firstBatchSize(); + if (firstBatchSize <= buffer.remaining()) { return memoryRecords; } else { + logger.info( + "Creating a new buffer; previous buffer {} cannot fit at least {} bytes", + buffer, + firstBatchSize + ); // Not enough bytes read; create a bigger buffer ByteBuffer newBuffer = bufferSupplier.get(memoryRecords.firstBatchSize()); allocatedBuffer = Optional.of(newBuffer); diff --git a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java index 7d2955dbecf84..03b41a3446c84 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.UnalignedRecords; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import java.io.IOException; import java.io.UncheckedIOException; diff --git a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java index 1f4af69498102..6dc9e1c8c792c 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import java.io.IOException; import java.io.UncheckedIOException; diff --git a/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java index 76b27f45c82f6..0eaa848278dfb 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriter.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.UnalignedMemoryRecords; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import java.util.function.Consumer; diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java index e303a5889f1d5..5c6d53adc2280 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.UnalignedRecords; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; /** * Interface for reading snapshots as a sequence of records. diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java index 07d8271e953f6..c11972c15be3a 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.UnalignedMemoryRecords; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; /** * Interface for writing snapshot as a sequence of records. diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java index fc815621d839d..b85b7a19eb9b1 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java @@ -20,9 +20,10 @@ import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.record.ControlRecordType; import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.Batch; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.internals.RecordsIterator; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.common.serialization.RecordSerde; import java.util.NoSuchElementException; @@ -112,11 +113,12 @@ public static RecordsSnapshotReader of( RecordSerde serde, BufferSupplier bufferSupplier, int maxBatchSize, - boolean doCrcValidation + boolean doCrcValidation, + LogContext logContext ) { return new RecordsSnapshotReader<>( snapshot.snapshotId(), - new RecordsIterator<>(snapshot.records(), serde, bufferSupplier, maxBatchSize, doCrcValidation) + new RecordsIterator<>(snapshot.records(), serde, bufferSupplier, maxBatchSize, doCrcValidation, logContext) ); } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java index 2162fc673c0dc..09f88306638e7 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotWriter.java @@ -27,11 +27,11 @@ import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.VoterSet; import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.BatchAccumulator.CompletedBatch; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.common.serialization.RecordSerde; import java.util.List; diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotPath.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotPath.java index 58ca000595b5c..f013fc7971259 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotPath.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotPath.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.snapshot; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import java.nio.file.Path; diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java index ffc0cf41b06d6..3a77d01f03525 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java @@ -17,7 +17,7 @@ package org.apache.kafka.snapshot; import org.apache.kafka.raft.Batch; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import java.util.Iterator; diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java index 117bcdda33613..2ccb1d758f44d 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java @@ -17,7 +17,7 @@ package org.apache.kafka.snapshot; import org.apache.kafka.common.message.SnapshotFooterRecord; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import java.util.List; diff --git a/raft/src/main/java/org/apache/kafka/snapshot/Snapshots.java b/raft/src/main/java/org/apache/kafka/snapshot/Snapshots.java index 716c6e00f1677..3a78be9c53469 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/Snapshots.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/Snapshots.java @@ -17,10 +17,11 @@ package org.apache.kafka.snapshot; import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.KafkaRaftClient; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.internals.IdentitySerde; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -159,23 +160,25 @@ public static Path markForDelete(Path logDir, OffsetAndEpoch snapshotId) { } } - public static long lastContainedLogTimestamp(RawSnapshotReader reader) { - try (RecordsSnapshotReader recordsSnapshotReader = - RecordsSnapshotReader.of( - reader, - IdentitySerde.INSTANCE, - new BufferSupplier.GrowableBufferSupplier(), - KafkaRaftClient.MAX_BATCH_SIZE_BYTES, - true - ) + public static long lastContainedLogTimestamp(RawSnapshotReader reader, LogContext logContext) { + try (var bufferSupplier = new BufferSupplier.GrowableBufferSupplier(); + RecordsSnapshotReader recordsSnapshotReader = + RecordsSnapshotReader.of( + reader, + IdentitySerde.INSTANCE, + bufferSupplier, + KafkaRaftClient.MAX_BATCH_SIZE_BYTES, + true, + logContext + ) ) { return recordsSnapshotReader.lastContainedLogTimestamp(); } } - public static long lastContainedLogTimestamp(Path logDir, OffsetAndEpoch snapshotId) { + public static long lastContainedLogTimestamp(Path logDir, OffsetAndEpoch snapshotId, LogContext logContext) { try (FileRawSnapshotReader reader = FileRawSnapshotReader.open(logDir, snapshotId)) { - return lastContainedLogTimestamp(reader); + return lastContainedLogTimestamp(reader, logContext); } } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java index f68b4b6f06112..e56b9c94b49ae 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java @@ -50,6 +50,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index dc04f85d0b81b..cdc351d318141 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -41,6 +41,7 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.snapshot.RecordsSnapshotReader; @@ -127,7 +128,8 @@ public void testLeaderWritesBootstrapRecords() throws Exception { context.serde, BufferSupplier.NO_CACHING, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, - false + false, + new LogContext() ) ) { SnapshotWriterReaderTest.assertControlSnapshot(expectedBootstrapRecords, reader); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index a0d45a349ee80..bc4be592a2082 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.requests.FetchSnapshotRequest; import org.apache.kafka.raft.internals.StringSerde; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; import org.apache.kafka.snapshot.RecordsSnapshotWriter; diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLog.java b/raft/src/test/java/org/apache/kafka/raft/MockLog.java index 209accdf7a2b0..6f0968202b747 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLog.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLog.java @@ -30,6 +30,7 @@ import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.MockRawSnapshotReader; import org.apache.kafka.snapshot.MockRawSnapshotWriter; import org.apache.kafka.snapshot.RawSnapshotReader; diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java index 3abfe5a1b120a..21d253a375c64 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java @@ -32,6 +32,7 @@ import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index 8376000f4ad06..76d942d2bd98b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.raft.internals.KafkaRaftMetrics; import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index c104af0ce7606..9f0a5084326c8 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -67,6 +67,7 @@ import org.apache.kafka.raft.internals.StringSerde; import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.RecordsSnapshotWriter; import org.apache.kafka.snapshot.SnapshotReader; diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index 7f478e38543ba..3f8d96bfc32b8 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -927,6 +927,7 @@ void start(int nodeId) { } private static class RaftNode { + final LogContext logContext; final int nodeId; final KafkaRaftClient client; final MockLog log; @@ -948,6 +949,7 @@ private RaftNode( Random random, RecordSerde intSerde ) { + this.logContext = logContext; this.nodeId = nodeId; this.client = client; this.log = log; @@ -997,6 +999,10 @@ public String toString() { logEndOffset() ); } + + LogContext logContext() { + return logContext; + } } private static class InflightRequest { @@ -1335,7 +1341,8 @@ private void assertCommittedData(RaftNode node) { node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE, - true + true, + node.logContext() ) ) { // Since the state machine is only on e value we only expect one data record in the snapshot diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java index e440be1dcb968..89faf33872161 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java @@ -51,6 +51,7 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.server.common.OffsetAndEpoch; import com.fasterxml.jackson.databind.JsonNode; diff --git a/raft/src/test/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/test/java/org/apache/kafka/raft/ReplicatedCounter.java index ac4da6d148653..d877a7ec30884 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/test/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.errors.NotLeaderException; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; diff --git a/raft/src/test/java/org/apache/kafka/raft/ValidOffsetAndEpochTest.java b/raft/src/test/java/org/apache/kafka/raft/ValidOffsetAndEpochTest.java index 4c5463fef0c5d..05b044ad8c5af 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ValidOffsetAndEpochTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ValidOffsetAndEpochTest.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.server.common.OffsetAndEpoch; + import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java index bab2f6af0e12e..2f7f9f2f508f4 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KRaftControlRecordStateMachineTest.java @@ -26,10 +26,10 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.ExternalKRaftMetrics; import org.apache.kafka.raft.MockLog; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.VoterSet; import org.apache.kafka.raft.VoterSetTest; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.RecordsSnapshotWriter; diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java index f531e6f6fe72b..c6a53742f7c0d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java @@ -24,13 +24,13 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.LogOffsetMetadata; import org.apache.kafka.raft.MockQuorumStateStore; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.QuorumState; import org.apache.kafka.raft.ReplicaKey; import org.apache.kafka.raft.VoterSet; import org.apache.kafka.raft.VoterSetTest; import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.params.ParameterizedTest; diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java index 6110be111c502..a32575e93b2d3 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.ControlRecord; import org.apache.kafka.raft.internals.RecordsIteratorTest.TestBatch; @@ -88,7 +89,8 @@ public void testLeaderChangeControlBatch() { BufferSupplier.NO_CACHING, MAX_BATCH_BYTES, ignore -> { }, - true + true, + new LogContext() ) ) { assertTrue(reader.hasNext()); @@ -128,7 +130,8 @@ private void testBatchReader( bufferSupplier, MAX_BATCH_BYTES, closeListener, - true + true, + new LogContext() ); try { for (TestBatch batch : expectedBatches) { diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java index 8fd8925ae54ea..8016a1e5381d9 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java @@ -37,13 +37,14 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.ControlRecord; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.VoterSet; import org.apache.kafka.raft.VoterSetTest; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.MockRawSnapshotWriter; import org.apache.kafka.snapshot.RecordsSnapshotWriter; @@ -322,7 +323,14 @@ static RecordsIterator createIterator( BufferSupplier bufferSupplier, boolean validateCrc ) { - return new RecordsIterator<>(records, STRING_SERDE, bufferSupplier, Records.HEADER_SIZE_UP_TO_MAGIC, validateCrc); + return new RecordsIterator<>( + records, + STRING_SERDE, + bufferSupplier, + Records.HEADER_SIZE_UP_TO_MAGIC, + validateCrc, + new LogContext() + ); } static BufferSupplier mockBufferSupplier(Set buffers) { diff --git a/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java b/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java index edd812e8dd606..990e7e9720af4 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java @@ -25,7 +25,7 @@ import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.utils.BufferSupplier.GrowableBufferSupplier; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; diff --git a/raft/src/test/java/org/apache/kafka/snapshot/MockRawSnapshotReader.java b/raft/src/test/java/org/apache/kafka/snapshot/MockRawSnapshotReader.java index b1c84c3d4593e..fb5df264ca7e3 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/MockRawSnapshotReader.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/MockRawSnapshotReader.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.record.UnalignedRecords; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import java.nio.ByteBuffer; diff --git a/raft/src/test/java/org/apache/kafka/snapshot/MockRawSnapshotWriter.java b/raft/src/test/java/org/apache/kafka/snapshot/MockRawSnapshotWriter.java index edc33ee4e2974..3b99178a1fd20 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/MockRawSnapshotWriter.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/MockRawSnapshotWriter.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.common.utils.ByteBufferOutputStream; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import java.nio.ByteBuffer; import java.util.function.Consumer; diff --git a/raft/src/test/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriterTest.java b/raft/src/test/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriterTest.java index 50c0fa0408b80..1cb4a09288190 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriterTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/NotifyingRawSnapshotWriterTest.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.UnalignedMemoryRecords; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.junit.jupiter.api.Test; diff --git a/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java index 3934b9f96f2ec..35d126dba979a 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/RecordsSnapshotWriterTest.java @@ -22,13 +22,14 @@ import org.apache.kafka.common.message.SnapshotHeaderRecord; import org.apache.kafka.common.record.ControlRecordType; import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.raft.Batch; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.VoterSet; import org.apache.kafka.raft.VoterSetTest; import org.apache.kafka.raft.internals.StringSerde; import org.apache.kafka.server.common.KRaftVersion; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.common.serialization.RecordSerde; import org.junit.jupiter.api.Test; @@ -68,7 +69,8 @@ void testBuilderKRaftVersion0() { STRING_SERDE, BufferSupplier.NO_CACHING, maxBatchSize, - true + true, + new LogContext() ) ) { // Consume the control record batch @@ -140,7 +142,8 @@ void testKBuilderRaftVersion1WithVoterSet() { STRING_SERDE, BufferSupplier.NO_CACHING, maxBatchSize, - true + true, + new LogContext() ) ) { // Consume the control record batch @@ -197,7 +200,8 @@ void testBuilderKRaftVersion1WithoutVoterSet() { STRING_SERDE, BufferSupplier.NO_CACHING, maxBatchSize, - true + true, + new LogContext() ) ) { // Consume the control record batch diff --git a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java index f8350222d5e73..3464c43d185f2 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java @@ -23,11 +23,12 @@ import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.BufferSupplier.GrowableBufferSupplier; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.ControlRecord; -import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RaftClientTestContext; import org.apache.kafka.raft.internals.StringSerde; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.junit.jupiter.api.Test; @@ -111,7 +112,7 @@ public void testWritingSnapshot() throws Exception { assertEquals((recordsPerBatch * batches) + delimiterCount, recordCount); assertDataSnapshot(expected, reader); - assertEquals(magicTimestamp, Snapshots.lastContainedLogTimestamp(snapshot)); + assertEquals(magicTimestamp, Snapshots.lastContainedLogTimestamp(snapshot, new LogContext())); } } @@ -195,7 +196,8 @@ private SnapshotReader readSnapshot( context.serde, BufferSupplier.create(), maxBatchSize, - true + true, + new LogContext() ); } @@ -249,7 +251,14 @@ record = records.next(); public static void assertDataSnapshot(List> batches, RawSnapshotReader reader) { assertDataSnapshot( batches, - RecordsSnapshotReader.of(reader, new StringSerde(), BufferSupplier.create(), Integer.MAX_VALUE, true) + RecordsSnapshotReader.of( + reader, + new StringSerde(), + BufferSupplier.create(), + Integer.MAX_VALUE, + true, + new LogContext() + ) ); } diff --git a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotsTest.java b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotsTest.java index 5c96b044b425f..36c537ddb5f43 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotsTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotsTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.snapshot; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; diff --git a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java index 93be50cbb1375..9e9ed6f8447e6 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/MetadataVersion.java @@ -112,13 +112,12 @@ public enum MetadataVersion { // // Enables ELR by default for new clusters (KIP-966). + // Share groups are preview in 4.1 (KIP-932). + // Streams groups are early access in 4.1 (KIP-1071). IBP_4_1_IV0(26, "4.1", "IV0", false), - // Enables share groups. Note, share groups are for preview only in 4.1. (KIP-932). - IBP_4_1_IV1(27, "4.1", "IV1", false), - // Insert any additional IBP_4_1_IVx versions above this comment, and bump the feature level of - // IBP_4_2_IV0 accordingly. When 4.2 development begins, IBP_4_2_IV0 will cease to be + // IBP_4_2_IVx accordingly. When 4.2 development begins, IBP_4_2_IV0 will cease to be // a placeholder. // Enables share groups by default for new clusters (KIP-932). @@ -127,7 +126,7 @@ public enum MetadataVersion { // *** SHARE GROUPS BECOME PRODUCTION-READY IN THE FUTURE. ITS DEFINITION ALLOWS A SHARE *** // *** GROUPS FEATURE TO BE DEFINED IN 4.1 BUT TURNED OFF BY DEFAULT, ABLE TO BE TURNED ON *** // *** DYNAMICALLY TO TRY OUT THE PREVIEW CAPABILITY. *** - IBP_4_2_IV0(28, "4.2", "IV0", false), + IBP_4_2_IV0(27, "4.2", "IV0", false), // Enables "streams" groups by default for new clusters (KIP-1071). // @@ -135,7 +134,7 @@ public enum MetadataVersion { // *** STREAMS GROUPS BECOME PRODUCTION-READY IN THE FUTURE. ITS DEFINITION ALLOWS A STREAMS *** // *** GROUPS FEATURE TO BE DEFINED IN 4.1 BUT TURNED OFF BY DEFAULT, ABLE TO BE TURNED ON *** // *** DYNAMICALLY TO TRY OUT THE EARLY ACCESS CAPABILITY. *** - IBP_4_2_IV1(29, "4.2", "IV1", false); + IBP_4_2_IV1(28, "4.2", "IV1", false); // NOTES when adding a new version: // Update the default version in @ClusterTest annotation to point to the latest version diff --git a/server-common/src/main/java/org/apache/kafka/server/common/OffsetAndEpoch.java b/server-common/src/main/java/org/apache/kafka/server/common/OffsetAndEpoch.java index a5953ae70bc00..a5f8c007a6f6e 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/OffsetAndEpoch.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/OffsetAndEpoch.java @@ -16,40 +16,11 @@ */ package org.apache.kafka.server.common; -public class OffsetAndEpoch { - private final long offset; - private final int leaderEpoch; - - public OffsetAndEpoch(long offset, int leaderEpoch) { - this.offset = offset; - this.leaderEpoch = leaderEpoch; - } - - public long offset() { - return offset; - } - - public int leaderEpoch() { - return leaderEpoch; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - OffsetAndEpoch that = (OffsetAndEpoch) o; - return offset == that.offset && leaderEpoch == that.leaderEpoch; - } - - @Override - public int hashCode() { - int result = leaderEpoch; - result = 31 * result + Long.hashCode(offset); - return result; - } - +public record OffsetAndEpoch(long offset, int epoch) implements Comparable { @Override - public String toString() { - return "(offset=" + offset + ", leaderEpoch=" + leaderEpoch + ")"; + public int compareTo(OffsetAndEpoch o) { + if (epoch == o.epoch) + return Long.compare(offset, o.offset); + return Integer.compare(epoch, o.epoch); } -} +} \ No newline at end of file diff --git a/server-common/src/main/java/org/apache/kafka/server/common/ShareVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/ShareVersion.java index ccbaa3e65add1..ef3513d5dc69b 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/ShareVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/ShareVersion.java @@ -25,7 +25,7 @@ public enum ShareVersion implements FeatureVersion { // Version 1 enables share groups (KIP-932). // This is a preview in 4.1, and production-ready in 4.2. - SV_1(1, MetadataVersion.IBP_4_2_IV0, Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_4_1_IV1.featureLevel())); + SV_1(1, MetadataVersion.IBP_4_2_IV0, Map.of()); public static final String FEATURE_NAME = "share.version"; diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 16b5c7a99a39c..2c2d44ef012e5 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -43,9 +43,6 @@ public class ServerLogConfigs { public static final String LOG_SEGMENT_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); public static final String LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file"; - public static final String INTERNAL_LOG_SEGMENT_BYTES_CONFIG = "internal." + ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); - public static final String INTERNAL_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; - public static final String LOG_ROLL_TIME_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_MS_CONFIG); public static final String LOG_ROLL_TIME_HOURS_CONFIG = LOG_PREFIX + "roll.hours"; public static final String LOG_ROLL_TIME_MILLIS_DOC = "The maximum time before a new log segment is rolled out (in milliseconds). If not set, the value in " + LOG_ROLL_TIME_HOURS_CONFIG + " is used"; diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java index 1b4a075651545..ae8b8c317c3ae 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java @@ -486,6 +486,7 @@ ReadShareGroupStateSummaryResult readSummaryResponsesToResult( partitionResult.partition(), partitionResult.stateEpoch(), partitionResult.startOffset(), + partitionResult.leaderEpoch(), partitionResult.errorCode(), partitionResult.errorMessage())) .toList(); @@ -495,6 +496,7 @@ ReadShareGroupStateSummaryResult readSummaryResponsesToResult( partition, -1, -1, + -1, Errors.UNKNOWN_SERVER_ERROR.code(), // No specific public error code exists for InterruptedException / ExecutionException "Error reading state from share coordinator: " + e.getMessage())); } diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/NoOpStatePersister.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/NoOpStatePersister.java index 2814fc3aa82c5..908891dd463bc 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/NoOpStatePersister.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/NoOpStatePersister.java @@ -92,7 +92,8 @@ public CompletableFuture readSummary(ReadShare for (TopicData topicData : reqData.topicsData()) { resultArgs.add(new TopicData<>(topicData.topicId(), topicData.partitions().stream(). map(partitionIdData -> PartitionFactory.newPartitionStateSummaryData( - partitionIdData.partition(), PartitionFactory.DEFAULT_STATE_EPOCH, PartitionFactory.UNINITIALIZED_START_OFFSET, PartitionFactory.DEFAULT_ERROR_CODE, PartitionFactory.DEFAULT_ERR_MESSAGE)) + partitionIdData.partition(), PartitionFactory.DEFAULT_STATE_EPOCH, PartitionFactory.UNINITIALIZED_START_OFFSET, + PartitionFactory.DEFAULT_LEADER_EPOCH, PartitionFactory.DEFAULT_ERROR_CODE, PartitionFactory.DEFAULT_ERR_MESSAGE)) .collect(Collectors.toList()))); } return CompletableFuture.completedFuture(new ReadShareGroupStateSummaryResult.Builder().setTopicsData(resultArgs).build()); diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionFactory.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionFactory.java index 009eb9cccc149..78a6902a170a9 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionFactory.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionFactory.java @@ -47,12 +47,8 @@ public static PartitionErrorData newPartitionErrorData(int partition, short erro return new PartitionData(partition, DEFAULT_STATE_EPOCH, UNINITIALIZED_START_OFFSET, errorCode, errorMessage, DEFAULT_LEADER_EPOCH, null); } - public static PartitionStateErrorData newPartitionStateErrorData(int partition, int stateEpoch, long startOffset, short errorCode, String errorMessage) { - return new PartitionData(partition, stateEpoch, startOffset, errorCode, errorMessage, DEFAULT_LEADER_EPOCH, null); - } - - public static PartitionStateSummaryData newPartitionStateSummaryData(int partition, int stateEpoch, long startOffset, short errorCode, String errorMessage) { - return new PartitionData(partition, stateEpoch, startOffset, errorCode, errorMessage, DEFAULT_LEADER_EPOCH, null); + public static PartitionStateSummaryData newPartitionStateSummaryData(int partition, int stateEpoch, long startOffset, int leaderEpoch, short errorCode, String errorMessage) { + return new PartitionData(partition, stateEpoch, startOffset, errorCode, errorMessage, leaderEpoch, null); } public static PartitionStateBatchData newPartitionStateBatchData(int partition, int stateEpoch, long startOffset, int leaderEpoch, List stateBatches) { diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionStateSummaryData.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionStateSummaryData.java index dc4732a79ae22..58a9dc1061520 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionStateSummaryData.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/PartitionStateSummaryData.java @@ -24,6 +24,8 @@ public interface PartitionStateSummaryData extends PartitionInfoData, PartitionIdData { int stateEpoch(); + int leaderEpoch(); + long startOffset(); short errorCode(); diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/ReadShareGroupStateSummaryResult.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/ReadShareGroupStateSummaryResult.java index 7e0bee13c3806..249eb20ed94aa 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/ReadShareGroupStateSummaryResult.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/ReadShareGroupStateSummaryResult.java @@ -38,7 +38,8 @@ public static ReadShareGroupStateSummaryResult from(ReadShareGroupStateSummaryRe .map(readStateSummaryResult -> new TopicData<>(readStateSummaryResult.topicId(), readStateSummaryResult.partitions().stream() .map(partitionResult -> PartitionFactory.newPartitionStateSummaryData( - partitionResult.partition(), partitionResult.stateEpoch(), partitionResult.startOffset(), partitionResult.errorCode(), partitionResult.errorMessage())) + partitionResult.partition(), partitionResult.stateEpoch(), partitionResult.startOffset(), + partitionResult.leaderEpoch(), partitionResult.errorCode(), partitionResult.errorMessage())) .collect(Collectors.toList()))) .collect(Collectors.toList())) .build(); diff --git a/server-common/src/test/java/org/apache/kafka/server/share/persister/DefaultStatePersisterTest.java b/server-common/src/test/java/org/apache/kafka/server/share/persister/DefaultStatePersisterTest.java index 1a668b5dc3275..697d958723a1a 100644 --- a/server-common/src/test/java/org/apache/kafka/server/share/persister/DefaultStatePersisterTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/share/persister/DefaultStatePersisterTest.java @@ -868,7 +868,7 @@ public void testReadStateSummarySuccess() { return requestGroupId.equals(groupId) && requestTopicId == topicId1 && requestPartition == partition1; }, - new ReadShareGroupStateSummaryResponse(ReadShareGroupStateSummaryResponse.toResponseData(topicId1, partition1, 0, 1)), + new ReadShareGroupStateSummaryResponse(ReadShareGroupStateSummaryResponse.toResponseData(topicId1, partition1, 0, 1, 1)), coordinatorNode1); client.prepareResponseFrom( @@ -880,7 +880,7 @@ public void testReadStateSummarySuccess() { return requestGroupId.equals(groupId) && requestTopicId == topicId2 && requestPartition == partition2; }, - new ReadShareGroupStateSummaryResponse(ReadShareGroupStateSummaryResponse.toResponseData(topicId2, partition2, 0, 1)), + new ReadShareGroupStateSummaryResponse(ReadShareGroupStateSummaryResponse.toResponseData(topicId2, partition2, 0, 1, 1)), coordinatorNode2); ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); @@ -930,12 +930,12 @@ public void testReadStateSummarySuccess() { HashSet expectedResultMap = new HashSet<>(); expectedResultMap.add( - (PartitionData) PartitionFactory.newPartitionStateSummaryData(partition1, 1, 0, Errors.NONE.code(), + (PartitionData) PartitionFactory.newPartitionStateSummaryData(partition1, 1, 0, 1, Errors.NONE.code(), null )); expectedResultMap.add( - (PartitionData) PartitionFactory.newPartitionStateSummaryData(partition2, 1, 0, Errors.NONE.code(), + (PartitionData) PartitionFactory.newPartitionStateSummaryData(partition2, 1, 0, 1, Errors.NONE.code(), null )); @@ -1437,6 +1437,7 @@ public void testReadStateSummaryResponseToResultPartialResults() { tp1.topicId(), tp1.partition(), 1L, + 1, 2 ) ) @@ -1468,7 +1469,7 @@ public void testReadStateSummaryResponseToResultPartialResults() { results.topicsData().contains( new TopicData<>( tp1.topicId(), - List.of(PartitionFactory.newPartitionStateSummaryData(tp1.partition(), 2, 1L, Errors.NONE.code(), null)) + List.of(PartitionFactory.newPartitionStateSummaryData(tp1.partition(), 2, 1L, 1, Errors.NONE.code(), null)) ) ) ); @@ -1476,7 +1477,7 @@ public void testReadStateSummaryResponseToResultPartialResults() { results.topicsData().contains( new TopicData<>( tp2.topicId(), - List.of(PartitionFactory.newPartitionStateSummaryData(tp2.partition(), 0, 0, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), "unknown tp")) + List.of(PartitionFactory.newPartitionStateSummaryData(tp2.partition(), 0, 0, 0, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), "unknown tp")) ) ) ); @@ -1496,6 +1497,7 @@ public void testReadStateSummaryResponseToResultFailedFuture() { tp1.topicId(), tp1.partition(), 1L, + 1, 2 ) ) @@ -1517,7 +1519,7 @@ public void testReadStateSummaryResponseToResultFailedFuture() { results.topicsData().contains( new TopicData<>( tp1.topicId(), - List.of(PartitionFactory.newPartitionStateSummaryData(tp1.partition(), 2, 1L, Errors.NONE.code(), null)) + List.of(PartitionFactory.newPartitionStateSummaryData(tp1.partition(), 2, 1L, 1, Errors.NONE.code(), null)) ) ) ); @@ -1525,7 +1527,7 @@ public void testReadStateSummaryResponseToResultFailedFuture() { results.topicsData().contains( new TopicData<>( tp2.topicId(), - List.of(PartitionFactory.newPartitionStateSummaryData(tp2.partition(), -1, -1L, Errors.UNKNOWN_SERVER_ERROR.code(), "Error reading state from share coordinator: java.lang.Exception: scary stuff")) + List.of(PartitionFactory.newPartitionStateSummaryData(tp2.partition(), -1, -1L, -1, Errors.UNKNOWN_SERVER_ERROR.code(), "Error reading state from share coordinator: java.lang.Exception: scary stuff")) ) ) ); diff --git a/server-common/src/test/java/org/apache/kafka/server/util/timer/MockTimer.java b/server-common/src/test/java/org/apache/kafka/server/util/timer/MockTimer.java index 8de1b91c32ddb..2bdbc9cb080e1 100644 --- a/server-common/src/test/java/org/apache/kafka/server/util/timer/MockTimer.java +++ b/server-common/src/test/java/org/apache/kafka/server/util/timer/MockTimer.java @@ -81,6 +81,9 @@ public MockTime time() { } public int size() { + synchronized (taskQueue) { + taskQueue.removeIf(TimerTaskEntry::cancelled); + } return taskQueue.size(); } diff --git a/server/src/main/java/org/apache/kafka/server/AssignmentsManager.java b/server/src/main/java/org/apache/kafka/server/AssignmentsManager.java index b6223418ab58a..6d493d96ec152 100644 --- a/server/src/main/java/org/apache/kafka/server/AssignmentsManager.java +++ b/server/src/main/java/org/apache/kafka/server/AssignmentsManager.java @@ -368,7 +368,7 @@ void handleResponse( previousGlobalFailures++; log.error("handleResponse: {} assignments failed; global error: {}. Retrying.", sent.size(), globalResponseError.get()); - sent.entrySet().forEach(e -> ready.putIfAbsent(e.getKey(), e.getValue())); + sent.forEach(ready::putIfAbsent); return; } previousGlobalFailures = 0; diff --git a/server/src/main/java/org/apache/kafka/server/metrics/ForwardingManagerMetrics.java b/server/src/main/java/org/apache/kafka/server/metrics/ForwardingManagerMetrics.java new file mode 100644 index 0000000000000..95dd84d09fb76 --- /dev/null +++ b/server/src/main/java/org/apache/kafka/server/metrics/ForwardingManagerMetrics.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.metrics; + +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Gauge; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Percentile; +import org.apache.kafka.common.metrics.stats.Percentiles; +import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing; + +import java.util.concurrent.atomic.AtomicInteger; + +public final class ForwardingManagerMetrics implements AutoCloseable { + + private final Metrics metrics; + + private static final String METRIC_GROUP_NAME = "ForwardingManager"; + private static final String QUEUE_TIME_MS_NAME = "QueueTimeMs"; + private static final String REMOTE_TIME_MS_NAME = "RemoteTimeMs"; + + /** + * A histogram describing the amount of time in milliseconds each admin request spends in the broker's forwarding manager queue, waiting to be sent to the controller. + * This does not include the time that the request spends waiting for a response from the controller. + */ + private final LatencyHistogram queueTimeMsHist; + + /** + * A histogram describing the amount of time in milliseconds each request sent by the ForwardingManager spends waiting for a response. + * This does not include the time spent in the queue. + */ + private final LatencyHistogram remoteTimeMsHist; + + private final MetricName queueLengthName; + private final AtomicInteger queueLength = new AtomicInteger(0); + + public ForwardingManagerMetrics(Metrics metrics, long timeoutMs) { + this.metrics = metrics; + + this.queueTimeMsHist = new LatencyHistogram(metrics, QUEUE_TIME_MS_NAME, METRIC_GROUP_NAME, timeoutMs); + this.remoteTimeMsHist = new LatencyHistogram(metrics, REMOTE_TIME_MS_NAME, METRIC_GROUP_NAME, timeoutMs); + + this.queueLengthName = metrics.metricName( + "QueueLength", + METRIC_GROUP_NAME, + "The current number of RPCs that are waiting in the broker's forwarding manager queue, waiting to be sent to the controller." + ); + metrics.addMetric(queueLengthName, (Gauge) (config, now) -> queueLength.get()); + } + + @Override + public void close() { + queueTimeMsHist.close(); + remoteTimeMsHist.close(); + metrics.removeMetric(queueLengthName); + } + + public LatencyHistogram queueTimeMsHist() { + return queueTimeMsHist; + } + + public LatencyHistogram remoteTimeMsHist() { + return remoteTimeMsHist; + } + + public MetricName queueLengthName() { + return queueLengthName; + } + + public void incrementQueueLength() { + queueLength.getAndIncrement(); + } + + public void decrementQueueLength() { + queueLength.getAndDecrement(); + } + + public static final class LatencyHistogram implements AutoCloseable { + private static final int SIZE_IN_BYTES = 4000; + private final Metrics metrics; + private final String name; + private final Sensor sensor; + private final MetricName latencyP99Name; + private final MetricName latencyP999Name; + + private LatencyHistogram(Metrics metrics, String name, String group, long maxLatency) { + this.metrics = metrics; + this.name = name; + this.sensor = metrics.sensor(name); + this.latencyP99Name = metrics.metricName(name + ".p99", group); + this.latencyP999Name = metrics.metricName(name + ".p999", group); + + sensor.add(new Percentiles( + SIZE_IN_BYTES, + maxLatency, + BucketSizing.CONSTANT, + new Percentile(latencyP99Name, 99), + new Percentile(latencyP999Name, 99.9) + )); + } + + @Override + public void close() { + metrics.removeSensor(name); + metrics.removeMetric(latencyP99Name); + metrics.removeMetric(latencyP999Name); + } + + public void record(long latencyMs) { + sensor.record(latencyMs); + } + + // visible for test + public MetricName latencyP99Name() { + return latencyP99Name; + } + + // visible for test + public MetricName latencyP999Name() { + return latencyP999Name; + } + } +} \ No newline at end of file diff --git a/server/src/main/java/org/apache/kafka/server/share/fetch/DelayedShareFetchGroupKey.java b/server/src/main/java/org/apache/kafka/server/share/fetch/DelayedShareFetchGroupKey.java index 0fe1a4774f501..88dbabf6e5cd2 100644 --- a/server/src/main/java/org/apache/kafka/server/share/fetch/DelayedShareFetchGroupKey.java +++ b/server/src/main/java/org/apache/kafka/server/share/fetch/DelayedShareFetchGroupKey.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.server.share.fetch; +import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.Uuid; import java.util.Objects; @@ -28,6 +29,10 @@ public class DelayedShareFetchGroupKey implements DelayedShareFetchKey { private final Uuid topicId; private final int partition; + public DelayedShareFetchGroupKey(String groupId, TopicIdPartition topicIdPartition) { + this(groupId, topicIdPartition.topicId(), topicIdPartition.partition()); + } + public DelayedShareFetchGroupKey(String groupId, Uuid topicId, int partition) { this.groupId = groupId; this.topicId = topicId; diff --git a/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java b/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java index 0b06ea535be7e..f0f37d9ec7dca 100644 --- a/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java +++ b/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.requests.ShareRequestMetadata; import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; import org.apache.kafka.server.metrics.KafkaMetricsGroup; +import org.apache.kafka.server.network.ConnectionDisconnectListener; import org.apache.kafka.server.share.CachedSharePartition; import com.yammer.metrics.core.Meter; @@ -53,10 +54,13 @@ public class ShareSessionCache { private final int maxEntries; private long numPartitions = 0; + private final ConnectionDisconnectListener connectionDisconnectListener; // A map of session key to ShareSession. private final Map sessions = new HashMap<>(); + private final Map connectionIdToSessionMap; + @SuppressWarnings("this-escape") public ShareSessionCache(int maxEntries) { this.maxEntries = maxEntries; @@ -64,6 +68,8 @@ public ShareSessionCache(int maxEntries) { KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "ShareSessionCache"); metricsGroup.newGauge(SHARE_SESSIONS_COUNT, this::size); metricsGroup.newGauge(SHARE_PARTITIONS_COUNT, this::totalPartitions); + this.connectionIdToSessionMap = new HashMap<>(); + this.connectionDisconnectListener = new ClientConnectionDisconnectListener(); this.evictionsMeter = metricsGroup.newMeter(SHARE_SESSION_EVICTIONS_PER_SEC, "evictions", TimeUnit.SECONDS); } @@ -123,21 +129,48 @@ public synchronized void updateNumPartitions(ShareSession session) { * @param groupId - The group id in the share fetch request. * @param memberId - The member id in the share fetch request. * @param partitionMap - The topic partitions to be added to the session. + * @param clientConnectionId - The client connection id. * @return - The session key if the session was created, or null if the session was not created. */ - public synchronized ShareSessionKey maybeCreateSession(String groupId, Uuid memberId, ImplicitLinkedHashCollection partitionMap) { + public synchronized ShareSessionKey maybeCreateSession( + String groupId, + Uuid memberId, + ImplicitLinkedHashCollection partitionMap, + String clientConnectionId + ) { if (sessions.size() < maxEntries) { ShareSession session = new ShareSession(new ShareSessionKey(groupId, memberId), partitionMap, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)); sessions.put(session.key(), session); updateNumPartitions(session); + connectionIdToSessionMap.put(clientConnectionId, session.key()); return session.key(); } return null; } + public ConnectionDisconnectListener connectionDisconnectListener() { + return connectionDisconnectListener; + } + // Visible for testing. Meter evictionsMeter() { return evictionsMeter; } + + private final class ClientConnectionDisconnectListener implements ConnectionDisconnectListener { + + // When the client disconnects, the corresponding session should be removed from the cache. + @Override + public void onDisconnect(String connectionId) { + ShareSessionKey shareSessionKey = connectionIdToSessionMap.remove(connectionId); + if (shareSessionKey != null) { + // Remove the session from the cache. + ShareSession removedSession = remove(shareSessionKey); + if (removedSession != null) { + evictionsMeter.mark(); + } + } + } + } } diff --git a/server/src/test/java/org/apache/kafka/server/metrics/ForwardingManagerMetricsTest.java b/server/src/test/java/org/apache/kafka/server/metrics/ForwardingManagerMetricsTest.java new file mode 100644 index 0000000000000..d3716366f9063 --- /dev/null +++ b/server/src/test/java/org/apache/kafka/server/metrics/ForwardingManagerMetricsTest.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.metrics; + +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.Metrics; + +import org.junit.jupiter.api.Test; + +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class ForwardingManagerMetricsTest { + @Test + void testMetricsNames() { + String expectedGroup = "ForwardingManager"; + Set expectedMetrics = Set.of( + new MetricName("QueueTimeMs.p99", expectedGroup, "", Map.of()), + new MetricName("QueueTimeMs.p999", expectedGroup, "", Map.of()), + new MetricName("QueueLength", expectedGroup, "", Map.of()), + new MetricName("RemoteTimeMs.p99", expectedGroup, "", Map.of()), + new MetricName("RemoteTimeMs.p999", expectedGroup, "", Map.of()) + ); + + try (Metrics metrics = new Metrics()) { + Map metricsMap = metrics.metrics().entrySet().stream() + .filter(entry -> entry.getKey().group().equals(expectedGroup)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + assertEquals(0, metricsMap.size()); + + try (ForwardingManagerMetrics ignore = new ForwardingManagerMetrics(metrics, 1000)) { + metricsMap = metrics.metrics().entrySet().stream() + .filter(entry -> entry.getKey().group().equals(expectedGroup)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + assertEquals(expectedMetrics.size(), metricsMap.size()); + metricsMap.keySet().forEach(name -> + assertTrue(expectedMetrics.contains(name), "Metric " + name + " not found in expected set") + ); + } finally { + metricsMap = metrics.metrics().entrySet().stream() + .filter(entry -> entry.getKey().group().equals(expectedGroup)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + assertEquals(0, metricsMap.size()); + } + } + } + + @Test + void testQueueTimeMs() { + try (Metrics metrics = new Metrics(); + ForwardingManagerMetrics forwardingManagerMetrics = new ForwardingManagerMetrics(metrics, 1000)) { + KafkaMetric queueTimeMsP99 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist().latencyP99Name()); + KafkaMetric queueTimeMsP999 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist().latencyP999Name()); + assertEquals(Double.NaN, (Double) queueTimeMsP99.metricValue()); + assertEquals(Double.NaN, (Double) queueTimeMsP999.metricValue()); + for (int i = 0; i < 1000; i++) { + forwardingManagerMetrics.queueTimeMsHist().record(i); + } + assertEquals(990.0, (Double) queueTimeMsP99.metricValue(), 0.1); + assertEquals(999.0, (Double) queueTimeMsP999.metricValue(), 0.1); + } + } + + @Test + void testQueueLength() { + try (Metrics metrics = new Metrics(); + ForwardingManagerMetrics forwardingManagerMetrics = new ForwardingManagerMetrics(metrics, 1000)) { + KafkaMetric queueLength = metrics.metrics().get(forwardingManagerMetrics.queueLengthName()); + assertEquals(0, (Integer) queueLength.metricValue()); + forwardingManagerMetrics.incrementQueueLength(); + assertEquals(1, (Integer) queueLength.metricValue()); + } + } + + @Test + void testRemoteTimeMs() { + try (Metrics metrics = new Metrics(); + ForwardingManagerMetrics forwardingManagerMetrics = new ForwardingManagerMetrics(metrics, 1000)) { + KafkaMetric remoteTimeMsP99 = metrics.metrics().get(forwardingManagerMetrics.remoteTimeMsHist().latencyP99Name()); + KafkaMetric remoteTimeMsP999 = metrics.metrics().get(forwardingManagerMetrics.remoteTimeMsHist().latencyP999Name()); + assertEquals(Double.NaN, (Double) remoteTimeMsP99.metricValue()); + assertEquals(Double.NaN, (Double) remoteTimeMsP999.metricValue()); + for (int i = 0; i < 1000; i++) { + forwardingManagerMetrics.remoteTimeMsHist().record(i); + } + assertEquals(990.0, (Double) remoteTimeMsP99.metricValue(), 0.1); + assertEquals(999.0, (Double) remoteTimeMsP999.metricValue(), 0.1); + } + } + + @Test + void testTimeoutMs() { + long timeoutMs = 500; + try (Metrics metrics = new Metrics(); + ForwardingManagerMetrics forwardingManagerMetrics = new ForwardingManagerMetrics(metrics, timeoutMs)) { + KafkaMetric queueTimeMsP99 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist().latencyP99Name()); + KafkaMetric queueTimeMsP999 = metrics.metrics().get(forwardingManagerMetrics.queueTimeMsHist().latencyP999Name()); + assertEquals(Double.NaN, (Double) queueTimeMsP99.metricValue()); + assertEquals(Double.NaN, (Double) queueTimeMsP999.metricValue()); + for (int i = 0; i < 100; i++) { + forwardingManagerMetrics.queueTimeMsHist().record(i); + } + forwardingManagerMetrics.queueTimeMsHist().record(1000); + + assertEquals(99.0, (Double) queueTimeMsP99.metricValue(), 0.1); + assertEquals(timeoutMs * 0.999, (Double) queueTimeMsP999.metricValue(), 0.1); + } + } +} \ No newline at end of file diff --git a/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java b/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java index ca18de5b65c47..c9692063b5c99 100644 --- a/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java +++ b/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java @@ -45,11 +45,11 @@ public void setUp() { public void testShareSessionCache() throws InterruptedException { ShareSessionCache cache = new ShareSessionCache(3); assertEquals(0, cache.size()); - ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(10)); - ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(20)); - ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(30)); - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(40))); - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5))); + ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(10), "conn-1"); + ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(20), "conn-2"); + ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(30), "conn-3"); + assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(40), "conn-4")); + assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5), "conn-5")); assertShareCacheContains(cache, List.of(key1, key2, key3)); assertMetricsValues(3, 60, 0, cache); @@ -60,7 +60,7 @@ public void testResizeCachedSessions() throws InterruptedException { ShareSessionCache cache = new ShareSessionCache(2); assertEquals(0, cache.size()); assertEquals(0, cache.totalPartitions()); - ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(2)); + ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(2), "conn-1"); assertNotNull(key1); assertShareCacheContains(cache, List.of(key1)); ShareSession session1 = cache.get(key1); @@ -70,7 +70,7 @@ public void testResizeCachedSessions() throws InterruptedException { assertMetricsValues(1, 2, 0, cache); - ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(4)); + ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(4), "conn-2"); assertNotNull(key2); assertShareCacheContains(cache, List.of(key1, key2)); ShareSession session2 = cache.get(key2); @@ -81,7 +81,7 @@ public void testResizeCachedSessions() throws InterruptedException { assertMetricsValues(2, 6, 0, cache); - ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5)); + ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5), "conn-3"); assertNull(key3); assertShareCacheContains(cache, List.of(key1, key2)); assertEquals(6, cache.totalPartitions()); @@ -109,6 +109,36 @@ public void testResizeCachedSessions() throws InterruptedException { assertMetricsValues(1, 3, 0, cache); } + @Test + public void testRemoveConnection() throws InterruptedException { + ShareSessionCache cache = new ShareSessionCache(3); + assertEquals(0, cache.size()); + ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(1), "conn-1"); + ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(2), "conn-2"); + ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(3), "conn-3"); + + assertMetricsValues(3, 6, 0, cache); + + // Since cache size is now equal to max entries allowed(3), no new session can be created. + assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(40), "conn-4")); + assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5), "conn-5")); + assertShareCacheContains(cache, List.of(key1, key2, key3)); + + assertMetricsValues(3, 6, 0, cache); + + // Simulating the disconnection of client with connection id conn-1 + cache.connectionDisconnectListener().onDisconnect("conn-1"); + assertShareCacheContains(cache, List.of(key2, key3)); + + assertMetricsValues(2, 5, 1, cache); + + // Since one client got disconnected, we can add another one now + ShareSessionKey key4 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(4), "conn-6"); + assertShareCacheContains(cache, List.of(key2, key3, key4)); + + assertMetricsValues(3, 9, 1, cache); + } + private ImplicitLinkedHashCollection mockedSharePartitionMap(int size) { ImplicitLinkedHashCollection cacheMap = new ImplicitLinkedHashCollection<>(size); diff --git a/server/src/test/java/org/apache/kafka/server/util/SchedulerTest.java b/server/src/test/java/org/apache/kafka/server/util/SchedulerTest.java new file mode 100644 index 0000000000000..933a14b77667e --- /dev/null +++ b/server/src/test/java/org/apache/kafka/server/util/SchedulerTest.java @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.util; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.coordinator.transaction.TransactionLogConfig; +import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache; +import org.apache.kafka.storage.internals.log.LoadedLogOffsets; +import org.apache.kafka.storage.internals.log.LocalLog; +import org.apache.kafka.storage.internals.log.LogConfig; +import org.apache.kafka.storage.internals.log.LogDirFailureChannel; +import org.apache.kafka.storage.internals.log.LogLoader; +import org.apache.kafka.storage.internals.log.LogOffsetsListener; +import org.apache.kafka.storage.internals.log.LogSegments; +import org.apache.kafka.storage.internals.log.ProducerStateManager; +import org.apache.kafka.storage.internals.log.ProducerStateManagerConfig; +import org.apache.kafka.storage.internals.log.UnifiedLog; +import org.apache.kafka.storage.log.metrics.BrokerTopicStats; +import org.apache.kafka.test.TestUtils; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class SchedulerTest { + + private final KafkaScheduler scheduler = new KafkaScheduler(1); + private final MockTime mockTime = new MockTime(); + private final AtomicInteger counter1 = new AtomicInteger(0); + private final AtomicInteger counter2 = new AtomicInteger(0); + + @BeforeEach + void setup() { + counter1.set(0); + counter2.set(0); + scheduler.startup(); + } + + @AfterEach + void teardown() throws InterruptedException { + scheduler.shutdown(); + } + + @Test + void testMockSchedulerNonPeriodicTask() { + mockTime.scheduler.scheduleOnce("test1", counter1::getAndIncrement, 1); + mockTime.scheduler.scheduleOnce("test2", counter2::getAndIncrement, 100); + assertEquals(0, counter1.get(), "Counter1 should not be incremented prior to task running."); + assertEquals(0, counter2.get(), "Counter2 should not be incremented prior to task running."); + mockTime.sleep(1); + assertEquals(1, counter1.get(), "Counter1 should be incremented"); + assertEquals(0, counter2.get(), "Counter2 should not be incremented"); + mockTime.sleep(100000); + assertEquals(1, counter1.get(), "More sleeping should not result in more incrementing on counter1."); + assertEquals(1, counter2.get(), "Counter2 should now be incremented."); + } + + @Test + void testMockSchedulerPeriodicTask() { + mockTime.scheduler.schedule("test1", counter1::getAndIncrement, 1, 1); + mockTime.scheduler.schedule("test2", counter2::getAndIncrement, 100, 100); + assertEquals(0, counter1.get(), "Counter1 should not be incremented prior to task running."); + assertEquals(0, counter2.get(), "Counter2 should not be incremented prior to task running."); + mockTime.sleep(1); + assertEquals(1, counter1.get(), "Counter1 should be incremented"); + assertEquals(0, counter2.get(), "Counter2 should not be incremented"); + mockTime.sleep(100); + assertEquals(101, counter1.get(), "Counter1 should be incremented 101 times"); + assertEquals(1, counter2.get(), "Counter2 should not be incremented once"); + } + + @Test + void testReentrantTaskInMockScheduler() { + mockTime.scheduler.scheduleOnce("test1", () -> mockTime.scheduler.scheduleOnce("test2", counter2::getAndIncrement, 0), 1); + mockTime.sleep(1); + assertEquals(1, counter2.get()); + } + + @Test + void testNonPeriodicTask() throws InterruptedException { + scheduler.scheduleOnce("test", counter1::getAndIncrement); + TestUtils.waitForCondition(() -> counter1.get() == 1, "Scheduled task was not executed"); + Thread.sleep(5); + assertEquals(1, counter1.get(), "Should only run once"); + } + + @Test + void testNonPeriodicTaskWhenPeriodIsZero() throws InterruptedException { + scheduler.schedule("test", counter1::getAndIncrement, 0, 0); + TestUtils.waitForCondition(() -> counter1.get() == 1, "Scheduled task was not executed"); + Thread.sleep(5); + assertEquals(1, counter1.get(), "Should only run once"); + } + + @Test + void testPeriodicTask() throws InterruptedException { + scheduler.schedule("test", counter1::getAndIncrement, 0, 5); + TestUtils.waitForCondition(() -> counter1.get() >= 20, "Should count to 20"); + } + + @Test + void testRestart() throws InterruptedException { + // schedule a task to increment a counter + mockTime.scheduler.scheduleOnce("test1", counter1::getAndIncrement, 1); + mockTime.sleep(1); + assertEquals(1, counter1.get()); + + // restart the scheduler + mockTime.scheduler.shutdown(); + mockTime.scheduler.startup(); + + // schedule another task to increment the counter + mockTime.scheduler.scheduleOnce("test1", counter1::getAndIncrement, 1); + mockTime.sleep(1); + assertEquals(2, counter1.get()); + } + + @Test + void testUnscheduleProducerTask() throws IOException { + File tmpDir = TestUtils.tempDirectory(); + File logDir = TestUtils.randomPartitionLogDir(tmpDir); + LogConfig logConfig = new LogConfig(new Properties()); + BrokerTopicStats brokerTopicStats = new BrokerTopicStats(); + int maxTransactionTimeoutMs = 5 * 60 * 1000; + int maxProducerIdExpirationMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT; + int producerIdExpirationCheckIntervalMs = TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT; + TopicPartition topicPartition = UnifiedLog.parseTopicPartitionName(logDir); + LogDirFailureChannel logDirFailureChannel = new LogDirFailureChannel(10); + LogSegments segments = new LogSegments(topicPartition); + LeaderEpochFileCache leaderEpochCache = UnifiedLog.createLeaderEpochCache(logDir, topicPartition, + logDirFailureChannel, Optional.empty(), mockTime.scheduler); + ProducerStateManagerConfig producerStateManagerConfig = new ProducerStateManagerConfig(maxProducerIdExpirationMs, false); + ProducerStateManager producerStateManager = new ProducerStateManager(topicPartition, logDir, + maxTransactionTimeoutMs, producerStateManagerConfig, mockTime); + LoadedLogOffsets offsets = new LogLoader( + logDir, + topicPartition, + logConfig, + scheduler, + mockTime, + logDirFailureChannel, + true, + segments, + 0L, + 0L, + leaderEpochCache, + producerStateManager, + new ConcurrentHashMap<>(), false).load(); + LocalLog localLog = new LocalLog(logDir, logConfig, segments, offsets.recoveryPoint, + offsets.nextOffsetMetadata, scheduler, mockTime, topicPartition, logDirFailureChannel); + UnifiedLog log = new UnifiedLog(offsets.logStartOffset, + localLog, + brokerTopicStats, + producerIdExpirationCheckIntervalMs, + leaderEpochCache, + producerStateManager, + Optional.empty(), + false, + LogOffsetsListener.NO_OP_OFFSETS_LISTENER); + assertTrue(scheduler.taskRunning(log.producerExpireCheck())); + log.close(); + assertFalse(scheduler.taskRunning(log.producerExpireCheck())); + } + + /** + * Verify that scheduler lock is not held when invoking task method, allowing new tasks to be scheduled + * when another is being executed. This is required to avoid deadlocks when: + *
    + *
  • Thread1 executes a task which attempts to acquire LockA
  • + *
  • Thread2 holding LockA attempts to schedule a new task
  • + *
+ */ + @Timeout(15) + @Test + void testMockSchedulerLocking() throws InterruptedException { + CountDownLatch initLatch = new CountDownLatch(1); + CountDownLatch completionLatch = new CountDownLatch(2); + List taskLatches = List.of(new CountDownLatch(1), new CountDownLatch(1)); + InterruptedConsumer scheduledTask = taskLatch -> { + initLatch.countDown(); + assertTrue(taskLatch.await(30, TimeUnit.SECONDS), "Timed out waiting for latch"); + completionLatch.countDown(); + }; + mockTime.scheduler.scheduleOnce("test1", interruptedRunnableWrapper(() -> scheduledTask.accept(taskLatches.get(0))), 1); + ScheduledExecutorService tickExecutor = Executors.newSingleThreadScheduledExecutor(); + try { + tickExecutor.scheduleWithFixedDelay(() -> mockTime.sleep(1), 0, 1, TimeUnit.MILLISECONDS); + + // wait for first task to execute and then schedule the next task while the first one is running + assertTrue(initLatch.await(10, TimeUnit.SECONDS)); + mockTime.scheduler.scheduleOnce("test2", interruptedRunnableWrapper(() -> scheduledTask.accept(taskLatches.get(1))), 1); + + taskLatches.forEach(CountDownLatch::countDown); + assertTrue(completionLatch.await(10, TimeUnit.SECONDS), "Tasks did not complete"); + } finally { + tickExecutor.shutdownNow(); + } + } + + @Test + void testPendingTaskSize() throws InterruptedException { + CountDownLatch latch1 = new CountDownLatch(1); + CountDownLatch latch2 = new CountDownLatch(2); + scheduler.scheduleOnce("task1", interruptedRunnableWrapper(latch1::await), 0); + scheduler.scheduleOnce("task2", latch2::countDown, 5); + scheduler.scheduleOnce("task3", latch2::countDown, 5); + TestUtils.waitForCondition(() -> scheduler.pendingTaskSize() <= 2, "Scheduled task was not executed"); + latch1.countDown(); + latch2.await(); + TestUtils.waitForCondition(() -> scheduler.pendingTaskSize() == 0, "Scheduled task was not executed"); + scheduler.shutdown(); + assertEquals(0, scheduler.pendingTaskSize()); + } + + @FunctionalInterface + private interface InterruptedConsumer { + void accept(T t) throws InterruptedException; + } + + @FunctionalInterface + private interface InterruptedRunnable { + void run() throws InterruptedException; + } + + private static Runnable interruptedRunnableWrapper(InterruptedRunnable runnable) { + return () -> { + try { + runnable.run(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + }; + } +} diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java index 321a1e92ac09e..09dcd82f2d81c 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java @@ -271,7 +271,7 @@ private void setupPeriodicJobs() { } private void setupRecordPruning() { - log.info("Scheduling share-group state topic prune job."); + log.debug("Scheduling share-group state topic prune job."); timer.add(new TimerTask(config.shareCoordinatorTopicPruneIntervalMs()) { @Override public void run() { @@ -291,7 +291,6 @@ public void run() { } private CompletableFuture performRecordPruning(TopicPartition tp) { - // This future will always be completed normally, exception or not. CompletableFuture fut = new CompletableFuture<>(); runtime.scheduleWriteOperation( @@ -322,11 +321,11 @@ private CompletableFuture performRecordPruning(TopicPartition tp) { return; } - log.info("Pruning records in {} till offset {}.", tp, off); + log.debug("Pruning records in {} till offset {}.", tp, off); writer.deleteRecords(tp, off) .whenComplete((res, exp) -> { if (exp != null) { - log.debug("Exception while deleting records in {} till offset {}.", tp, off, exp); + log.error("Exception while deleting records in {} till offset {}.", tp, off, exp); fut.completeExceptionally(exp); return; } @@ -347,7 +346,7 @@ private CompletableFuture performRecordPruning(TopicPartition tp) { } private void setupSnapshotColdPartitions() { - log.info("Scheduling cold share-partition snapshotting."); + log.debug("Scheduling cold share-partition snapshotting."); timer.add(new TimerTask(config.shareCoordinatorColdPartitionSnapshotIntervalMs()) { @Override public void run() { diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java index d38564fd6f849..7f03f9254b1e3 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java @@ -453,6 +453,7 @@ public CoordinatorResult doClean(LogToClean cleanable, long currentT List> groupedSegments = groupSegmentsBySize( log.logSegments(0, endOffset), - log.config().segmentSize(), + log.config().segmentSize, log.config().maxIndexSize, cleanable.firstUncleanableOffset() ); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 88f7b54cd320c..21c92cd84dff4 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.LegacyRecord; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ConfigUtils; import org.apache.kafka.common.utils.Utils; @@ -139,8 +140,6 @@ public Optional serverConfigName(String configName) { public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs - public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; - // Visible for testing public static final Set CONFIGS_WITH_NO_SERVER_DEFAULTS = Set.of( TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, @@ -187,13 +186,12 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) - .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) - .defineInternal(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_DOC); + .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { CONFIG. - define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(1024 * 1024), MEDIUM, + define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), MEDIUM, TopicConfig.SEGMENT_BYTES_DOC) .define(TopicConfig.SEGMENT_MS_CONFIG, LONG, DEFAULT_SEGMENT_MS, atLeast(1), MEDIUM, TopicConfig.SEGMENT_MS_DOC) .define(TopicConfig.SEGMENT_JITTER_MS_CONFIG, LONG, DEFAULT_SEGMENT_JITTER_MS, atLeast(0), MEDIUM, @@ -255,8 +253,7 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) - .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) - .defineInternal(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_DOC); + .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC); } public final Set overriddenConfigs; @@ -265,8 +262,7 @@ public Optional serverConfigName(String configName) { * Important note: Any configuration parameter that is passed along from KafkaConfig to LogConfig * should also be in `KafkaConfig#extractLogConfigMap`. */ - private final int segmentSize; - private final Integer internalSegmentSize; + public final int segmentSize; public final long segmentMs; public final long segmentJitterMs; public final int maxIndexSize; @@ -310,7 +306,6 @@ public LogConfig(Map props, Set overriddenConfigs) { this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); - this.internalSegmentSize = getInt(INTERNAL_SEGMENT_BYTES_CONFIG); this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); @@ -372,11 +367,6 @@ private Optional getCompression() { } } - public int segmentSize() { - if (internalSegmentSize == null) return segmentSize; - return internalSegmentSize; - } - // Exposed as a method so it can be mocked public int maxMessageSize() { return maxMessageSize; @@ -398,7 +388,7 @@ public long maxSegmentMs() { public int initFileSize() { if (preallocate) - return segmentSize(); + return segmentSize; else return 0; } @@ -638,7 +628,7 @@ public static void validate(Map existingConfigs, @Override public String toString() { return "LogConfig{" + - "segmentSize=" + segmentSize() + + "segmentSize=" + segmentSize + ", segmentMs=" + segmentMs + ", segmentJitterMs=" + segmentJitterMs + ", maxIndexSize=" + maxIndexSize + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index ff689898ecd62..972aeb095815a 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1177,9 +1177,9 @@ private LogAppendInfo append(MemoryRecords records, }); // check messages size does not exceed config.segmentSize - if (validRecords.sizeInBytes() > config().segmentSize()) { + if (validRecords.sizeInBytes() > config().segmentSize) { throw new RecordBatchTooLargeException("Message batch size is " + validRecords.sizeInBytes() + " bytes in append " + - "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize() + "."); + "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize + "."); } // maybe roll the log if this segment is full @@ -2034,12 +2034,12 @@ private LogSegment maybeRoll(int messagesSize, LogAppendInfo appendInfo) throws long maxTimestampInMessages = appendInfo.maxTimestamp(); long maxOffsetInMessages = appendInfo.lastOffset(); - if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize(), appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { + if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize, appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { logger.debug("Rolling new log segment (log_size = {}/{}}, " + "offset_index_size = {}/{}, " + "time_index_size = {}/{}, " + "inactive_time_ms = {}/{}).", - segment.size(), config().segmentSize(), + segment.size(), config().segmentSize, segment.offsetIndex().entries(), segment.offsetIndex().maxEntries(), segment.timeIndex().entries(), segment.timeIndex().maxEntries(), segment.timeWaitedForRoll(now, maxTimestampInMessages), config().segmentMs - segment.rollJitterMs()); @@ -2416,15 +2416,13 @@ public static void rebuildProducerState(ProducerStateManager producerStateManage Time time, boolean reloadFromCleanShutdown, String logPrefix) throws IOException { - List> offsetsToSnapshot = new ArrayList<>(); - if (segments.nonEmpty()) { - long lastSegmentBaseOffset = segments.lastSegment().get().baseOffset(); - Optional lowerSegment = segments.lowerSegment(lastSegmentBaseOffset); - Optional nextLatestSegmentBaseOffset = lowerSegment.map(LogSegment::baseOffset); - offsetsToSnapshot.add(nextLatestSegmentBaseOffset); - offsetsToSnapshot.add(Optional.of(lastSegmentBaseOffset)); - } - offsetsToSnapshot.add(Optional.of(lastOffset)); + List offsetsToSnapshot = new ArrayList<>(); + segments.lastSegment().ifPresent(lastSegment -> { + long lastSegmentBaseOffset = lastSegment.baseOffset(); + segments.lowerSegment(lastSegmentBaseOffset).ifPresent(s -> offsetsToSnapshot.add(s.baseOffset())); + offsetsToSnapshot.add(lastSegmentBaseOffset); + }); + offsetsToSnapshot.add(lastOffset); LOG.info("{}Loading producer state till offset {}", logPrefix, lastOffset); @@ -2443,11 +2441,9 @@ public static void rebuildProducerState(ProducerStateManager producerStateManage // To avoid an expensive scan through all the segments, we take empty snapshots from the start of the // last two segments and the last offset. This should avoid the full scan in the case that the log needs // truncation. - for (Optional offset : offsetsToSnapshot) { - if (offset.isPresent()) { - producerStateManager.updateMapEndOffset(offset.get()); - producerStateManager.takeSnapshot(); - } + for (long offset : offsetsToSnapshot) { + producerStateManager.updateMapEndOffset(offset); + producerStateManager.takeSnapshot(); } } else { LOG.info("{}Reloading from producer snapshot and rebuilding producer state from offset {}", logPrefix, lastOffset); @@ -2469,7 +2465,7 @@ public static void rebuildProducerState(ProducerStateManager producerStateManage long startOffset = Utils.max(segment.baseOffset(), producerStateManager.mapEndOffset(), logStartOffset); producerStateManager.updateMapEndOffset(startOffset); - if (offsetsToSnapshot.contains(Optional.of(segment.baseOffset()))) { + if (offsetsToSnapshot.contains(segment.baseOffset())) { producerStateManager.takeSnapshot(); } int maxPosition = segment.size(); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index d58c520af0972..7dfba8f7a5954 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -197,7 +198,7 @@ public void testUpdateConfig() { assertEquals(oldConfig, log.config()); Properties props = new Properties(); - props.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); + props.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldConfig.segmentSize + 1); LogConfig newConfig = new LogConfig(props); log.updateConfig(newConfig); assertEquals(newConfig, log.config()); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java index 865c88fec6491..d85304b3dd6da 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/RemoteIndexCacheTest.java @@ -376,8 +376,10 @@ public void testCleanerThreadShutdown() throws IOException, InterruptedException // close the cache properly cache.close(); // verify that the thread is closed properly - threads = getRunningCleanerThread(); - assertTrue(threads.isEmpty(), "Found unexpected " + threads.size() + " threads=" + threads.stream().map(Thread::getName).collect(Collectors.joining(", "))); + TestUtils.waitForCondition( + () -> getRunningCleanerThread().isEmpty(), + () -> "Failed while waiting for cleaner threads to shutdown. Remaining threads: " + + getRunningCleanerThread().stream().map(Thread::getName).collect(Collectors.joining(", "))); // if the thread is correctly being shutdown it will not be running assertFalse(cache.cleanerScheduler().isStarted(), "Unexpected thread state=running. Check error logs."); } diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java index 6857bf9e380eb..df96837458c69 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/AdjustStreamThreadCountTest.java @@ -51,6 +51,7 @@ import java.util.List; import java.util.Optional; import java.util.Properties; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -69,6 +70,7 @@ import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -239,10 +241,30 @@ public void shouldAddAndRemoveThreadsMultipleTimes() throws InterruptedException final CountDownLatch latch = new CountDownLatch(2); final Thread one = adjustCountHelperThread(kafkaStreams, 4, latch); final Thread two = adjustCountHelperThread(kafkaStreams, 6, latch); - two.start(); - one.start(); - latch.await(30, TimeUnit.SECONDS); - assertThat(kafkaStreams.metadataForLocalThreads().size(), equalTo(oldThreadCount)); + Set threadMetadata = null; + + AssertionError testError = null; + try { + two.start(); + one.start(); + + assertTrue(latch.await(30, TimeUnit.SECONDS)); + one.join(); + two.join(); + + threadMetadata = kafkaStreams.metadataForLocalThreads(); + assertThat(threadMetadata.size(), equalTo(oldThreadCount)); + } catch (final AssertionError e) { + System.err.println(threadMetadata); + testError = e; + } finally { + one.join(); + two.join(); + } + + if (testError != null) { + throw testError; + } waitForTransitionFromRebalancingToRunning(); } @@ -274,7 +296,7 @@ public void testRebalanceHappensBeforeStreamThreadGetDown() throws Exception { try { // block the pending shutdown thread to test whether other running thread // can make kafka streams running - latchBeforeDead.await(DEFAULT_DURATION.toMillis(), TimeUnit.MILLISECONDS); + assertFalse(latchBeforeDead.await(DEFAULT_DURATION.toMillis(), TimeUnit.MILLISECONDS)); } catch (final InterruptedException e) { throw new RuntimeException(e); } diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index 21d87556f75df..6023706847f15 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -171,7 +171,7 @@ public void setup() { streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); - streamsConfiguration.put(StreamsConfig.topicPrefix("internal.segment.bytes"), PURGE_SEGMENT_BYTES); + streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size final StreamsBuilder builder = new StreamsBuilder(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/BatchingStateRestoreCallback.java b/streams/src/main/java/org/apache/kafka/streams/processor/BatchingStateRestoreCallback.java index 37801d1579d29..c37d45a801613 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/BatchingStateRestoreCallback.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/BatchingStateRestoreCallback.java @@ -28,11 +28,10 @@ * It is expected that implementations of this class will not call the {@link StateRestoreCallback#restore(byte[], * byte[])} method. */ -@FunctionalInterface public interface BatchingStateRestoreCallback extends StateRestoreCallback { /** - * Called to restore a number of records. This method is called repeatedly until the {@link StateStore} is fulled + * Called to restore a number of records. This method is called repeatedly until the {@link StateStore} is fully * restored. * * @param records the records to restore. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java index eb164e2b27fd7..6ad61851ddae3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java @@ -178,10 +178,10 @@ private static void assignActive(final ApplicationState applicationState, // assign any remaining unassigned tasks final List sortedTasks = new ArrayList<>(unassigned); Collections.sort(sortedTasks); - for (final TaskId taskId : sortedTasks) { - final Set candidateClients = clients.stream() + final Set candidateClients = clients.stream() .map(KafkaStreamsState::processId) .collect(Collectors.toSet()); + for (final TaskId taskId : sortedTasks) { final ProcessId bestClient = assignmentState.findBestClientForTask(taskId, candidateClients); assignmentState.finalizeAssignment(taskId, bestClient, AssignedTask.Type.ACTIVE); } diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index e726debccd274..f081a768815fa 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -248,7 +248,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { props.put(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, 99_999L); props.put(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, 7L); props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "dummy:host"); - props.put(StreamsConfig.topicPrefix("internal.segment.bytes"), 100); + props.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), 100); final StreamsConfig streamsConfig = new StreamsConfig(props); final Map returnedProps = streamsConfig.getMainConsumerConfigs(groupId, clientId, threadIdx); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java index 2bace901779fe..0315120ae1b1e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedDeserializerTest.java @@ -44,37 +44,44 @@ public void testSessionWindowedDeserializerConstructor() { assertInstanceOf(StringDeserializer.class, inner, "Inner deserializer type should be StringDeserializer"); } + @Deprecated @Test public void shouldSetSerializerThroughWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); - final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>(); - deserializer.configure(props, false); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + try (final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>()) { + deserializer.configure(props, false); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + } } @Test public void shouldSetSerializerThroughWindowedInnerDeserializerClassConfig() { props.put(SessionWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); - final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>(); - deserializer.configure(props, false); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + try (final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>()) { + deserializer.configure(props, false); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + } } + @Deprecated @Test public void shouldIgnoreWindowedInnerClassSerdeConfigIfWindowedInnerDeserializerClassConfigIsSet() { props.put(SessionWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); - final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>(); - deserializer.configure(props, false); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + try (final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>()) { + deserializer.configure(props, false); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + } } @Test public void shouldThrowErrorIfWindowedInnerClassSerdeAndSessionWindowedDeserializerClassAreNotSet() { - final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>(); - assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); + try (final SessionWindowedDeserializer deserializer = new SessionWindowedDeserializer<>()) { + assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); + } } + @Deprecated @Test public void shouldThrowErrorIfDeserializersConflictInConstructorAndWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); @@ -87,6 +94,7 @@ public void shouldThrowErrorIfDeserializersConflictInConstructorAndWindowedInner assertThrows(IllegalArgumentException.class, () -> sessionWindowedDeserializer.configure(props, false)); } + @Deprecated @Test public void shouldThrowConfigExceptionWhenInvalidWindowedInnerClassSerdeSupplied() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java index d7e30bc3fe448..212b0c810e5f7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowedSerializerTest.java @@ -44,37 +44,44 @@ public void testSessionWindowedSerializerConstructor() { assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); } + @Deprecated @Test public void shouldSetSerializerThroughWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); - final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>(); - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + try (final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>()) { + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + } } @Test public void shouldSetSerializerThroughWindowedInnerSerializerClassConfig() { props.put(SessionWindowedSerializer.WINDOWED_INNER_SERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); - final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>(); - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + try (final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>()) { + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + } } + @Deprecated @Test public void shouldIgnoreWindowedInnerClassSerdeConfigIfWindowedInnerSerializerClassConfigIsSet() { props.put(SessionWindowedSerializer.WINDOWED_INNER_SERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); - final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>(); - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + try (final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>()) { + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + } } @Test public void shouldThrowErrorIfWindowedInnerClassSerdeAndWindowedInnerSerializerClassAreNotSet() { - final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>(); - assertThrows(IllegalArgumentException.class, () -> serializer.configure(props, false)); + try (final SessionWindowedSerializer serializer = new SessionWindowedSerializer<>()) { + assertThrows(IllegalArgumentException.class, () -> serializer.configure(props, false)); + } } + @Deprecated @Test public void shouldThrowErrorIfSerializersConflictInConstructorAndWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); @@ -87,6 +94,7 @@ public void shouldThrowErrorIfSerializersConflictInConstructorAndWindowedInnerSe assertThrows(IllegalArgumentException.class, () -> sessionWindowedSerializer.configure(props, false)); } + @Deprecated @Test public void shouldThrowConfigExceptionWhenInvalidWindowedInnerClassSerdeSupplied() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java index bfb8c80cf094d..d2485c7785d78 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedDeserializerTest.java @@ -49,42 +49,49 @@ public void testTimeWindowedDeserializerConstructor() { assertThat(timeWindowedDeserializer.getWindowSize(), is(5000000L)); } + @Deprecated @Test public void shouldSetWindowSizeAndDeserializerThroughWindowSizeMsAndWindowedInnerClassSerdeConfigs() { props.put(StreamsConfig.WINDOW_SIZE_MS_CONFIG, "500"); props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); - final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); - deserializer.configure(props, false); - assertThat(deserializer.getWindowSize(), is(500L)); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { + deserializer.configure(props, false); + assertThat(deserializer.getWindowSize(), is(500L)); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + } } @Test public void shouldSetWindowSizeAndDeserializerThroughWindowSizeMsAndWindowedInnerDeserializerClassConfigs() { props.put(TimeWindowedDeserializer.WINDOW_SIZE_MS_CONFIG, "500"); props.put(TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); - final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); - deserializer.configure(props, false); - assertThat(deserializer.getWindowSize(), is(500L)); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { + deserializer.configure(props, false); + assertThat(deserializer.getWindowSize(), is(500L)); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + } } + @Deprecated @Test public void shouldHaveSameConfigNameForWindowSizeMs() { assertEquals(TimeWindowedDeserializer.WINDOW_SIZE_MS_CONFIG, StreamsConfig.WINDOW_SIZE_MS_CONFIG); } + @Deprecated @Test public void shouldIgnoreWindowedInnerClassSerdeConfigIfWindowedInnerDeserializerClassConfigIsSet() { props.put(TimeWindowedDeserializer.WINDOW_SIZE_MS_CONFIG, "500"); props.put(TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); - final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); - deserializer.configure(props, false); - assertThat(deserializer.getWindowSize(), is(500L)); - assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { + deserializer.configure(props, false); + assertThat(deserializer.getWindowSize(), is(500L)); + assertInstanceOf(ByteArrayDeserializer.class, deserializer.innerDeserializer()); + } } + @Deprecated @Test public void shouldThrowErrorIfWindowSizeSetInStreamsConfigAndConstructor() { props.put(StreamsConfig.WINDOW_SIZE_MS_CONFIG, "500"); @@ -97,34 +104,41 @@ public void shouldThrowErrorIfWindowSizeSetInConstructorConfigAndConstructor() { assertThrows(IllegalArgumentException.class, () -> timeWindowedDeserializer.configure(props, false)); } + @Deprecated @Test public void shouldThrowErrorIfWindowSizeIsNotSetAndWindowedInnerClassSerdeIsSet() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); - final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); - assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); + try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { + assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); + } } @Test public void shouldThrowErrorIfWindowSizeIsNotSetAndWindowedInnerDeserializerClassIsSet() { - props.put(TimeWindowedDeserializer.WINDOW_SIZE_MS_CONFIG, Serdes.ByteArraySerde.class.getName()); - final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); - assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); + props.put(TimeWindowedDeserializer.WINDOWED_INNER_DESERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); + try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { + assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); + } } + @Deprecated @Test public void shouldThrowErrorIfWindowedInnerClassSerdeIsNotSetAndWindowSizeMsInStreamsConfigIsSet() { props.put(StreamsConfig.WINDOW_SIZE_MS_CONFIG, "500"); - final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); - assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); + try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { + assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); + } } @Test public void shouldThrowErrorIfWindowedInnerClassSerdeIsNotSetAndWindowSizeMsInConstructorConfigIsSet() { props.put(TimeWindowedDeserializer.WINDOW_SIZE_MS_CONFIG, "500"); - final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>(); - assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); + try (final TimeWindowedDeserializer deserializer = new TimeWindowedDeserializer<>()) { + assertThrows(IllegalArgumentException.class, () -> deserializer.configure(props, false)); + } } + @Deprecated @Test public void shouldThrowErrorIfDeserializerConflictInConstructorAndWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); @@ -137,6 +151,7 @@ public void shouldThrowErrorIfDeserializerConflictInConstructorAndWindowedInnerD assertThrows(IllegalArgumentException.class, () -> timeWindowedDeserializer.configure(props, false)); } + @Deprecated @Test public void shouldThrowConfigExceptionWhenInvalidWindowedInnerClassSerdeSupplied() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java index 7a13117db4ad7..5fd96f72c1bcf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowedSerializerTest.java @@ -44,37 +44,44 @@ public void testTimeWindowedSerializerConstructor() { assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); } + @Deprecated @Test public void shouldSetSerializerThroughWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); - final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>(); - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + try (final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>()) { + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + } } @Test public void shouldSetSerializerThroughWindowedInnerSerializerClassConfig() { props.put(TimeWindowedSerializer.WINDOWED_INNER_SERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); - final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>(); - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + try (final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>()) { + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + } } + @Deprecated @Test public void shouldIgnoreWindowedInnerClassSerdeConfigIfWindowedInnerSerializerClassConfigIsSet() { props.put(TimeWindowedSerializer.WINDOWED_INNER_SERIALIZER_CLASS, Serdes.ByteArraySerde.class.getName()); props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); - final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>(); - serializer.configure(props, false); - assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + try (final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>()) { + serializer.configure(props, false); + assertInstanceOf(ByteArraySerializer.class, serializer.innerSerializer()); + } } @Test public void shouldThrowErrorIfWindowedInnerClassSerdeAndWindowedInnerSerializerClassAreNotSet() { - final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>(); - assertThrows(IllegalArgumentException.class, () -> serializer.configure(props, false)); + try (final TimeWindowedSerializer serializer = new TimeWindowedSerializer<>()) { + assertThrows(IllegalArgumentException.class, () -> serializer.configure(props, false)); + } } + @Deprecated @Test public void shouldThrowErrorIfSerializerConflictInConstructorAndWindowedInnerClassSerdeConfig() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, Serdes.ByteArraySerde.class.getName()); @@ -87,6 +94,7 @@ public void shouldThrowErrorIfSerializerConflictInConstructorAndWindowedInnerSer assertThrows(IllegalArgumentException.class, () -> timeWindowedSerializer.configure(props, false)); } + @Deprecated @Test public void shouldThrowConfigExceptionWhenInvalidWindowedInnerClassSerdeSupplied() { props.put(StreamsConfig.WINDOWED_INNER_CLASS_SERDE, "some.non.existent.class"); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java index c6d4cf4d2390e..536d2e26ee892 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java @@ -62,7 +62,6 @@ import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -96,12 +95,7 @@ public class InternalTopicManagerTest { private final Node broker1 = new Node(0, "dummyHost-1", 1234); private final Node broker2 = new Node(1, "dummyHost-2", 1234); - private final List cluster = new ArrayList<>(2) { - { - add(broker1); - add(broker2); - } - }; + private final List cluster = List.of(broker1, broker2); private final String topic1 = "test_topic"; private final String topic2 = "test_topic_2"; private final String topic3 = "test_topic_3"; @@ -115,16 +109,14 @@ public class InternalTopicManagerTest { private InternalTopicManager internalTopicManager; private final MockTime time = new MockTime(0); - private final Map config = new HashMap<>() { - { - put(StreamsConfig.APPLICATION_ID_CONFIG, "app-id"); - put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, broker1.host() + ":" + broker1.port()); - put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 1); - put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), 16384); - put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), 100); - put(StreamsConfig.RETRY_BACKOFF_MS_CONFIG, 10); - } - }; + private final Map config = Map.of( + StreamsConfig.APPLICATION_ID_CONFIG, "app-id", + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, broker1.host() + ":" + broker1.port(), + StreamsConfig.REPLICATION_FACTOR_CONFIG, 1, + StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), 16384, + StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), 100, + StreamsConfig.RETRY_BACKOFF_MS_CONFIG, 10 + ); @BeforeEach public void init() { @@ -710,26 +702,38 @@ public void shouldCreateRequiredTopics() throws Exception { internalTopicManager.makeReady(Collections.singletonMap(topic4, topicConfig4)); assertEquals(Set.of(topic1, topic2, topic3, topic4), mockAdminClient.listTopics().names().get()); - assertEquals(new TopicDescription(topic1, false, new ArrayList<>() { - { - add(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList())); - } - }), mockAdminClient.describeTopics(Collections.singleton(topic1)).topicNameValues().get(topic1).get()); - assertEquals(new TopicDescription(topic2, false, new ArrayList<>() { - { - add(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList())); - } - }), mockAdminClient.describeTopics(Collections.singleton(topic2)).topicNameValues().get(topic2).get()); - assertEquals(new TopicDescription(topic3, false, new ArrayList<>() { - { - add(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList())); - } - }), mockAdminClient.describeTopics(Collections.singleton(topic3)).topicNameValues().get(topic3).get()); - assertEquals(new TopicDescription(topic4, false, new ArrayList<>() { - { - add(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList())); - } - }), mockAdminClient.describeTopics(Collections.singleton(topic4)).topicNameValues().get(topic4).get()); + assertEquals( + new TopicDescription( + topic1, + false, + List.of(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList())) + ), + mockAdminClient.describeTopics(Collections.singleton(topic1)).topicNameValues().get(topic1).get() + ); + assertEquals( + new TopicDescription( + topic2, + false, + List.of(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList())) + ), + mockAdminClient.describeTopics(Collections.singleton(topic2)).topicNameValues().get(topic2).get() + ); + assertEquals( + new TopicDescription( + topic3, + false, + List.of(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList())) + ), + mockAdminClient.describeTopics(Collections.singleton(topic3)).topicNameValues().get(topic3).get() + ); + assertEquals( + new TopicDescription( + topic4, + false, + List.of(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList())) + ), + mockAdminClient.describeTopics(Collections.singleton(topic4)).topicNameValues().get(topic4).get() + ); final ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, topic1); final ConfigResource resource2 = new ConfigResource(ConfigResource.Type.TOPIC, topic2); @@ -804,13 +808,12 @@ public void shouldNotCreateTopicIfExistsWithDifferentPartitions() { mockAdminClient.addTopic( false, topic1, - new ArrayList<>() { - { - add(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList())); - add(new TopicPartitionInfo(1, broker1, singleReplica, Collections.emptyList())); - } - }, - null); + List.of( + new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList()), + new TopicPartitionInfo(1, broker1, singleReplica, Collections.emptyList()) + ), + null + ); try { final InternalTopicConfig internalTopicConfig = new RepartitionTopicConfig(topic1, Collections.emptyMap()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ReadOnlyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ReadOnlyTaskTest.java index c95cf8c5a5674..1eb999a2ae665 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ReadOnlyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ReadOnlyTaskTest.java @@ -26,7 +26,6 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.Collections; -import java.util.LinkedList; import java.util.List; import java.util.Optional; import java.util.Set; @@ -41,32 +40,28 @@ class ReadOnlyTaskTest { - private final List readOnlyMethods = new LinkedList<>() { - { - add("needsInitializationOrRestoration"); - add("inputPartitions"); - add("changelogPartitions"); - add("commitRequested"); - add("commitNeeded"); - add("isActive"); - add("changelogOffsets"); - add("state"); - add("id"); - add("store"); - } - }; - - private final List objectMethods = new LinkedList<>() { - { - add("wait"); - add("equals"); - add("getClass"); - add("hashCode"); - add("notify"); - add("notifyAll"); - add("toString"); - } - }; + private final List readOnlyMethods = List.of( + "needsInitializationOrRestoration", + "inputPartitions", + "changelogPartitions", + "commitRequested", + "commitNeeded", + "isActive", + "changelogOffsets", + "state", + "id", + "store" + ); + + private final List objectMethods = List.of( + "wait", + "equals", + "getClass", + "hashCode", + "notify", + "notifyAll", + "toString" + ); final Task task = statelessTask(new TaskId(1, 0)).build(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java index 3ce8cb63efc10..bc01473e4e3a9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java @@ -58,11 +58,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.MatcherAssert.assertThat; @@ -81,41 +81,39 @@ @MockitoSettings(strictness = Strictness.STRICT_STUBS) public class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest { - private final List walRelatedMethods = new LinkedList<>() { - { - add("setManualWalFlush"); - add("setMaxTotalWalSize"); - add("setWalBytesPerSync"); - add("setWalDir"); - add("setWalFilter"); - add("setWalRecoveryMode"); - add("setWalSizeLimitMB"); - add("setWalTtlSeconds"); - } - }; - - private final List ignoreMethods = new LinkedList<>() { - { - add("isOwningHandle"); - add("getNativeHandle"); - add("dispose"); - add("wait"); - add("equals"); - add("getClass"); - add("hashCode"); - add("notify"); - add("notifyAll"); - add("toString"); - add("getOptionStringFromProps"); - add("maxBackgroundCompactions"); - add("setMaxBackgroundCompactions"); - add("maxBackgroundFlushes"); - add("setMaxBackgroundFlushes"); - add("tablePropertiesCollectorFactory"); - add("setTablePropertiesCollectorFactory"); - addAll(walRelatedMethods); - } - }; + private final List walRelatedMethods = List.of( + "setManualWalFlush", + "setMaxTotalWalSize", + "setWalBytesPerSync", + "setWalDir", + "setWalFilter", + "setWalRecoveryMode", + "setWalSizeLimitMB", + "setWalTtlSeconds" + ); + + private final List ignoreMethods = Stream.concat( + Stream.of( + "isOwningHandle", + "getNativeHandle", + "dispose", + "wait", + "equals", + "getClass", + "hashCode", + "notify", + "notifyAll", + "toString", + "getOptionStringFromProps", + "maxBackgroundCompactions", + "setMaxBackgroundCompactions", + "maxBackgroundFlushes", + "setMaxBackgroundFlushes", + "tablePropertiesCollectorFactory", + "setTablePropertiesCollectorFactory" + ), + walRelatedMethods.stream() + ).collect(Collectors.toList()); @Test public void shouldOverwriteAllOptionsMethods() throws Exception { diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java index 4bca7f995a52a..ecd2068f5fd09 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java @@ -635,6 +635,7 @@ public static Properties getConfig(final String broker, final String application, final String id, final String processingGuarantee, + final String groupProtocol, final String stateDir) { final Properties properties = mkProperties( @@ -644,7 +645,8 @@ public static Properties getConfig(final String broker, mkEntry(StreamsConfig.CLIENT_ID_CONFIG, id), mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuarantee), mkEntry(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"), - mkEntry(StreamsConfig.STATE_DIR_CONFIG, stateDir) + mkEntry(StreamsConfig.STATE_DIR_CONFIG, stateDir), + mkEntry(StreamsConfig.GROUP_PROTOCOL_CONFIG, groupProtocol) ) ); properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L); @@ -656,9 +658,10 @@ public static KafkaStreams startSync(final String broker, final String application, final String id, final String processingGuarantee, + final String groupProtocol, final String stateDir) throws InterruptedException { final KafkaStreams kafkaStreams = - new KafkaStreams(getTopology(), getConfig(broker, application, id, processingGuarantee, stateDir)); + new KafkaStreams(getTopology(), getConfig(broker, application, id, processingGuarantee, groupProtocol, stateDir)); final CountDownLatch startUpLatch = new CountDownLatch(1); kafkaStreams.setStateListener((newState, oldState) -> { if (oldState == KafkaStreams.State.REBALANCING && newState == KafkaStreams.State.RUNNING) { @@ -989,8 +992,9 @@ public static void main(final String[] args) { case "application": { final String nodeId = args[2]; final String processingGuarantee = args[3]; - final String stateDir = args[4]; - App.startSync(kafka, UUID.randomUUID().toString(), nodeId, processingGuarantee, stateDir); + final String groupProtocol = args[4]; + final String stateDir = args[5]; + App.startSync(kafka, UUID.randomUUID().toString(), nodeId, processingGuarantee, groupProtocol, stateDir); break; } default: diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTestTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTestTest.java index 4cf19de4c36ac..bbe7d2dad0fdf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTestTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTestTest.java @@ -45,6 +45,7 @@ public void verifySmokeTestLogic() { "test", "test", StreamsConfig.AT_LEAST_ONCE, + "classic", TestUtils.tempDirectory().getAbsolutePath() ))) { diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java index d23915790b2cf..27771b5be16af 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java @@ -64,7 +64,6 @@ public static void main(final String[] args) throws IOException { Exit.exit(1); } - streamsProperties.put(StreamsConfig.APPLICATION_ID_CONFIG, "kafka-streams-standby-tasks"); streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); streamsProperties.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); streamsProperties.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); diff --git a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/ClusterInstance.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/ClusterInstance.java index 4b887eff9d68e..d07e3bd2ea2e0 100644 --- a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/ClusterInstance.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/ClusterInstance.java @@ -290,6 +290,19 @@ default void createTopic(String topicName, int partitions, short replicas, Map/dev/null; then must_do -v docker network rm ducknet fi - must_do -v docker network create ducknet + network_create_args="" + if [[ "${ipv6}" == "true" ]]; then + subnet_cidr_prefix="${DUCKER_SUBNET_CIDR:-"fc00:cf17"}" + network_create_args="--ipv6 --subnet ${subnet_cidr_prefix}::/64" + fi + must_do -v docker network create ${network_create_args} ducknet if [[ -n "${custom_ducktape}" ]]; then setup_custom_ducktape "${custom_ducktape}" "${image_name}" fi @@ -412,7 +421,11 @@ attempting to start new ones." exec 3<> "${ducker_dir}/build/node_hosts" for n in $(seq -f %02g 1 ${num_nodes}); do local node="ducker${n}" - docker exec --user=root "${node}" grep "${node}" /etc/hosts >&3 + if [[ "${ipv6}" == "true" ]]; then + docker exec --user=root "${node}" grep "${node}" /etc/hosts | grep "${subnet_cidr_prefix}" >&3 + else + docker exec --user=root "${node}" grep "${node}" /etc/hosts >&3 + fi [[ $? -ne 0 ]] && die "failed to find the /etc/hosts entry for ${node}" done exec 3>&- @@ -421,6 +434,12 @@ attempting to start new ones." docker exec --user=root "${node}" \ bash -c "grep -v ${node} /opt/kafka-dev/tests/docker/build/node_hosts >> /etc/hosts" [[ $? -ne 0 ]] && die "failed to append to the /etc/hosts file on ${node}" + # Filter out ipv4 addresses if ipv6 + if [[ "${ipv6}" == "true" ]]; then + docker exec --user=root "${node}" \ + bash -c "grep -v -E '([0-9]{1,3}\.){3}[0-9]{1,3}' /opt/kafka-dev/tests/docker/build/node_hosts >> /etc/hosts" + [[ $? -ne 0 ]] && die "failed to append to the /etc/hosts file on ${node}" + fi done if [ "$kafka_mode" == "native" ]; then diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index ab02856c4b3e3..329df370d195a 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -74,7 +74,6 @@ DELEGATION_TOKEN_SECRET_KEY="delegation.token.secret.key" SASL_ENABLED_MECHANISMS="sasl.enabled.mechanisms" -GROUP_COORDINATOR_REBALANCE_PROTOCOLS="group.coordinator.rebalance.protocols" CONSUMER_GROUP_MIGRATION_POLICY = "group.consumer.migration.policy" SHARE_COORDINATOR_STATE_TOPIC_REPLICATION_FACTOR ="share.coordinator.state.topic.replication.factor" @@ -82,6 +81,7 @@ SHARE_GROUP_ENABLE = "group.share.enable" UNSTABLE_API_VERSIONS_ENABLE = "unstable.api.versions.enable" +UNSTABLE_FEATURE_VERSIONS_ENABLE = "unstable.feature.versions.enable" """ From KafkaConfig.scala diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 47023ff31c65a..ca19ca8bd1147 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -300,10 +300,6 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.use_share_groups = use_share_groups self.use_streams_groups = use_streams_groups - # Set offsets_commit_timeout based on context - if context.injected_args is not None: - self.offsets_commit_timeout = context.injected_args.get('offsets_commit_timeout') - # Set consumer_group_migration_policy based on context and arguments. if consumer_group_migration_policy is None: arg_name = 'consumer_group_migration_policy' @@ -757,8 +753,7 @@ def prop_file(self, node): config_template = self.render('kafka.properties', node=node, broker_id=self.idx(node), security_config=self.security_config, num_nodes=self.num_nodes, listener_security_config=self.listener_security_config, - use_share_groups=self.use_share_groups, - offsets_commit_timeout=self.offsets_commit_timeout) + use_share_groups=self.use_share_groups) configs = dict( l.rstrip().split('=', 1) for l in config_template.split('\n') if not l.startswith("#") and "=" in l ) @@ -784,17 +779,12 @@ def prop_file(self, node): for prop in self.per_node_server_prop_overrides.get(self.idx(node), []): override_configs[prop[0]] = prop[1] - enabledProtocols = 'classic,consumer' - if self.use_share_groups is not None and self.use_share_groups is True: override_configs[config_property.SHARE_GROUP_ENABLE] = str(self.use_share_groups) - enabledProtocols += ',share' if self.use_streams_groups is True: override_configs[config_property.UNSTABLE_API_VERSIONS_ENABLE] = str(True) - enabledProtocols += ',streams' - - override_configs[config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS] = enabledProtocols + override_configs[config_property.UNSTABLE_FEATURE_VERSIONS_ENABLE] = str(True) #update template configs with test override configs configs.update(override_configs) diff --git a/tests/kafkatest/services/kafka/templates/kafka.properties b/tests/kafkatest/services/kafka/templates/kafka.properties index e0b854595904b..861c63014c532 100644 --- a/tests/kafkatest/services/kafka/templates/kafka.properties +++ b/tests/kafkatest/services/kafka/templates/kafka.properties @@ -130,8 +130,4 @@ group.initial.rebalance.delay.ms=100 {% if use_share_groups is not none and use_share_groups %} share.coordinator.state.topic.replication.factor={{ 3 if num_nodes > 3 else num_nodes }} share.coordinator.state.topic.min.isr=1 -{% endif %} - -{% if offsets_commit_timeout is defined and offsets_commit_timeout is not none %} -offsets.commit.timeout.ms={{ offsets_commit_timeout }} {% endif %} \ No newline at end of file diff --git a/tests/kafkatest/tests/client/share_consumer_test.py b/tests/kafkatest/tests/client/share_consumer_test.py index 8367b1f409791..f47ac1e771b42 100644 --- a/tests/kafkatest/tests/client/share_consumer_test.py +++ b/tests/kafkatest/tests/client/share_consumer_test.py @@ -208,17 +208,15 @@ def test_broker_rolling_bounce(self, clean_shutdown, metadata_quorum=quorum.isol clean_shutdown=[True, False], metadata_quorum=[quorum.isolated_kraft], num_failed_brokers=[1, 2], - use_share_groups=[True], - offsets_commit_timeout=[20000] + use_share_groups=[True] ) @matrix( clean_shutdown=[True, False], metadata_quorum=[quorum.combined_kraft], num_failed_brokers=[1], - use_share_groups=[True], - offsets_commit_timeout=[20000] + use_share_groups=[True] ) - def test_broker_failure(self, clean_shutdown, metadata_quorum=quorum.isolated_kraft, num_failed_brokers=1, use_share_groups=True, offsets_commit_timeout=20000): + def test_broker_failure(self, clean_shutdown, metadata_quorum=quorum.isolated_kraft, num_failed_brokers=1, use_share_groups=True): producer = self.setup_producer(self.TOPIC2["name"]) consumer = self.setup_share_group(self.TOPIC2["name"], offset_reset_strategy="earliest") diff --git a/tests/kafkatest/tests/streams/base_streams_test.py b/tests/kafkatest/tests/streams/base_streams_test.py index 00b4c37889705..81cad7a4d1b8d 100644 --- a/tests/kafkatest/tests/streams/base_streams_test.py +++ b/tests/kafkatest/tests/streams/base_streams_test.py @@ -33,6 +33,7 @@ def __init__(self, test_context, topics, num_controllers=1, num_brokers=3): self.num_controllers = num_controllers self.num_brokers = num_brokers self.topics = topics + self.use_streams_groups = True self.kafka = KafkaService( test_context, self.num_brokers, @@ -47,6 +48,8 @@ def __init__(self, test_context, topics, num_controllers=1, num_brokers=3): def setUp(self): self.kafka.start() + if self.use_streams_groups: + self.kafka.run_features_command("upgrade", "streams.version", 1) def get_consumer(self, client_id, topic, num_messages): return VerifiableConsumer(self.test_context, @@ -95,6 +98,17 @@ def assert_consume(self, client_id, test_state, topic, num_messages=5, timeout_s timeout_sec=timeout_sec, err_msg="At %s streams did not process messages in %s seconds " % (test_state, timeout_sec)) + def configure_standby_replicas(self, group_id, num_standby_replicas): + force_use_zk_connection = not self.kafka.all_nodes_configs_command_uses_bootstrap_server() + node = self.kafka.nodes[0] + cmd = "%s --alter --add-config streams.num.standby.replicas=%d --entity-type groups --entity-name %s" % \ + ( + self.kafka.kafka_configs_cmd_with_optional_security_settings(node, force_use_zk_connection), + num_standby_replicas, + group_id + ) + node.account.ssh(cmd) + @staticmethod def get_configs(group_protocol="classic", extra_configs=""): # Consumer max.poll.interval > min(max.block.ms, ((retries + 1) * request.timeout) diff --git a/tests/kafkatest/tests/streams/streams_relational_smoke_test.py b/tests/kafkatest/tests/streams/streams_relational_smoke_test.py index c174948ef00e1..72deeded059a1 100644 --- a/tests/kafkatest/tests/streams/streams_relational_smoke_test.py +++ b/tests/kafkatest/tests/streams/streams_relational_smoke_test.py @@ -20,13 +20,13 @@ from kafkatest.services.kafka import quorum from kafkatest.services.kafka.util import get_log4j_config_param, get_log4j_config_for_tools from kafkatest.services.streams import StreamsTestBaseService -from kafkatest.tests.kafka_test import KafkaTest +from kafkatest.tests.streams.base_streams_test import BaseStreamsTest from kafkatest.version import LATEST_4_0 from kafkatest.version import get_version class StreamsRelationalSmokeTestService(StreamsTestBaseService): - def __init__(self, test_context, kafka, mode, nodeId, processing_guarantee): + def __init__(self, test_context, kafka, mode, nodeId, processing_guarantee, group_protocol): super(StreamsRelationalSmokeTestService, self).__init__( test_context, kafka, @@ -36,12 +36,13 @@ def __init__(self, test_context, kafka, mode, nodeId, processing_guarantee): self.mode = mode self.nodeId = nodeId self.processing_guarantee = processing_guarantee + self.group_protocol = group_protocol self.log4j_template = "log4j2_template.yaml" if (get_version(self.node) >= LATEST_4_0) else "log4j_template.properties" def start_cmd(self, node): return "( export KAFKA_LOG4J_OPTS=\"%(log4j_param)s%(log4j)s\"; " \ "INCLUDE_TEST_JARS=true %(kafka_run_class)s org.apache.kafka.streams.tests.RelationalSmokeTest " \ - " %(mode)s %(kafka)s %(nodeId)s %(processing_guarantee)s %(state_dir)s" \ + " %(mode)s %(kafka)s %(nodeId)s %(processing_guarantee)s %(group_protocol)s %(state_dir)s" \ " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % { "log4j_param": get_log4j_config_param(node), "log4j": get_log4j_config_for_tools(node), @@ -50,6 +51,7 @@ def start_cmd(self, node): "kafka": self.kafka.bootstrap_servers(), "nodeId": self.nodeId, "processing_guarantee": self.processing_guarantee, + "group_protocol": self.group_protocol, "state_dir": self.PERSISTENT_ROOT, "stdout": self.STDOUT_FILE, "stderr": self.STDERR_FILE, @@ -75,13 +77,13 @@ def await_command(self, command): ) -class StreamsRelationalSmokeTest(KafkaTest): +class StreamsRelationalSmokeTest(BaseStreamsTest): """ Simple test of Kafka Streams. """ def __init__(self, test_context): - super(StreamsRelationalSmokeTest, self).__init__(test_context, num_zk=1, num_brokers=3, topics={ + super(StreamsRelationalSmokeTest, self).__init__(test_context, num_controllers=1, num_brokers=3, topics={ 'in-article': {'partitions': 3, 'replication-factor': 1}, 'in-comment': {'partitions': 5, 'replication-factor': 1}, 'out-augmented-article': {'partitions': 3, 'replication-factor': 1}, @@ -91,15 +93,16 @@ def __init__(self, test_context): @cluster(num_nodes=8) @matrix(crash=[False, True], - metadata_quorum=[quorum.combined_kraft]) - def test_streams(self, crash, metadata_quorum): - driver = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "driver", "ignored", "ignored") + metadata_quorum=[quorum.combined_kraft], + group_protocol=["classic", "streams"]) + def test_streams(self, crash, metadata_quorum, group_protocol): + driver = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "driver", "ignored", "ignored", "ignored") processing_guarantee='exactly_once_v2' LOG_FILE = driver.LOG_FILE # this is the same for all instances of the service, so we can just declare a "constant" - processor1 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor1", processing_guarantee) - processor2 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor2", processing_guarantee) + processor1 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor1", processing_guarantee, group_protocol) + processor2 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor2", processing_guarantee, group_protocol) processor1.start() processor2.start() @@ -114,7 +117,7 @@ def test_streams(self, crash, metadata_quorum): processor1.stop_nodes(not crash) - processor3 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor3", processing_guarantee) + processor3 = StreamsRelationalSmokeTestService(self.test_context, self.kafka, "application", "processor3", processing_guarantee, group_protocol) processor3.start() processor3.await_command("grep -q 'Streams has started' %s" % LOG_FILE) diff --git a/tests/kafkatest/tests/streams/streams_standby_replica_test.py b/tests/kafkatest/tests/streams/streams_standby_replica_test.py index 8270652645a5e..77e3c65aab93c 100644 --- a/tests/kafkatest/tests/streams/streams_standby_replica_test.py +++ b/tests/kafkatest/tests/streams/streams_standby_replica_test.py @@ -47,19 +47,23 @@ def __init__(self, test_context): }) @cluster(num_nodes=10) - @matrix(metadata_quorum=[quorum.isolated_kraft]) - def test_standby_tasks_rebalance(self, metadata_quorum): + @matrix(metadata_quorum=[quorum.combined_kraft], + group_protocol=["classic", "streams"]) + def test_standby_tasks_rebalance(self, metadata_quorum, group_protocol): # TODO KIP-441: consider rewriting the test for HighAvailabilityTaskAssignor configs = self.get_configs( - group_protocol="classic", + group_protocol=group_protocol, extra_configs= - ",sourceTopic=%s,sinkTopic1=%s,sinkTopic2=%s,internal.task.assignor.class=org.apache.kafka.streams.processor.internals.assignment.LegacyStickyTaskAssignor" % ( + ",application.id=test_standby_tasks_rebalance,sourceTopic=%s,sinkTopic1=%s,sinkTopic2=%s,internal.task.assignor.class=org.apache.kafka.streams.processor.internals.assignment.LegacyStickyTaskAssignor" % ( self.streams_source_topic, self.streams_sink_topic_1, self.streams_sink_topic_2 ) ) + if group_protocol == "streams": + self.configure_standby_replicas("test_standby_tasks_rebalance", 1) + producer = self.get_producer(self.streams_source_topic, self.num_messages, throughput=15000, repeating_keys=6) producer.start() diff --git a/server-common/src/main/java/org/apache/kafka/server/common/AdminCommandFailedException.java b/tools/src/main/java/org/apache/kafka/tools/AdminCommandFailedException.java similarity index 96% rename from server-common/src/main/java/org/apache/kafka/server/common/AdminCommandFailedException.java rename to tools/src/main/java/org/apache/kafka/tools/AdminCommandFailedException.java index 62bb842554607..0af83023f0486 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/AdminCommandFailedException.java +++ b/tools/src/main/java/org/apache/kafka/tools/AdminCommandFailedException.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.server.common; +package org.apache.kafka.tools; public class AdminCommandFailedException extends RuntimeException { public AdminCommandFailedException(String message) { diff --git a/server-common/src/main/java/org/apache/kafka/server/common/AdminOperationException.java b/tools/src/main/java/org/apache/kafka/tools/AdminOperationException.java similarity index 96% rename from server-common/src/main/java/org/apache/kafka/server/common/AdminOperationException.java rename to tools/src/main/java/org/apache/kafka/tools/AdminOperationException.java index 03826d1ac00c5..2595610fbbb21 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/AdminOperationException.java +++ b/tools/src/main/java/org/apache/kafka/tools/AdminOperationException.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.server.common; +package org.apache.kafka.tools; public class AdminOperationException extends RuntimeException { public AdminOperationException(String message) { diff --git a/tools/src/main/java/org/apache/kafka/tools/ClusterTool.java b/tools/src/main/java/org/apache/kafka/tools/ClusterTool.java index 370d756d493d7..ccffaeae0f1d0 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ClusterTool.java +++ b/tools/src/main/java/org/apache/kafka/tools/ClusterTool.java @@ -167,7 +167,7 @@ static void listEndpoints(PrintStream stream, Admin adminClient, boolean listCon if (listControllerEndpoints) { String format = "%-10s %-" + maxHostLength + "s %-10s %-" + maxRackLength + "s %-15s%n"; stream.printf(format, "ID", "HOST", "PORT", "RACK", "ENDPOINT_TYPE"); - nodes.stream().forEach(node -> stream.printf(format, + nodes.forEach(node -> stream.printf(format, node.idString(), node.host(), node.port(), @@ -177,7 +177,7 @@ static void listEndpoints(PrintStream stream, Admin adminClient, boolean listCon } else { String format = "%-10s %-" + maxHostLength + "s %-10s %-" + maxRackLength + "s %-10s %-15s%n"; stream.printf(format, "ID", "HOST", "PORT", "RACK", "STATE", "ENDPOINT_TYPE"); - nodes.stream().forEach(node -> stream.printf(format, + nodes.forEach(node -> stream.printf(format, node.idString(), node.host(), node.port(), diff --git a/tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java b/tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java index 8e68b174d83e9..2756675299261 100644 --- a/tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/DeleteRecordsCommand.java @@ -23,8 +23,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.server.common.AdminCommandFailedException; -import org.apache.kafka.server.common.AdminOperationException; import org.apache.kafka.server.util.CommandDefaultOptions; import org.apache.kafka.server.util.CommandLineUtils; import org.apache.kafka.server.util.Json; diff --git a/tools/src/main/java/org/apache/kafka/tools/LeaderElectionCommand.java b/tools/src/main/java/org/apache/kafka/tools/LeaderElectionCommand.java index 6fd652199e42a..2814034e167aa 100644 --- a/tools/src/main/java/org/apache/kafka/tools/LeaderElectionCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/LeaderElectionCommand.java @@ -25,8 +25,6 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.server.common.AdminCommandFailedException; -import org.apache.kafka.server.common.AdminOperationException; import org.apache.kafka.server.util.CommandDefaultOptions; import org.apache.kafka.server.util.CommandLineUtils; import org.apache.kafka.server.util.Json; diff --git a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java index ae6015ec18a5d..e90bbff8a6241 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java @@ -42,6 +42,7 @@ import java.util.List; import java.util.Optional; import java.util.Properties; +import java.util.Scanner; import java.util.SplittableRandom; import static net.sourceforge.argparse4j.impl.Arguments.store; @@ -194,13 +195,17 @@ static List readPayloadFile(String payloadFilePath, String payloadDelimi throw new IllegalArgumentException("File does not exist or empty file provided."); } - String[] payloadList = Files.readString(path).split(payloadDelimiter); + try (Scanner payLoadScanner = new Scanner(path, StandardCharsets.UTF_8)) { + //setting the delimiter while parsing the file, avoids loading entire data in memory before split + payLoadScanner.useDelimiter(payloadDelimiter); + while (payLoadScanner.hasNext()) { + byte[] payloadBytes = payLoadScanner.next().getBytes(StandardCharsets.UTF_8); + payloadByteList.add(payloadBytes); + } + } - System.out.println("Number of messages read: " + payloadList.length); + System.out.println("Number of messages read: " + payloadByteList.size()); - for (String payload : payloadList) { - payloadByteList.add(payload.getBytes(StandardCharsets.UTF_8)); - } } return payloadByteList; } diff --git a/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java index 32272d4e92685..9c67917d76a66 100644 --- a/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java @@ -46,8 +46,6 @@ import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.server.common.AdminCommandFailedException; -import org.apache.kafka.server.common.AdminOperationException; import org.apache.kafka.server.util.CommandDefaultOptions; import org.apache.kafka.server.util.CommandLineUtils; import org.apache.kafka.storage.internals.log.LogConfig; diff --git a/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java index 508347c819e8c..c1dbefcef6045 100644 --- a/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/TransactionsCommand.java @@ -483,12 +483,18 @@ public void addSubparser(Subparsers subparsers) { .action(store()) .type(Long.class) .required(false); + subparser.addArgument("--transactional-id-pattern") + .help("Transactional id regular expression pattern to filter by") + .action(store()) + .type(String.class) + .required(false); } @Override public void execute(Admin admin, Namespace ns, PrintStream out) throws Exception { ListTransactionsOptions options = new ListTransactionsOptions(); Optional.ofNullable(ns.getLong("duration_filter")).ifPresent(options::filterOnDuration); + Optional.ofNullable(ns.getString("transactional_id_pattern")).ifPresent(options::filterOnTransactionalIdPattern); final Map> result; diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java index b001ae7c6f7fd..94d24fc34a9e3 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommand.java @@ -177,10 +177,11 @@ static Set groupStatesFromString(String input) { @SuppressWarnings("Regexp") static Set consumerGroupTypesFromString(String input) { + Set validTypes = Set.of(GroupType.CLASSIC, GroupType.CONSUMER); Set parsedTypes = Stream.of(input.toLowerCase().split(",")).map(s -> GroupType.parse(s.trim())).collect(Collectors.toSet()); - if (parsedTypes.contains(GroupType.UNKNOWN)) { - List validTypes = Arrays.stream(GroupType.values()).filter(t -> t != GroupType.UNKNOWN).map(Object::toString).collect(Collectors.toList()); - throw new IllegalArgumentException("Invalid types list '" + input + "'. Valid types are: " + String.join(", ", validTypes)); + if (!validTypes.containsAll(parsedTypes)) { + throw new IllegalArgumentException("Invalid types list '" + input + "'. Valid types are: " + + String.join(", ", validTypes.stream().map(GroupType::toString).collect(Collectors.toSet()))); } return parsedTypes; } @@ -779,30 +780,29 @@ void deleteOffsets() { switch (topLevelResult) { case NONE: - System.out.println("Request succeed for deleting offsets with topic " + String.join(", ", topics) + " group " + groupId); + System.out.println("Request succeeded for deleting offsets from group " + groupId + "."); break; case INVALID_GROUP_ID: - printError("'" + groupId + "' is not valid.", Optional.empty()); - break; case GROUP_ID_NOT_FOUND: - printError("'" + groupId + "' does not exist.", Optional.empty()); - break; case GROUP_AUTHORIZATION_FAILED: - printError("Access to '" + groupId + "' is not authorized.", Optional.empty()); - break; case NON_EMPTY_GROUP: - printError("Deleting offsets of a consumer group '" + groupId + "' is forbidden if the group is not empty.", Optional.empty()); + printError(topLevelResult.message(), Optional.empty()); break; case GROUP_SUBSCRIBED_TO_TOPIC: case TOPIC_AUTHORIZATION_FAILED: case UNKNOWN_TOPIC_OR_PARTITION: - printError("Encounter some partition level error, see the follow-up details:", Optional.empty()); + printError("Encountered some partition-level error, see the follow-up details.", Optional.empty()); break; default: - printError("Encounter some unknown error: " + topLevelResult, Optional.empty()); + printError("Encountered some unknown error: " + topLevelResult, Optional.empty()); + } + + int maxTopicLen = 15; + for (TopicPartition tp : partitionLevelResult.keySet()) { + maxTopicLen = Math.max(maxTopicLen, tp.topic().length()); } - String format = "%n%-30s %-15s %-15s"; + String format = "%n%" + (-maxTopicLen) + "s %-10s %-15s"; System.out.printf(format, "TOPIC", "PARTITION", "STATUS"); partitionLevelResult.entrySet().stream() @@ -812,7 +812,7 @@ void deleteOffsets() { Throwable error = e.getValue(); System.out.printf(format, tp.topic(), - tp.partition() >= 0 ? tp.partition() : "Not Provided", + tp.partition() >= 0 ? tp.partition() : MISSING_COLUMN_VALUE, error != null ? "Error: " + error.getMessage() : "Successful" ); }); diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommand.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommand.java index dcebff0d3ea62..dcbf25109220d 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommand.java @@ -19,6 +19,8 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AbstractOptions; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.DeleteShareGroupOffsetsOptions; +import org.apache.kafka.clients.admin.DeleteShareGroupOffsetsResult; import org.apache.kafka.clients.admin.DeleteShareGroupsOptions; import org.apache.kafka.clients.admin.DescribeShareGroupsOptions; import org.apache.kafka.clients.admin.GroupListing; @@ -28,11 +30,13 @@ import org.apache.kafka.clients.admin.ShareGroupDescription; import org.apache.kafka.clients.admin.ShareMemberAssignment; import org.apache.kafka.clients.admin.ShareMemberDescription; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.GroupState; import org.apache.kafka.common.GroupType; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.GroupNotEmptyException; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.util.CommandLineUtils; @@ -41,6 +45,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashSet; @@ -89,7 +94,7 @@ public static void run(ShareGroupCommandOptions opts) { } else if (opts.options.has(opts.resetOffsetsOpt)) { throw new UnsupportedOperationException("--reset-offsets option is not yet implemented"); } else if (opts.options.has(opts.deleteOffsetsOpt)) { - throw new UnsupportedOperationException("--delete-offsets option is not yet implemented"); + shareGroupService.deleteOffsets(); } } catch (IllegalArgumentException e) { CommandLineUtils.printUsageAndExit(opts.parser, e.getMessage()); @@ -148,9 +153,8 @@ public void listGroups() throws ExecutionException, InterruptedException { List listShareGroups() { try { - ListGroupsResult result = adminClient.listGroups(new ListGroupsOptions() - .timeoutMs(opts.options.valueOf(opts.timeoutMsOpt).intValue()) - .withTypes(Set.of(GroupType.SHARE))); + ListGroupsResult result = adminClient.listGroups(ListGroupsOptions.forShareGroups() + .timeoutMs(opts.options.valueOf(opts.timeoutMsOpt).intValue())); Collection listings = result.all().get(); return listings.stream().map(GroupListing::groupId).collect(Collectors.toList()); } catch (InterruptedException | ExecutionException e) { @@ -160,9 +164,8 @@ List listShareGroups() { List listDetailedShareGroups() { try { - ListGroupsResult result = adminClient.listGroups(new ListGroupsOptions() - .timeoutMs(opts.options.valueOf(opts.timeoutMsOpt).intValue()) - .withTypes(Set.of(GroupType.SHARE))); + ListGroupsResult result = adminClient.listGroups(ListGroupsOptions.forShareGroups() + .timeoutMs(opts.options.valueOf(opts.timeoutMsOpt).intValue())); Collection listings = result.all().get(); return listings.stream().toList(); } catch (InterruptedException | ExecutionException e) { @@ -171,9 +174,8 @@ List listDetailedShareGroups() { } List listShareGroupsInStates(Set states) throws ExecutionException, InterruptedException { - ListGroupsResult result = adminClient.listGroups(new ListGroupsOptions() + ListGroupsResult result = adminClient.listGroups(ListGroupsOptions.forShareGroups() .timeoutMs(opts.options.valueOf(opts.timeoutMsOpt).intValue()) - .withTypes(Set.of(GroupType.SHARE)) .inGroupStates(states)); return new ArrayList<>(result.all().get()); } @@ -285,6 +287,85 @@ Map deleteShareGroups() { return failed; } + void deleteOffsets() { + String groupId = opts.options.valueOf(opts.groupOpt); + List topics = opts.options.valuesOf(opts.topicOpt); + + Entry> res = sendDeleteShareGroupOffsetsRequest(groupId, new HashSet<>(topics)); + + Throwable topLevelResult = res.getKey(); + Map topicLevelResult = res.getValue(); + + if (topLevelResult != null) { + Errors topLevelError = Errors.forException(topLevelResult); + switch (topLevelError) { + case INVALID_GROUP_ID: + case GROUP_ID_NOT_FOUND: + case GROUP_AUTHORIZATION_FAILED: + case NON_EMPTY_GROUP: + printError(topLevelResult.getMessage(), Optional.empty()); + break; + case TOPIC_AUTHORIZATION_FAILED: + case UNKNOWN_TOPIC_OR_PARTITION: + // These are expected topic-level errors which will be reported in the topic-level results + break; + default: + printError("Encounter some unknown error: " + topLevelResult, Optional.empty()); + } + } + + if (topicLevelResult != null && !topicLevelResult.isEmpty()) { + int maxTopicLen = 15; + for (String topic : topicLevelResult.keySet()) { + maxTopicLen = Math.max(maxTopicLen, topic.length()); + } + + String format = "%n%" + (-maxTopicLen) + "s %s"; + + System.out.printf(format, "TOPIC", "STATUS"); + topicLevelResult.entrySet().stream() + .sorted(Entry.comparingByKey()) + .forEach(e -> { + String topic = e.getKey(); + Throwable error = e.getValue(); + System.out.printf(format, + topic, + error != null ? "Error: " + error.getMessage() : "Successful" + ); + }); + } + + System.out.println(); + } + + Entry> sendDeleteShareGroupOffsetsRequest(String groupId, Set topics) { + Map topicLevelResult = new HashMap<>(); + + DeleteShareGroupOffsetsResult deleteResult = adminClient.deleteShareGroupOffsets( + groupId, + new HashSet<>(topics), + withTimeoutMs(new DeleteShareGroupOffsetsOptions())); + + Throwable topLevelException = null; + + try { + deleteResult.all().get(); + } catch (ExecutionException | InterruptedException e) { + topLevelException = e.getCause(); + } + + topics.forEach(topic -> { + try { + deleteResult.topicResult(topic).get(); + topicLevelResult.put(topic, null); + } catch (ExecutionException | InterruptedException e) { + topicLevelResult.put(topic, e.getCause()); + } + }); + + return new SimpleImmutableEntry<>(topLevelException, topicLevelResult); + } + private > T withTimeoutMs(T options) { int t = opts.options.valueOf(opts.timeoutMsOpt).intValue(); return options.timeoutMs(t); @@ -320,19 +401,30 @@ TreeMap earliestResult = adminClient.listShareGroupOffsets(groupSpecs).all().get().get(groupId); + Map startOffsets = adminClient.listShareGroupOffsets(groupSpecs).all().get().get(groupId); Set partitionOffsets = new HashSet<>(); - for (Entry tp : earliestResult.entrySet()) { - SharePartitionOffsetInformation partitionOffsetInfo = new SharePartitionOffsetInformation( - groupId, - tp.getKey().topic(), - tp.getKey().partition(), - Optional.ofNullable(earliestResult.get(tp.getKey())) - ); - partitionOffsets.add(partitionOffsetInfo); - } + startOffsets.forEach((tp, offsetAndMetadata) -> { + if (offsetAndMetadata != null) { + partitionOffsets.add(new SharePartitionOffsetInformation( + groupId, + tp.topic(), + tp.partition(), + Optional.of(offsetAndMetadata.offset()), + offsetAndMetadata.leaderEpoch() + )); + } else { + partitionOffsets.add(new SharePartitionOffsetInformation( + groupId, + tp.topic(), + tp.partition(), + Optional.empty(), + Optional.empty() + )); + } + }); + groupOffsets.put(groupId, new SimpleImmutableEntry<>(shareGroup, partitionOffsets)); } catch (InterruptedException | ExecutionException e) { throw new RuntimeException(e); @@ -344,7 +436,12 @@ TreeMap>> offsets, boolean verbose) { offsets.forEach((groupId, tuple) -> { - Collection offsetsInfo = tuple.getValue(); + Collection offsetsInfo = tuple.getValue().stream() + .sorted(Comparator + .comparing((SharePartitionOffsetInformation info) -> info.topic) + .thenComparingInt(info -> info.partition)) + .toList(); + String fmt = printOffsetFormat(groupId, offsetsInfo, verbose); if (verbose) { @@ -359,7 +456,7 @@ private void printOffsets(TreeMap descriptions, b descriptions.forEach((groupId, description) -> { int groupLen = Math.max(15, groupId.length()); int maxConsumerIdLen = 15, maxHostLen = 15, maxClientIdLen = 15; - Collection members = description.members(); + Collection members = description.members() + .stream() + .sorted(Comparator.comparing(ShareMemberDescription::consumerId)) + .toList(); if (maybePrintEmptyGroupState(groupId, description.groupState(), description.members().size())) { for (ShareMemberDescription member : members) { maxConsumerIdLen = Math.max(maxConsumerIdLen, member.consumerId().length()); @@ -478,17 +578,20 @@ static class SharePartitionOffsetInformation { final String topic; final int partition; final Optional offset; + final Optional leaderEpoch; SharePartitionOffsetInformation( String group, String topic, int partition, - Optional offset + Optional offset, + Optional leaderEpoch ) { this.group = group; this.topic = topic; this.partition = partition; this.offset = offset; + this.leaderEpoch = leaderEpoch; } } } diff --git a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandOptions.java b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandOptions.java index 3ba0a707ee58e..be99d2946a727 100644 --- a/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandOptions.java +++ b/tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandOptions.java @@ -145,7 +145,7 @@ public ShareGroupCommandOptions(String[] args) { .availableIf(describeOpt); allGroupSelectionScopeOpts = new HashSet<>(Arrays.asList(groupOpt, allGroupsOpt)); - allShareGroupLevelOpts = new HashSet<>(Arrays.asList(listOpt, describeOpt, deleteOpt, deleteOffsetsOpt, resetOffsetsOpt)); + allShareGroupLevelOpts = new HashSet<>(Arrays.asList(listOpt, describeOpt, deleteOpt, resetOffsetsOpt)); allResetOffsetScenarioOpts = new HashSet<>(Arrays.asList(resetToDatetimeOpt, resetToEarliestOpt, resetToLatestOpt)); allDeleteOffsetsOpts = new HashSet<>(Arrays.asList(groupOpt, topicOpt)); @@ -208,7 +208,7 @@ public void checkArgs() { } CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, minus(allGroupSelectionScopeOpts, groupOpt)); - CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, minus(allShareGroupLevelOpts, describeOpt, deleteOpt, deleteOffsetsOpt, resetOffsetsOpt)); + CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, minus(allShareGroupLevelOpts, describeOpt, deleteOpt, resetOffsetsOpt)); CommandLineUtils.checkInvalidArgs(parser, options, topicOpt, minus(allShareGroupLevelOpts, deleteOpt, resetOffsetsOpt)); } } diff --git a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java index 46915a40b776b..72c49410e13eb 100644 --- a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java @@ -43,14 +43,14 @@ import org.apache.kafka.metadata.placement.StripedReplicaPlacer; import org.apache.kafka.metadata.placement.TopicAssignment; import org.apache.kafka.metadata.placement.UsableBroker; -import org.apache.kafka.server.common.AdminCommandFailedException; -import org.apache.kafka.server.common.AdminOperationException; import org.apache.kafka.server.config.QuotaConfig; import org.apache.kafka.server.util.CommandLineUtils; import org.apache.kafka.server.util.Json; import org.apache.kafka.server.util.json.DecodeJson; import org.apache.kafka.server.util.json.JsonObject; import org.apache.kafka.server.util.json.JsonValue; +import org.apache.kafka.tools.AdminCommandFailedException; +import org.apache.kafka.tools.AdminOperationException; import org.apache.kafka.tools.TerseException; import org.apache.kafka.tools.ToolsUtils; diff --git a/tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java b/tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java index 1a8be4104aa90..0422786e7cb75 100644 --- a/tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/streams/StreamsGroupCommand.java @@ -136,9 +136,8 @@ public void listGroups() throws ExecutionException, InterruptedException { List listStreamsGroups() { try { - ListGroupsResult result = adminClient.listGroups(new ListGroupsOptions() - .timeoutMs(opts.options.valueOf(opts.timeoutMsOpt).intValue()) - .withTypes(Set.of(GroupType.STREAMS))); + ListGroupsResult result = adminClient.listGroups(ListGroupsOptions.forStreamsGroups() + .timeoutMs(opts.options.valueOf(opts.timeoutMsOpt).intValue())); Collection listings = result.all().get(); return listings.stream().map(GroupListing::groupId).collect(Collectors.toList()); } catch (InterruptedException | ExecutionException e) { @@ -147,9 +146,8 @@ List listStreamsGroups() { } List listStreamsGroupsInStates(Set states) throws ExecutionException, InterruptedException { - ListGroupsResult result = adminClient.listGroups(new ListGroupsOptions() + ListGroupsResult result = adminClient.listGroups(ListGroupsOptions.forStreamsGroups() .timeoutMs(opts.options.valueOf(opts.timeoutMsOpt).intValue()) - .withTypes(Set.of(GroupType.STREAMS)) .inGroupStates(states)); return new ArrayList<>(result.all().get()); } diff --git a/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java index f798ed702a6d1..f5bab9443940a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java @@ -26,8 +26,6 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.server.common.AdminCommandFailedException; -import org.apache.kafka.server.common.AdminOperationException; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java index f1c68eb0d3980..c416cfb67d541 100644 --- a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java @@ -122,7 +122,7 @@ public void testDowngradeMetadataVersionWithKRaft(ClusterInstance cluster) { ); // Change expected message to reflect possible MetadataVersion range 1-N (N increases when adding a new version) assertEquals("Could not disable metadata.version. The update failed for all features since the following " + - "feature had an error: Invalid update version 0 for feature metadata.version. Local controller 3000 only supports versions 7-29", commandOutput); + "feature had an error: Invalid update version 0 for feature metadata.version. Local controller 3000 only supports versions 7-28", commandOutput); commandOutput = ToolsTestUtils.captureStandardOut(() -> assertEquals(1, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 5ee3b38564032..5bb23cabdd98a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -40,7 +40,6 @@ import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig; import org.apache.kafka.server.log.remote.storage.LocalTieredStorage; import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig; -import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.test.TestUtils; import java.time.Duration; @@ -98,7 +97,7 @@ private void setUpRemoteLogTopics() { Map rlsConfigs = new HashMap<>(); rlsConfigs.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"); rlsConfigs.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "1"); - rlsConfigs.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "100"); + rlsConfigs.put(TopicConfig.SEGMENT_BYTES_CONFIG, "100"); setupTopics(this::getRemoteLogStorageEnabledTopicName, rlsConfigs); sendProducerRecords(this::getRemoteLogStorageEnabledTopicName); } diff --git a/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java index b834c7784a8b8..a17256908fb70 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java @@ -466,7 +466,7 @@ public void testListGroupsFailsWithException() { @SuppressWarnings({"NPathComplexity", "CyclomaticComplexity"}) @ClusterTest( serverProperties = { - @ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer,share,streams"), + @ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer,streams"), @ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), @ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1") } diff --git a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandErrorTest.java b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandErrorTest.java index ed7f8895b60b3..46942be75f36e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandErrorTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandErrorTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.tools; import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.server.common.AdminCommandFailedException; import org.junit.jupiter.api.Test; diff --git a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java index 23bd9a5c19ece..78f6e9ac156ba 100644 --- a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java @@ -28,7 +28,6 @@ import org.apache.kafka.common.test.api.ClusterConfigProperty; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.server.common.AdminCommandFailedException; import org.mockito.ArgumentCaptor; import org.mockito.MockedStatic; diff --git a/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java b/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java index 10b790bf91422..9117daabe333a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java @@ -104,6 +104,81 @@ public void testReadProps() throws Exception { Utils.delete(producerConfig); } + @Test + public void testReadPayloadFileWithAlternateDelimiters() throws Exception { + List payloadByteList; + + payloadByteList = generateListFromFileUsingDelimiter("Hello~~Kafka", "~~"); + assertEquals(2, payloadByteList.size()); + assertEquals("Hello", new String(payloadByteList.get(0))); + assertEquals("Kafka", new String(payloadByteList.get(1))); + + payloadByteList = generateListFromFileUsingDelimiter("Hello,Kafka,", ","); + assertEquals(2, payloadByteList.size()); + assertEquals("Hello", new String(payloadByteList.get(0))); + assertEquals("Kafka", new String(payloadByteList.get(1))); + + payloadByteList = generateListFromFileUsingDelimiter("Hello\t\tKafka", "\t"); + assertEquals(3, payloadByteList.size()); + assertEquals("Hello", new String(payloadByteList.get(0))); + assertEquals("Kafka", new String(payloadByteList.get(2))); + + payloadByteList = generateListFromFileUsingDelimiter("Hello\n\nKafka\n", "\n"); + assertEquals(3, payloadByteList.size()); + assertEquals("Hello", new String(payloadByteList.get(0))); + assertEquals("Kafka", new String(payloadByteList.get(2))); + + payloadByteList = generateListFromFileUsingDelimiter("Hello::Kafka::World", "\\s*::\\s*"); + assertEquals(3, payloadByteList.size()); + assertEquals("Hello", new String(payloadByteList.get(0))); + assertEquals("Kafka", new String(payloadByteList.get(1))); + + } + + @Test + public void testCompareStringSplitWithScannerDelimiter() throws Exception { + + String contents = "Hello~~Kafka"; + String payloadDelimiter = "~~"; + compareList(generateListFromFileUsingDelimiter(contents, payloadDelimiter), contents.split(payloadDelimiter)); + + contents = "Hello,Kafka,"; + payloadDelimiter = ","; + compareList(generateListFromFileUsingDelimiter(contents, payloadDelimiter), contents.split(payloadDelimiter)); + + contents = "Hello\t\tKafka"; + payloadDelimiter = "\t"; + compareList(generateListFromFileUsingDelimiter(contents, payloadDelimiter), contents.split(payloadDelimiter)); + + contents = "Hello\n\nKafka\n"; + payloadDelimiter = "\n"; + compareList(generateListFromFileUsingDelimiter(contents, payloadDelimiter), contents.split(payloadDelimiter)); + + contents = "Hello::Kafka::World"; + payloadDelimiter = "\\s*::\\s*"; + compareList(generateListFromFileUsingDelimiter(contents, payloadDelimiter), contents.split(payloadDelimiter)); + + } + + private void compareList(List payloadByteList, String[] payloadByteListFromSplit) { + assertEquals(payloadByteListFromSplit.length, payloadByteList.size()); + for (int i = 0; i < payloadByteListFromSplit.length; i++) { + assertEquals(payloadByteListFromSplit[i], new String(payloadByteList.get(i))); + } + } + + private List generateListFromFileUsingDelimiter(String fileContent, String payloadDelimiter) throws Exception { + File payloadFile = null; + List payloadByteList; + try { + payloadFile = createTempFile(fileContent); + payloadByteList = ProducerPerformance.readPayloadFile(payloadFile.getAbsolutePath(), payloadDelimiter); + } finally { + Utils.delete(payloadFile); + } + return payloadByteList; + } + @Test public void testNumberOfCallsForSendAndClose() throws IOException { doReturn(null).when(producerMock).send(any(), any()); diff --git a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java index 0f2e6a918079e..deb57ae8cbc90 100644 --- a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java @@ -56,8 +56,6 @@ import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.metadata.LeaderAndIsr; -import org.apache.kafka.server.common.AdminCommandFailedException; -import org.apache.kafka.server.common.AdminOperationException; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Assertions; diff --git a/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java index 8785d4f1159bc..59d9c1823a91b 100644 --- a/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java @@ -208,11 +208,11 @@ public void testListTransactions(boolean hasDurationFilter) throws Exception { } Map> transactions = new HashMap<>(); - transactions.put(0, asList( + transactions.put(0, List.of( new TransactionListing("foo", 12345L, TransactionState.ONGOING), new TransactionListing("bar", 98765L, TransactionState.PREPARE_ABORT) )); - transactions.put(1, singletonList( + transactions.put(1, List.of( new TransactionListing("baz", 13579L, TransactionState.COMPLETE_COMMIT) )); @@ -334,6 +334,42 @@ public void testDescribeTransaction() throws Exception { assertEquals(expectedRow, table.get(1)); } + @Test + public void testListTransactionsWithTransactionalIdPattern() throws Exception { + String[] args = new String[] { + "--bootstrap-server", + "localhost:9092", + "list", + "--transactional-id-pattern", + "ba.*" + }; + + Map> transactions = new HashMap<>(); + transactions.put(0, List.of( + new TransactionListing("bar", 98765L, TransactionState.PREPARE_ABORT) + )); + transactions.put(1, List.of( + new TransactionListing("baz", 13579L, TransactionState.COMPLETE_COMMIT) + )); + + expectListTransactions(new ListTransactionsOptions().filterOnTransactionalIdPattern("ba.*"), transactions); + + execute(args); + assertNormalExit(); + + List> table = readOutputAsTable(); + assertEquals(3, table.size()); + + // Assert expected headers + List expectedHeaders = TransactionsCommand.ListTransactionsCommand.HEADERS; + assertEquals(expectedHeaders, table.get(0)); + Set> expectedRows = Set.of( + List.of("bar", "0", "98765", "PrepareAbort"), + List.of("baz", "1", "13579", "CompleteCommit") + ); + assertEquals(expectedRows, new HashSet<>(table.subList(1, table.size()))); + } + @Test public void testDescribeTransactionsStartOffsetOrProducerIdRequired() throws Exception { assertCommandFailure(new String[]{ diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java index dba06a3e098e3..fd50aa3b7d741 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java @@ -662,6 +662,10 @@ public void testConsumerGroupTypesFromString() { result = ConsumerGroupCommand.consumerGroupTypesFromString("Consumer, Classic"); Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(GroupType.CONSUMER, GroupType.CLASSIC)), result); + Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString("Share")); + + Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString("streams")); + Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString("bad, wrong")); Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString(" bad, generic")); diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java index c3690d953ab37..546cab50e0d28 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClientTestUtils; +import org.apache.kafka.clients.admin.DeleteShareGroupOffsetsResult; import org.apache.kafka.clients.admin.DeleteShareGroupsResult; import org.apache.kafka.clients.admin.DescribeShareGroupsOptions; import org.apache.kafka.clients.admin.DescribeShareGroupsResult; @@ -31,6 +32,7 @@ import org.apache.kafka.clients.admin.ShareGroupDescription; import org.apache.kafka.clients.admin.ShareMemberAssignment; import org.apache.kafka.clients.admin.ShareMemberDescription; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.GroupState; import org.apache.kafka.common.GroupType; import org.apache.kafka.common.KafkaFuture; @@ -38,6 +40,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.GroupIdNotFoundException; import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.test.TestUtils; import org.apache.kafka.tools.ToolsTestUtils; @@ -77,6 +80,7 @@ import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -188,7 +192,7 @@ public void testDescribeOffsetsOfExistingGroup() throws Exception { ListShareGroupOffsetsResult listShareGroupOffsetsResult = AdminClientTestUtils.createListShareGroupOffsetsResult( Map.of( firstGroup, - KafkaFuture.completedFuture(Map.of(new TopicPartition("topic1", 0), 0L)) + KafkaFuture.completedFuture(Map.of(new TopicPartition("topic1", 0), new OffsetAndMetadata(0L, Optional.of(1), ""))) ) ); @@ -205,7 +209,7 @@ public void testDescribeOffsetsOfExistingGroup() throws Exception { List expectedValues; if (describeType.contains("--verbose")) { - expectedValues = List.of(firstGroup, "topic1", "0", "-", "0"); + expectedValues = List.of(firstGroup, "topic1", "0", "1", "0"); } else { expectedValues = List.of(firstGroup, "topic1", "0", "0"); } @@ -296,13 +300,13 @@ public void testDescribeOffsetsOfAllExistingGroups() throws Exception { ListShareGroupOffsetsResult listShareGroupOffsetsResult1 = AdminClientTestUtils.createListShareGroupOffsetsResult( Map.of( firstGroup, - KafkaFuture.completedFuture(Map.of(new TopicPartition("topic1", 0), 0L)) + KafkaFuture.completedFuture(Map.of(new TopicPartition("topic1", 0), new OffsetAndMetadata(0, Optional.of(1), ""))) ) ); ListShareGroupOffsetsResult listShareGroupOffsetsResult2 = AdminClientTestUtils.createListShareGroupOffsetsResult( Map.of( secondGroup, - KafkaFuture.completedFuture(Map.of(new TopicPartition("topic1", 0), 0L)) + KafkaFuture.completedFuture(Map.of(new TopicPartition("topic1", 0), new OffsetAndMetadata(0, Optional.of(1), ""))) ) ); @@ -330,8 +334,8 @@ public void testDescribeOffsetsOfAllExistingGroups() throws Exception { List expectedValues1, expectedValues2; if (describeType.contains("--verbose")) { - expectedValues1 = List.of(firstGroup, "topic1", "0", "-", "0"); - expectedValues2 = List.of(secondGroup, "topic1", "0", "-", "0"); + expectedValues1 = List.of(firstGroup, "topic1", "0", "1", "0"); + expectedValues2 = List.of(secondGroup, "topic1", "0", "1", "0"); } else { expectedValues1 = List.of(firstGroup, "topic1", "0", "0"); expectedValues2 = List.of(secondGroup, "topic1", "0", "0"); @@ -612,6 +616,204 @@ public void testGroupStatesFromString() { assertThrows(IllegalArgumentException.class, () -> ShareGroupCommand.groupStatesFromString(" , ,")); } + @Test + public void testDeleteShareGroupOffsetsArgsWithoutTopic() { + String bootstrapServer = "localhost:9092"; + Admin adminClient = mock(KafkaAdminClient.class); + + // no group spec args + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", "groupId"}; + AtomicBoolean exited = new AtomicBoolean(false); + Exit.setExitProcedure(((statusCode, message) -> { + assertNotEquals(0, statusCode); + assertTrue(message.contains("Option [delete-offsets] takes the following options: [topic], [group]")); + exited.set(true); + })); + try { + getShareGroupService(cgcArgs, adminClient); + } finally { + assertTrue(exited.get()); + } + } + + @Test + public void testDeleteShareGroupOffsetsArgsWithoutGroup() { + String bootstrapServer = "localhost:9092"; + Admin adminClient = mock(KafkaAdminClient.class); + + // no group spec args + String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServer, "--delete-offsets", "--topic", "t1"}; + AtomicBoolean exited = new AtomicBoolean(false); + Exit.setExitProcedure(((statusCode, message) -> { + assertNotEquals(0, statusCode); + assertTrue(message.contains("Option [delete-offsets] takes the following options: [topic], [group]")); + exited.set(true); + })); + try { + getShareGroupService(cgcArgs, adminClient); + } finally { + assertTrue(exited.get()); + } + } + + @Test + public void testDeleteShareGroupOffsets() throws Exception { + String firstGroup = "first-group"; + String firstTopic = "t1"; + String secondTopic = "t2"; + String bootstrapServer = "localhost:9092"; + + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); + Admin adminClient = mock(KafkaAdminClient.class); + DeleteShareGroupOffsetsResult result = mock(DeleteShareGroupOffsetsResult.class); + + when(result.all()).thenReturn(KafkaFuture.completedFuture(null)); + + when(result.topicResult(eq(firstTopic))).thenReturn(KafkaFuture.completedFuture(null)); + when(result.topicResult(eq(secondTopic))).thenReturn(KafkaFuture.completedFuture(null)); + + when(adminClient.deleteShareGroupOffsets(any(), any(), any())).thenReturn(result); + + try (ShareGroupService service = getShareGroupService(cgcArgs.toArray(new String[0]), adminClient)) { + TestUtils.waitForCondition(() -> { + Entry res = ToolsTestUtils.grabConsoleOutputAndError(deleteOffsets(service)); + String[] lines = res.getKey().trim().split("\n"); + if (lines.length != 3 && !res.getValue().isEmpty()) { + return false; + } + + List expectedResultHeader = List.of("TOPIC", "STATUS"); + List expectedResultValues1 = List.of(firstTopic, "Successful"); + List expectedResultValues2 = List.of(secondTopic, "Successful"); + + return Arrays.stream(lines[0].trim().split("\\s+")).toList().equals(expectedResultHeader) && + Arrays.stream(lines[1].trim().split("\\s+")).toList().equals(expectedResultValues1) && + Arrays.stream(lines[2].trim().split("\\s+")).toList().equals(expectedResultValues2); + }, "Expected a data row and no error in delete offsets result with group: " + firstGroup + " and topic: " + firstTopic); + } + } + + @Test + public void testDeleteShareGroupOffsetsMultipleGroups() { + String firstGroup = "first-group"; + String secondGroup = "second-group"; + String firstTopic = "t1"; + String secondTopic = "t2"; + String bootstrapServer = "localhost:9092"; + + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--group", secondGroup, "--topic", firstTopic, "--topic", secondTopic)); + Admin adminClient = mock(KafkaAdminClient.class); + + try (ShareGroupService service = getShareGroupService(cgcArgs.toArray(new String[0]), adminClient)) { + service.deleteOffsets(); + fail("Expected error was not detected while trying delete offsets multiple groups"); + } catch (Exception e) { + String expectedErrorMessage = "Found multiple arguments for option group, but you asked for only one"; + assertEquals(expectedErrorMessage, e.getMessage()); + } + } + + @Test + public void testDeleteShareGroupOffsetsTopLevelError() throws Exception { + String firstGroup = "first-group"; + String firstTopic = "t1"; + String secondTopic = "t2"; + String bootstrapServer = "localhost:9092"; + + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); + Admin adminClient = mock(KafkaAdminClient.class); + DeleteShareGroupOffsetsResult result = mock(DeleteShareGroupOffsetsResult.class); + + KafkaFutureImpl resultFuture = new KafkaFutureImpl<>(); + String errorMessage = "Group g3 not found."; + GroupIdNotFoundException exception = new GroupIdNotFoundException(errorMessage); + + resultFuture.completeExceptionally(exception); + when(result.all()).thenReturn(resultFuture); + + when(result.topicResult(eq(firstTopic))).thenReturn(resultFuture); + when(result.topicResult(eq(secondTopic))).thenReturn(resultFuture); + + when(adminClient.deleteShareGroupOffsets(any(), any(), any())).thenReturn(result); + + try (ShareGroupService service = getShareGroupService(cgcArgs.toArray(new String[0]), adminClient)) { + TestUtils.waitForCondition(() -> { + Entry res = ToolsTestUtils.grabConsoleOutputAndError(deleteOffsets(service)); + String[] lines = res.getKey().trim().split("\n"); + if (lines.length != 5 && !res.getValue().isEmpty()) { + return false; + } + + List error = Stream.concat( + Stream.of("Error:"), + Arrays.stream(errorMessage.trim().split("\\s+")) + ).toList(); + + List errorLine = new ArrayList<>(error); + List expectedResultHeader = List.of("TOPIC", "STATUS"); + List expectedResultValue1 = new ArrayList<>(); + expectedResultValue1.add(firstTopic); + expectedResultValue1.addAll(error); + List expectedResultValue2 = new ArrayList<>(); + expectedResultValue2.add(secondTopic); + expectedResultValue2.addAll(error); + + return Arrays.stream(lines[0].trim().split("\\s+")).toList().equals(errorLine) && + Arrays.stream(lines[2].trim().split("\\s+")).toList().equals(expectedResultHeader) && + Arrays.stream(lines[3].trim().split("\\s+")).toList().equals(expectedResultValue1) && + Arrays.stream(lines[4].trim().split("\\s+")).toList().equals(expectedResultValue2); + }, "Expected a data row and no error in delete offsets result with group: " + firstGroup + " and topic: " + firstTopic); + } + } + + @Test + public void testDeleteShareGroupOffsetsTopicLevelError() throws Exception { + String firstGroup = "first-group"; + String firstTopic = "t1"; + String secondTopic = "t2"; + String bootstrapServer = "localhost:9092"; + + List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); + Admin adminClient = mock(KafkaAdminClient.class); + DeleteShareGroupOffsetsResult result = mock(DeleteShareGroupOffsetsResult.class); + + KafkaFutureImpl resultFuture = new KafkaFutureImpl<>(); + String errorMessage = Errors.UNKNOWN_TOPIC_OR_PARTITION.message(); + + resultFuture.completeExceptionally(Errors.UNKNOWN_TOPIC_OR_PARTITION.exception()); + when(result.all()).thenReturn(resultFuture); + + when(result.topicResult(eq(firstTopic))).thenReturn(KafkaFuture.completedFuture(null)); + when(result.topicResult(eq(secondTopic))).thenReturn(resultFuture); + + when(adminClient.deleteShareGroupOffsets(any(), any(), any())).thenReturn(result); + + try (ShareGroupService service = getShareGroupService(cgcArgs.toArray(new String[0]), adminClient)) { + TestUtils.waitForCondition(() -> { + Entry res = ToolsTestUtils.grabConsoleOutputAndError(deleteOffsets(service)); + String[] lines = res.getKey().trim().split("\n"); + if (lines.length != 5 && !res.getValue().isEmpty()) { + return false; + } + + List error = Stream.concat( + Stream.of("Error:"), + Arrays.stream(errorMessage.trim().split("\\s+")) + ).toList(); + + List expectedResultHeader = List.of("TOPIC", "STATUS"); + List expectedResultValue1 = List.of(firstTopic, "Successful"); + List expectedResultValue2 = new ArrayList<>(); + expectedResultValue2.add(secondTopic); + expectedResultValue2.addAll(error); + + return Arrays.stream(lines[0].trim().split("\\s+")).toList().equals(expectedResultHeader) && + Arrays.stream(lines[1].trim().split("\\s+")).toList().equals(expectedResultValue1) && + Arrays.stream(lines[2].trim().split("\\s+")).toList().equals(expectedResultValue2); + }, "Expected a data row and no error in delete offsets result with group: " + firstGroup + " and topic: " + firstTopic); + } + } + @Test public void testDeleteShareGroupsArgs() { String bootstrapServer = "localhost:9092"; @@ -873,6 +1075,10 @@ private Runnable describeGroups(ShareGroupCommand.ShareGroupService service) { return () -> Assertions.assertDoesNotThrow(service::describeGroups); } + private Runnable deleteOffsets(ShareGroupCommand.ShareGroupService service) { + return () -> Assertions.assertDoesNotThrow(service::deleteOffsets); + } + private boolean checkArgsHeaderOutput(List args, String output) { if (!output.contains("GROUP")) { return false; diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java index 335e8fba693ae..6dd087dc50ff0 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java @@ -29,9 +29,9 @@ import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Time; import org.apache.kafka.metadata.placement.UsableBroker; -import org.apache.kafka.server.common.AdminCommandFailedException; -import org.apache.kafka.server.common.AdminOperationException; import org.apache.kafka.server.config.QuotaConfig; +import org.apache.kafka.tools.AdminCommandFailedException; +import org.apache.kafka.tools.AdminOperationException; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; diff --git a/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionState.java b/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionState.java new file mode 100644 index 0000000000000..c03066945be42 --- /dev/null +++ b/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionState.java @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * 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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.transaction; + +import java.util.Arrays; +import java.util.EnumSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Represents the states of a transaction in the transaction coordinator. + * This enum corresponds to the Scala sealed trait TransactionState in kafka.coordinator.transaction. + */ +public enum TransactionState { + /** + * Transaction has not existed yet + *

+ * transition: received AddPartitionsToTxnRequest => Ongoing + * received AddOffsetsToTxnRequest => Ongoing + * received EndTxnRequest with abort and TransactionV2 enabled => PrepareAbort + */ + EMPTY((byte) 0, org.apache.kafka.clients.admin.TransactionState.EMPTY.toString(), true), + /** + * Transaction has started and ongoing + *

+ * transition: received EndTxnRequest with commit => PrepareCommit + * received EndTxnRequest with abort => PrepareAbort + * received AddPartitionsToTxnRequest => Ongoing + * received AddOffsetsToTxnRequest => Ongoing + */ + ONGOING((byte) 1, org.apache.kafka.clients.admin.TransactionState.ONGOING.toString(), false), + /** + * Group is preparing to commit + * transition: received acks from all partitions => CompleteCommit + */ + PREPARE_COMMIT((byte) 2, org.apache.kafka.clients.admin.TransactionState.PREPARE_COMMIT.toString(), false), + /** + * Group is preparing to abort + *

+ * transition: received acks from all partitions => CompleteAbort + *

+ * Note, In transaction v2, we allow Empty, CompleteCommit, CompleteAbort to transition to PrepareAbort. because the + * client may not know the txn state on the server side, it needs to send endTxn request when uncertain. + */ + PREPARE_ABORT((byte) 3, org.apache.kafka.clients.admin.TransactionState.PREPARE_ABORT.toString(), false), + /** + * Group has completed commit + *

+ * Will soon be removed from the ongoing transaction cache + */ + COMPLETE_COMMIT((byte) 4, org.apache.kafka.clients.admin.TransactionState.COMPLETE_COMMIT.toString(), true), + /** + * Group has completed abort + *

+ * Will soon be removed from the ongoing transaction cache + */ + COMPLETE_ABORT((byte) 5, org.apache.kafka.clients.admin.TransactionState.COMPLETE_ABORT.toString(), true), + /** + * TransactionalId has expired and is about to be removed from the transaction cache + */ + DEAD((byte) 6, "Dead", false), + /** + * We are in the middle of bumping the epoch and fencing out older producers. + */ + PREPARE_EPOCH_FENCE((byte) 7, org.apache.kafka.clients.admin.TransactionState.PREPARE_EPOCH_FENCE.toString(), false); + + private static final Map NAME_TO_ENUM = Arrays.stream(values()) + .collect(Collectors.toUnmodifiableMap(TransactionState::stateName, Function.identity())); + + private static final Map ID_TO_ENUM = Arrays.stream(values()) + .collect(Collectors.toUnmodifiableMap(TransactionState::id, Function.identity())); + + public static final Set ALL_STATES = Set.copyOf(EnumSet.allOf(TransactionState.class)); + + private final byte id; + private final String stateName; + public static final Map> VALID_PREVIOUS_STATES = Map.of( + EMPTY, Set.of(EMPTY, COMPLETE_COMMIT, COMPLETE_ABORT), + ONGOING, Set.of(ONGOING, EMPTY, COMPLETE_COMMIT, COMPLETE_ABORT), + PREPARE_COMMIT, Set.of(ONGOING), + PREPARE_ABORT, Set.of(ONGOING, PREPARE_EPOCH_FENCE, EMPTY, COMPLETE_COMMIT, COMPLETE_ABORT), + COMPLETE_COMMIT, Set.of(PREPARE_COMMIT), + COMPLETE_ABORT, Set.of(PREPARE_ABORT), + DEAD, Set.of(EMPTY, COMPLETE_ABORT, COMPLETE_COMMIT), + PREPARE_EPOCH_FENCE, Set.of(ONGOING) + ); + + private final boolean expirationAllowed; + + TransactionState(byte id, String name, boolean expirationAllowed) { + this.id = id; + this.stateName = name; + this.expirationAllowed = expirationAllowed; + } + + /** + * @return The state id byte. + */ + public byte id() { + return id; + } + + /** + * Get the name of this state. This is exposed through the `DescribeTransactions` API. + * @return The state name string. + */ + public String stateName() { + return stateName; + } + + /** + * @return The set of states from which it is valid to transition into this state. + */ + public Set validPreviousStates() { + return VALID_PREVIOUS_STATES.getOrDefault(this, Set.of()); + } + + /** + * @return True if expiration is allowed in this state, false otherwise. + */ + public boolean isExpirationAllowed() { + return expirationAllowed; + } + + /** + * Finds a TransactionState by its name. + * @param name The name of the state. + * @return An Optional containing the TransactionState if found, otherwise empty. + */ + public static Optional fromName(String name) { + return Optional.ofNullable(NAME_TO_ENUM.get(name)); + } + + /** + * Finds a TransactionState by its ID. + * @param id The byte ID of the state. + * @return The TransactionState corresponding to the ID. + * @throws IllegalStateException if the ID is unknown. + */ + public static TransactionState fromId(byte id) { + TransactionState state = ID_TO_ENUM.get(id); + if (state == null) { + throw new IllegalStateException("Unknown transaction state id " + id + " from the transaction status message"); + } + return state; + } +} From e715e525e7da95809236b79142a5f5f0e2290032 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 12 May 2025 20:28:27 +0800 Subject: [PATCH 57/74] completed --- .../clients/consumer/ShareConsumerTest.java | 2 + .../apache/kafka/connect/data/ValuesTest.java | 2 + .../scala/kafka/raft/KafkaMetadataLog.scala | 20 ++---- .../api/PlaintextAdminIntegrationTest.scala | 2 +- .../api/SaslSslAdminIntegrationTest.scala | 4 +- .../DynamicBrokerReconfigurationTest.scala | 2 +- .../kafka/raft/KafkaMetadataLogTest.scala | 13 ++-- .../kafka/cluster/AbstractPartitionTest.scala | 2 +- .../kafka/cluster/PartitionLockTest.scala | 2 +- .../AbstractLogCleanerIntegrationTest.scala | 2 +- .../kafka/log/LogCleanerManagerTest.scala | 18 ++--- .../scala/unit/kafka/log/LogCleanerTest.scala | 72 +++++++++---------- .../unit/kafka/log/LogConcurrencyTest.scala | 3 +- .../scala/unit/kafka/log/LogConfigTest.scala | 1 + .../scala/unit/kafka/log/LogLoaderTest.scala | 3 +- .../scala/unit/kafka/log/LogManagerTest.scala | 4 +- .../scala/unit/kafka/log/LogTestUtils.scala | 2 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 2 +- .../AlterReplicaLogDirsRequestTest.scala | 4 +- .../server/DynamicBrokerConfigTest.scala | 12 +--- .../server/DynamicConfigChangeTest.scala | 8 +-- .../unit/kafka/server/KafkaConfigTest.scala | 2 + .../kafka/tools/DumpLogSegmentsTest.scala | 1 - .../kafka/metadata/KafkaConfigSchema.java | 3 + .../apache/kafka/raft/MetadataLogConfig.java | 22 +++--- .../kafka/server/config/ServerLogConfigs.java | 3 + .../kafka/storage/internals/log/Cleaner.java | 2 +- .../storage/internals/log/LogConfig.java | 27 +++++-- .../storage/internals/log/UnifiedLog.java | 8 +-- .../storage/internals/log/LocalLogTest.java | 3 +- .../PurgeRepartitionTopicIntegrationTest.java | 4 +- .../kafka/streams/StreamsConfigTest.java | 5 +- .../kafka/tools/GetOffsetShellTest.java | 3 +- 33 files changed, 128 insertions(+), 135 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java index 7bb348143d146..73d239070cb48 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java @@ -63,6 +63,7 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Timeout; @@ -107,6 +108,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; +@Disabled @SuppressWarnings("ClassFanOutComplexity") @Timeout(1200) @Tag("integration") diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java index ac6eef6fa6800..807d84f7a95c2 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.connect.data.Values.Parser; import org.apache.kafka.connect.errors.DataException; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -885,6 +886,7 @@ public void shouldConvertTimeValues() { assertEquals(currentMillis, t5.getTime()); } + @Disabled @Test public void shouldConvertDateValues() { LocalDateTime localTime = LocalDateTime.now(); diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index f7b776f1336c2..a94bcbd78c8ef 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.record.{MemoryRecords, Records} import org.apache.kafka.common.utils.LogContext import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} -import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} +import org.apache.kafka.raft.{Isolation, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, MetadataLogConfig, OffsetMetadata, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} import org.apache.kafka.server.common.OffsetAndEpoch import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.storage.log.FetchIsolation @@ -589,8 +589,10 @@ object KafkaMetadataLog extends Logging { ): 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) + if (config.internalLogSegmentBytes != null) + props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) + else + props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT.toString) // Disable time and byte retention when deleting segments @@ -599,11 +601,7 @@ object KafkaMetadataLog extends Logging { LogConfig.validate(props) val defaultLogConfig = new LogConfig(props) - if (config.logSegmentBytes < config.logSegmentMinBytes) { - throw new InvalidConfigurationException( - s"Cannot set ${MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG} below ${config.logSegmentMinBytes}: ${config.logSegmentBytes}" - ) - } else if (defaultLogConfig.retentionMs >= 0) { + if (defaultLogConfig.retentionMs >= 0) { throw new InvalidConfigurationException( s"Cannot set ${TopicConfig.RETENTION_MS_CONFIG} above -1: ${defaultLogConfig.retentionMs}." ) @@ -639,12 +637,6 @@ object KafkaMetadataLog extends Logging { nodeId ) - // Print a warning if users have overridden the internal config - if (config.logSegmentMinBytes != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) { - metadataLog.error(s"Overriding ${MetadataLogConfig.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.") - } - // 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() diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 1fbda492d4f73..37b9b95a4f304 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -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 diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 66fca0db9badc..4ef32c2f2ebea 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -567,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(TopicConfig.SEGMENT_BYTES_CONFIG -> "3000000").asJava 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)) @@ -581,7 +581,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu val topicConfigs = result.config(topic1).get().entries.asScala assertTrue(topicConfigs.nonEmpty) val segmentBytesConfig = topicConfigs.find(_.name == TopicConfig.SEGMENT_BYTES_CONFIG).get - assertEquals(100000, segmentBytesConfig.value.toLong) + assertEquals(3000000, segmentBytesConfig.value.toLong) assertEquals(ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, segmentBytesConfig.source) val compressionConfig = topicConfigs.find(_.name == TopicConfig.COMPRESSION_TYPE_CONFIG).get assertEquals(ServerLogConfigs.COMPRESSION_TYPE_DEFAULT, compressionConfig.value) diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 3c5cd9396bdba..9d1f699f238ac 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -653,7 +653,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) => diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 3b7aa0decef46..6c7edb2e7cfc6 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -20,7 +20,7 @@ import kafka.server.{KafkaConfig, KafkaRaftServer} import kafka.utils.TestUtils import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.errors.CorruptRecordException -import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException} +import org.apache.kafka.common.errors.RecordTooLargeException import org.apache.kafka.common.protocol import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable} import org.apache.kafka.common.record.ArbitraryMemoryRecords @@ -43,6 +43,7 @@ import org.junit.jupiter.params.provider.ArgumentsSource import net.jqwik.api.AfterFailureMode import net.jqwik.api.ForAll import net.jqwik.api.Property +import org.apache.kafka.common.config.ConfigException import org.apache.kafka.server.common.OffsetAndEpoch import java.io.File @@ -78,13 +79,13 @@ final class KafkaMetadataLogTest { props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240)) props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024)) - assertThrows(classOf[InvalidConfigurationException], () => { + assertThrows(classOf[ConfigException], () => { val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) }) - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) + props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10 * 1024 * 1024)) val kafkaConfig = KafkaConfig.fromProps(props) val metadataConfig = new MetadataLogConfig(kafkaConfig) buildMetadataLog(tempDir, mockTime, metadataConfig) @@ -689,7 +690,6 @@ final class KafkaMetadataLogTest { val recordSize = 64 val config = new MetadataLogConfig( DefaultMetadataLogConfig.logSegmentBytes, - DefaultMetadataLogConfig.logSegmentMinBytes, DefaultMetadataLogConfig.logSegmentMillis, DefaultMetadataLogConfig.retentionMaxBytes, DefaultMetadataLogConfig.retentionMillis, @@ -908,7 +908,6 @@ final class KafkaMetadataLogTest { @Test def testAdvanceLogStartOffsetAfterCleaning(): Unit = { val config = new MetadataLogConfig( - 512, 512, 10 * 1000, 256, @@ -945,7 +944,6 @@ final class KafkaMetadataLogTest { def testDeleteSnapshots(): Unit = { // Generate some logs and a few snapshots, set retention low and verify that cleaning occurs val config = new MetadataLogConfig( - 1024, 1024, 10 * 1000, 1024, @@ -979,7 +977,6 @@ final class KafkaMetadataLogTest { def testSoftRetentionLimit(): Unit = { // Set retention equal to the segment size and generate slightly more than one segment of logs val config = new MetadataLogConfig( - 10240, 10240, 10 * 1000, 10240, @@ -1023,7 +1020,6 @@ final class KafkaMetadataLogTest { @Test def testSegmentsLessThanLatestSnapshot(): Unit = { val config = new MetadataLogConfig( - 10240, 10240, 10 * 1000, 10240, @@ -1082,7 +1078,6 @@ object KafkaMetadataLogTest { } val DefaultMetadataLogConfig = new MetadataLogConfig( - 100 * 1024, 100 * 1024, 10 * 1000, 100 * 1024, diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index d9eaa7b2aacc0..fff1930a71836 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -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) } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 4a3051ddc9567..77b098cf68298 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -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) } diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index acf21e69ec377..b1e161b975340 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -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) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index f93d703f07777..8445baa7719fb 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -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) @@ -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) @@ -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)) @@ -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)) @@ -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)) @@ -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)) @@ -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)) @@ -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) @@ -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 diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 09a9d1c40f140..1bebfaa49e173 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -56,7 +56,7 @@ class LogCleanerTest extends Logging { val tmpdir = TestUtils.tempDir() val dir = TestUtils.randomPartitionLogDir(tmpdir) 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 = new LogConfig(logProps) @@ -148,7 +148,7 @@ class LogCleanerTest extends Logging { def testCleanSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) 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)) @@ -181,7 +181,7 @@ class LogCleanerTest extends Logging { // Construct a log instance. The replaceSegments() method of the log instance is overridden so that // it waits for another thread to execute deleteOldSegments() 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.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE) val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = UnifiedLog.parseTopicPartitionName(dir) @@ -271,7 +271,7 @@ class LogCleanerTest extends Logging { val originalMaxFileSize = 1024 val cleaner = makeCleaner(2) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, originalMaxFileSize: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact": java.lang.String) logProps.put(TopicConfig.PREALLOCATE_CONFIG, "true": java.lang.String) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -295,7 +295,7 @@ class LogCleanerTest extends Logging { def testDuplicateCheckAfterCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) var log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -464,7 +464,7 @@ class LogCleanerTest extends Logging { def testBasicTransactionAwareCleaning(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -497,7 +497,7 @@ class LogCleanerTest extends Logging { def testCleanWithTransactionsSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) 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)) val producerEpoch = 0.toShort @@ -543,7 +543,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -591,7 +591,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(capacity = Int.MaxValue, maxMessageSize = 100) val logProps = new Properties() logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, 100: java.lang.Integer) - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1000: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -619,7 +619,7 @@ class LogCleanerTest extends Logging { def testCommitMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -682,7 +682,7 @@ class LogCleanerTest extends Logging { def testCleanEmptyControlBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -714,7 +714,7 @@ class LogCleanerTest extends Logging { def testCommittedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -736,7 +736,7 @@ class LogCleanerTest extends Logging { def testAbortedTransactionSpanningSegments(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 128: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort val producerId = 1L @@ -765,7 +765,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRemoval(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -799,7 +799,7 @@ class LogCleanerTest extends Logging { val producerId = 1L val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val appendFirstTransaction = appendTransactionalAsLeader(log, producerId, producerEpoch, 0, AppendOrigin.REPLICATION) @@ -832,7 +832,7 @@ class LogCleanerTest extends Logging { def testAbortMarkerRetentionWithEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 256: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val producerEpoch = 0.toShort @@ -896,7 +896,7 @@ class LogCleanerTest extends Logging { // Create cleaner with very small default max message size val cleaner = makeCleaner(Int.MaxValue, maxMessageSize=1024) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -968,7 +968,7 @@ class LogCleanerTest extends Logging { def createLogWithMessagesLargerThanMaxSize(largeMessageSize: Int): (UnifiedLog, FakeOffsetMap) = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, largeMessageSize * 16: java.lang.Integer) logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, largeMessageSize * 2: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -994,7 +994,7 @@ class LogCleanerTest extends Logging { def testCleaningWithDeletes(): Unit = { val cleaner = makeCleaner(Int.MaxValue) 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)) @@ -1021,7 +1021,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 3 messages in the map val cleaner = makeCleaner(4) 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)) @@ -1050,7 +1050,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsProducerLastSequence(): Unit = { val cleaner = makeCleaner(10) 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)) log.appendAsLeader(record(0, 0), 0) // offset 0 @@ -1073,7 +1073,7 @@ class LogCleanerTest extends Logging { def testLogCleanerRetainsLastSequenceEvenIfTransactionAborted(): Unit = { val cleaner = makeCleaner(10) 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)) val producerEpoch = 0.toShort @@ -1107,7 +1107,7 @@ class LogCleanerTest extends Logging { def testCleaningWithKeysConflictingWithTxnMarkerKeys(): Unit = { val cleaner = makeCleaner(10) 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)) val leaderEpoch = 5 val producerEpoch = 0.toShort @@ -1151,7 +1151,7 @@ class LogCleanerTest extends Logging { // because loadFactor is 0.75, this means we can fit 1 message in the map val cleaner = makeCleaner(2) 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)) @@ -1182,7 +1182,7 @@ class LogCleanerTest extends Logging { def testCleaningWithUncleanableSection(): Unit = { val cleaner = makeCleaner(Int.MaxValue) 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)) @@ -1225,7 +1225,7 @@ class LogCleanerTest extends Logging { def testLogToClean(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1243,7 +1243,7 @@ class LogCleanerTest extends Logging { def testLogToCleanWithUncleanableSection(): Unit = { // create a log with small segment size val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 100: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) // create 6 segments with only one message in each segment @@ -1276,7 +1276,7 @@ class LogCleanerTest extends Logging { // create a log with compaction turned off so we can append unkeyed messages 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.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1334,7 +1334,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsWithAbort(): Unit = { val cleaner = makeCleaner(Int.MaxValue, abortCheckDone) 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)) @@ -1356,7 +1356,7 @@ class LogCleanerTest extends Logging { def testCleanSegmentsRetainingLastEmptyBatch(): Unit = { val cleaner = makeCleaner(Int.MaxValue) 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)) @@ -1388,7 +1388,7 @@ class LogCleanerTest extends Logging { def testSegmentGrouping(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1489,7 +1489,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) @@ -1541,7 +1541,7 @@ class LogCleanerTest extends Logging { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 400: java.lang.Integer) //mimic the effect of loading an empty index file logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 400: java.lang.Integer) @@ -1666,7 +1666,7 @@ class LogCleanerTest extends Logging { def testRecoveryAfterCrash(): Unit = { val cleaner = makeCleaner(Int.MaxValue) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 300: java.lang.Integer) logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, 1: java.lang.Integer) logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, 10: java.lang.Integer) @@ -1797,7 +1797,7 @@ class LogCleanerTest extends Logging { def testBuildOffsetMapFakeLarge(): Unit = { val map = new FakeOffsetMap(1000) val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 120: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) @@ -1945,7 +1945,7 @@ class LogCleanerTest extends Logging { @Test def testCleaningBeyondMissingOffsets(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024*1024: java.lang.Integer) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = new LogConfig(logProps) val cleaner = makeCleaner(Int.MaxValue) diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 0da8366f443ee..854be39808661 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -20,7 +20,6 @@ package kafka.log import java.util.{Optional, Properties} import java.util.concurrent.{Callable, Executors} import kafka.utils.TestUtils -import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfig @@ -60,7 +59,7 @@ class LogConcurrencyTest { @Test def testUncommittedDataNotConsumedFrequentSegmentRolls(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, 237: Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 237: Integer) val logConfig = new LogConfig(logProps) testUncommittedDataNotConsumed(createLog(logConfig)) } diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 1e26d653bbccf..752d41e54a9c1 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -94,6 +94,7 @@ class LogConfigTest { case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1") case TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") case TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG => assertPropertyInvalid(name, "not_a_number", "remove", "0") + case LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG => // no op case _ => assertPropertyInvalid(name, "not_a_number", "-1") }) diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 8e417a695ee36..0c465cf2138df 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -21,7 +21,6 @@ import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.compress.Compression -import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, SimpleRecord, TimestampType} import org.apache.kafka.common.utils.{Time, Utils} @@ -245,7 +244,7 @@ class LogLoaderTest { @Test def testProducerSnapshotsRecoveryAfterUncleanShutdown(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "640") + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "640") val logConfig = new LogConfig(logProps) var log = createLog(logDir, logConfig) assertEquals(OptionalLong.empty(), log.oldestProducerSnapshotOffset) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 67880e0ced54c..04f4acca5debd 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -60,7 +60,7 @@ class LogManagerTest { val maxRollInterval = 100 val maxLogAgeMs: Int = 10 * 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.SEGMENT_INDEX_BYTES_CONFIG, 4096: java.lang.Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, maxLogAgeMs: java.lang.Integer) val logConfig = new LogConfig(logProps) @@ -391,7 +391,7 @@ class LogManagerTest { logManager.shutdown() val segmentBytes = 10 * setSize val properties = new Properties() - properties.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes.toString) + properties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes.toString) properties.put(TopicConfig.RETENTION_BYTES_CONFIG, (5L * 10L * setSize + 10L).toString) val configRepository = MockConfigRepository.forTopic(name, properties) diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index 04c91741037bb..0ff68988d76fb 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -75,7 +75,7 @@ object LogTestUtils { remoteLogDeleteOnDisable: Boolean = DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG): LogConfig = { val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_MS_CONFIG, segmentMs: java.lang.Long) - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, segmentBytes: Integer) + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, segmentBytes: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: java.lang.Long) logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localRetentionMs: java.lang.Long) logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, retentionBytes: java.lang.Long) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 3f73f8f731a71..42c813074aa9a 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -2734,7 +2734,7 @@ class UnifiedLogTest { @Test def testLeaderEpochCacheCreatedAfterMessageFormatUpgrade(): Unit = { val logProps = new Properties() - logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1000") + logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000") logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, "1") logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, "65536") val logConfig = new LogConfig(logProps) diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index f5bd92ce15ec7..07c135132f4a5 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.message.AlterReplicaLogDirsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AlterReplicaLogDirsRequest, AlterReplicaLogDirsResponse} import org.apache.kafka.server.config.ServerLogConfigs -import org.apache.kafka.storage.internals.log.LogFileUtils +import org.apache.kafka.storage.internals.log.{LogConfig, LogFileUtils} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -144,7 +144,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { // We don't want files with `.deleted` suffix are removed too fast, // so we can validate there will be orphan files and orphan files will be removed eventually. topicProperties.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, "10000") - topicProperties.put(TopicConfig.SEGMENT_BYTES_CONFIG, "1024") + topicProperties.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1024") createTopic(topic, partitionNum, 1, topicProperties) assertEquals(logDir1, brokers.head.logManager.getLog(tp).get.dir.getParent) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 5f87b20d1f579..c20cd042fc5fb 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.internals.Plugin import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig} +import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.server.DynamicThreadPool import org.apache.kafka.server.authorizer._ @@ -670,16 +670,6 @@ class DynamicBrokerConfigTest { assertTrue(m.currentReporters.isEmpty) } - @Test - def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = { - val props = TestUtils.createBrokerConfig(0, port = 8181) - props.put(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, "1024") - val config = new KafkaConfig(props) - assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) - config.updateCurrentConfig(new KafkaConfig(props)) - assertFalse(config.nonInternalValues.containsKey(MetadataLogConfig.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)) - } - @Test def testDynamicLogLocalRetentionMsConfig(): Unit = { val props = TestUtils.createBrokerConfig(0, port = 8181) diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 61105b176dce7..a74e25e6c17b9 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -95,21 +95,21 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { @Test def testDynamicTopicConfigChange(): Unit = { val tp = new TopicPartition("test", 0) - val oldSegmentSize = 1000 + val oldSegmentSize = 2 * 1024 * 1024 val logProps = new Properties() logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldSegmentSize.toString) createTopic(tp.topic, 1, 1, logProps) TestUtils.retry(10000) { val logOpt = this.brokers.head.logManager.getLog(tp) assertTrue(logOpt.isDefined) - assertEquals(oldSegmentSize, logOpt.get.config.segmentSize) + assertEquals(oldSegmentSize, logOpt.get.config.segmentSize()) } val newSegmentSize = 2000 val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(TopicConfig.SEGMENT_BYTES_CONFIG, newSegmentSize.toString), + val op = new AlterConfigOp(new ConfigEntry(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), OpType.SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get } finally { @@ -117,7 +117,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } val log = brokers.head.logManager.getLog(tp).get TestUtils.retry(10000) { - assertEquals(newSegmentSize, log.config.segmentSize) + assertEquals(newSegmentSize, log.config.segmentSize()) } (1 to 50).foreach(i => TestUtils.produceMessage(brokers, tp.topic, i.toString)) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 7f78f2ba59047..50c988b4649c7 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -822,6 +822,8 @@ class KafkaConfigTest { case ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG => // no op + case ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG => assertPropertyInvalid(baseProperties, name, "unknown_policy", "0") diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 4320e46901661..fd199504030f9 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -545,7 +545,6 @@ class DumpLogSegmentsTest { time, time.scheduler, new MetadataLogConfig( - 100 * 1024, 100 * 1024, 10 * 1000, 100 * 1024, diff --git a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java index f61c4c70e0680..13c1743ee5560 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java @@ -166,6 +166,9 @@ public Map resolveEffectiveTopicConfigs( ConfigDef configDef = configDefs.getOrDefault(ConfigResource.Type.TOPIC, EMPTY_CONFIG_DEF); HashMap effectiveConfigs = new HashMap<>(); for (ConfigDef.ConfigKey configKey : configDef.configKeys().values()) { + if (configKey.internalConfig && !dynamicTopicConfigs.containsKey(configKey.name)) { + continue; + } ConfigEntry entry = resolveEffectiveTopicConfig(configKey, staticNodeConfig, dynamicClusterConfigs, dynamicNodeConfigs, dynamicTopicConfigs); effectiveConfigs.put(entry.name(), entry); diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index b0a6f9f045ad6..805d16d361c7b 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -52,10 +52,6 @@ public class MetadataLogConfig { "configuration. The Kafka node will generate a snapshot when either the maximum time interval is reached or the " + "maximum bytes limit is reached."; - public static final String METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG = "metadata.log.segment.min.bytes"; - public static final String METADATA_LOG_SEGMENT_MIN_BYTES_DOC = "Override the minimum size for a single metadata log file. This should be used for testing only."; - public static final int METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT = 8 * 1024 * 1024; - public static final String METADATA_LOG_SEGMENT_BYTES_CONFIG = "metadata.log.segment.bytes"; public static final String METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file."; public static final int METADATA_LOG_SEGMENT_BYTES_DEFAULT = 1024 * 1024 * 1024; @@ -85,14 +81,13 @@ public class MetadataLogConfig { .define(METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC) .define(METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, METADATA_LOG_DIR_DOC) .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) - .defineInternal(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_MIN_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_MIN_BYTES_DOC) .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); private final int logSegmentBytes; - private final int logSegmentMinBytes; + private final Integer internalLogSegmentBytes; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -103,7 +98,6 @@ public class MetadataLogConfig { /** * Configuration for the metadata log * @param logSegmentBytes The maximum size of a single metadata log file - * @param logSegmentMinBytes The minimum size of a single metadata log file * @param logSegmentMillis The maximum time before a new metadata log file is rolled out * @param retentionMaxBytes The size of the metadata log and snapshots before deleting old snapshots and log files * @param retentionMillis The time to keep a metadata log file or snapshot before deleting it @@ -112,7 +106,6 @@ public class MetadataLogConfig { * @param deleteDelayMillis The amount of time to wait before deleting a file from the filesystem */ public MetadataLogConfig(int logSegmentBytes, - int logSegmentMinBytes, long logSegmentMillis, long retentionMaxBytes, long retentionMillis, @@ -120,7 +113,7 @@ public MetadataLogConfig(int logSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = logSegmentBytes; - this.logSegmentMinBytes = logSegmentMinBytes; + this.internalLogSegmentBytes = logSegmentBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; @@ -131,7 +124,7 @@ public MetadataLogConfig(int logSegmentBytes, public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); - this.logSegmentMinBytes = config.getInt(METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG); + this.internalLogSegmentBytes = config.getInt(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); @@ -141,11 +134,12 @@ public MetadataLogConfig(AbstractConfig config) { } public int logSegmentBytes() { - return logSegmentBytes; + if (internalLogSegmentBytes == null) return logSegmentBytes; + return internalLogSegmentBytes; } - - public int logSegmentMinBytes() { - return logSegmentMinBytes; + + public Integer internalLogSegmentBytes() { + return internalLogSegmentBytes; } public long logSegmentMillis() { diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 2c2d44ef012e5..16b5c7a99a39c 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -43,6 +43,9 @@ public class ServerLogConfigs { public static final String LOG_SEGMENT_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); public static final String LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file"; + public static final String INTERNAL_LOG_SEGMENT_BYTES_CONFIG = "internal." + ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); + public static final String INTERNAL_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; + public static final String LOG_ROLL_TIME_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_MS_CONFIG); public static final String LOG_ROLL_TIME_HOURS_CONFIG = LOG_PREFIX + "roll.hours"; public static final String LOG_ROLL_TIME_MILLIS_DOC = "The maximum time before a new log segment is rolled out (in milliseconds). If not set, the value in " + LOG_ROLL_TIME_HOURS_CONFIG + " is used"; diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java index b7d4ccedb2ec1..54ad6a4e79a97 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java @@ -171,7 +171,7 @@ public Map.Entry doClean(LogToClean cleanable, long currentT List> groupedSegments = groupSegmentsBySize( log.logSegments(0, endOffset), - log.config().segmentSize, + log.config().segmentSize(), log.config().maxIndexSize, cleanable.firstUncleanableOffset() ); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 21c92cd84dff4..5a1509ac7b1c2 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.record.LegacyRecord; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ConfigUtils; import org.apache.kafka.common.utils.Utils; @@ -140,11 +139,16 @@ public Optional serverConfigName(String configName) { public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs + public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; + public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; + + // Visible for testing public static final Set CONFIGS_WITH_NO_SERVER_DEFAULTS = Set.of( TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, + INTERNAL_SEGMENT_BYTES_CONFIG, QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG ); @@ -186,12 +190,13 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) - .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC); + .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) + .defineInternal(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { CONFIG. - define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), MEDIUM, + define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(1024 * 1024), MEDIUM, TopicConfig.SEGMENT_BYTES_DOC) .define(TopicConfig.SEGMENT_MS_CONFIG, LONG, DEFAULT_SEGMENT_MS, atLeast(1), MEDIUM, TopicConfig.SEGMENT_MS_DOC) .define(TopicConfig.SEGMENT_JITTER_MS_CONFIG, LONG, DEFAULT_SEGMENT_JITTER_MS, atLeast(0), MEDIUM, @@ -253,7 +258,8 @@ public Optional serverConfigName(String configName) { .define(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM, TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) .define(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_COPY_DISABLE_DOC) - .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC); + .define(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, BOOLEAN, false, MEDIUM, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_DOC) + .defineInternal(INTERNAL_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, INTERNAL_SEGMENT_BYTES_DOC); } public final Set overriddenConfigs; @@ -262,7 +268,8 @@ public Optional serverConfigName(String configName) { * Important note: Any configuration parameter that is passed along from KafkaConfig to LogConfig * should also be in `KafkaConfig#extractLogConfigMap`. */ - public final int segmentSize; + private final int segmentSize; + private final Integer internalSegmentSize; public final long segmentMs; public final long segmentJitterMs; public final int maxIndexSize; @@ -306,6 +313,7 @@ public LogConfig(Map props, Set overriddenConfigs) { this.overriddenConfigs = Collections.unmodifiableSet(overriddenConfigs); this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG); + this.internalSegmentSize = getInt(INTERNAL_SEGMENT_BYTES_CONFIG); this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG); this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG); this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); @@ -367,6 +375,11 @@ private Optional getCompression() { } } + public int segmentSize() { + if (internalSegmentSize == null) return segmentSize; + return internalSegmentSize; + } + // Exposed as a method so it can be mocked public int maxMessageSize() { return maxMessageSize; @@ -388,7 +401,7 @@ public long maxSegmentMs() { public int initFileSize() { if (preallocate) - return segmentSize; + return segmentSize(); else return 0; } @@ -628,7 +641,7 @@ public static void validate(Map existingConfigs, @Override public String toString() { return "LogConfig{" + - "segmentSize=" + segmentSize + + "segmentSize=" + segmentSize() + ", segmentMs=" + segmentMs + ", segmentJitterMs=" + segmentJitterMs + ", maxIndexSize=" + maxIndexSize + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index 972aeb095815a..3b10e657745ac 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1177,9 +1177,9 @@ private LogAppendInfo append(MemoryRecords records, }); // check messages size does not exceed config.segmentSize - if (validRecords.sizeInBytes() > config().segmentSize) { + if (validRecords.sizeInBytes() > config().segmentSize()) { throw new RecordBatchTooLargeException("Message batch size is " + validRecords.sizeInBytes() + " bytes in append " + - "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize + "."); + "to partition " + topicPartition() + ", which exceeds the maximum configured segment size of " + config().segmentSize() + "."); } // maybe roll the log if this segment is full @@ -2034,12 +2034,12 @@ private LogSegment maybeRoll(int messagesSize, LogAppendInfo appendInfo) throws long maxTimestampInMessages = appendInfo.maxTimestamp(); long maxOffsetInMessages = appendInfo.lastOffset(); - if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize, appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { + if (segment.shouldRoll(new RollParams(config().maxSegmentMs(), config().segmentSize(), appendInfo.maxTimestamp(), appendInfo.lastOffset(), messagesSize, now))) { logger.debug("Rolling new log segment (log_size = {}/{}}, " + "offset_index_size = {}/{}, " + "time_index_size = {}/{}, " + "inactive_time_ms = {}/{}).", - segment.size(), config().segmentSize, + segment.size(), config().segmentSize(), segment.offsetIndex().entries(), segment.offsetIndex().maxEntries(), segment.timeIndex().entries(), segment.timeIndex().maxEntries(), segment.timeWaitedForRoll(now, maxTimestampInMessages), config().segmentMs - segment.rollJitterMs()); diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index 7dfba8f7a5954..d58c520af0972 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; -import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -198,7 +197,7 @@ public void testUpdateConfig() { assertEquals(oldConfig, log.config()); Properties props = new Properties(); - props.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldConfig.segmentSize + 1); + props.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); LogConfig newConfig = new LogConfig(props); log.updateConfig(newConfig); assertEquals(newConfig, log.config()); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index 6023706847f15..be85c9fd0af86 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -114,7 +114,7 @@ public final boolean conditionMet() { .get(); return config.get(TopicConfig.CLEANUP_POLICY_CONFIG).value().equals(TopicConfig.CLEANUP_POLICY_DELETE) && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(PURGE_INTERVAL_MS.toString()) - && config.get(TopicConfig.SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); + && config.get("internal.segment.bytes").value().equals(PURGE_SEGMENT_BYTES.toString()); } catch (final Exception e) { return false; } @@ -171,7 +171,7 @@ public void setup() { streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); - streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); + streamsConfiguration.put(StreamsConfig.topicPrefix("internal.segment.bytes"), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size final StreamsBuilder builder = new StreamsBuilder(); diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index f081a768815fa..2db5cc682d9f2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; @@ -248,7 +247,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { props.put(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, 99_999L); props.put(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, 7L); props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "dummy:host"); - props.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), 100); + props.put(StreamsConfig.topicPrefix("internal.segment.bytes"), 100); final StreamsConfig streamsConfig = new StreamsConfig(props); final Map returnedProps = streamsConfig.getMainConsumerConfigs(groupId, clientId, threadIdx); @@ -263,7 +262,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { ); assertEquals(7L, returnedProps.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG)); assertEquals("dummy:host", returnedProps.get(StreamsConfig.APPLICATION_SERVER_CONFIG)); - assertEquals(100, returnedProps.get(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG))); + assertEquals(100, returnedProps.get(StreamsConfig.topicPrefix("internal.segment.bytes"))); } @Test diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 5bb23cabdd98a..5ee3b38564032 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -40,6 +40,7 @@ import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig; import org.apache.kafka.server.log.remote.storage.LocalTieredStorage; import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig; +import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.test.TestUtils; import java.time.Duration; @@ -97,7 +98,7 @@ private void setUpRemoteLogTopics() { Map rlsConfigs = new HashMap<>(); rlsConfigs.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true"); rlsConfigs.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, "1"); - rlsConfigs.put(TopicConfig.SEGMENT_BYTES_CONFIG, "100"); + rlsConfigs.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "100"); setupTopics(this::getRemoteLogStorageEnabledTopicName, rlsConfigs); sendProducerRecords(this::getRemoteLogStorageEnabledTopicName); } From 898a42af9d32cc13518cec46d3ae072a479fe86c Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 12 May 2025 20:30:17 +0800 Subject: [PATCH 58/74] update some test --- .../org/apache/kafka/clients/consumer/ShareConsumerTest.java | 2 -- .../src/test/java/org/apache/kafka/connect/data/ValuesTest.java | 2 -- 2 files changed, 4 deletions(-) diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java index 73d239070cb48..7bb348143d146 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java @@ -63,7 +63,6 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Timeout; @@ -108,7 +107,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; -@Disabled @SuppressWarnings("ClassFanOutComplexity") @Timeout(1200) @Tag("integration") diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java index 807d84f7a95c2..ac6eef6fa6800 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.connect.data.Values.Parser; import org.apache.kafka.connect.errors.DataException; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -886,7 +885,6 @@ public void shouldConvertTimeValues() { assertEquals(currentMillis, t5.getTime()); } - @Disabled @Test public void shouldConvertDateValues() { LocalDateTime localTime = LocalDateTime.now(); From 21ef99bc94d4bba7677d1f114af7e594464abb9a Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 12 May 2025 21:00:16 +0800 Subject: [PATCH 59/74] addressed by comments --- checkstyle/import-control.xml | 1 + .../java/org/apache/kafka/raft/MetadataLogConfig.java | 10 +++++----- .../PurgeRepartitionTopicIntegrationTest.java | 5 +++-- 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 979b5379cc21c..b130609bd31dc 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -430,6 +430,7 @@ + diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index 805d16d361c7b..c86bbaa66b0d5 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -96,8 +96,8 @@ public class MetadataLogConfig { private final long deleteDelayMillis; /** - * Configuration for the metadata log - * @param logSegmentBytes The maximum size of a single metadata log file + * Configuration for the metadata log, the constructor only for testing. + * @param internalLogSegmentBytes The maximum size of a single metadata log file * @param logSegmentMillis The maximum time before a new metadata log file is rolled out * @param retentionMaxBytes The size of the metadata log and snapshots before deleting old snapshots and log files * @param retentionMillis The time to keep a metadata log file or snapshot before deleting it @@ -105,15 +105,15 @@ public class MetadataLogConfig { * @param maxFetchSizeInBytes The maximum number of bytes to read when fetching from the metadata log * @param deleteDelayMillis The amount of time to wait before deleting a file from the filesystem */ - public MetadataLogConfig(int logSegmentBytes, + public MetadataLogConfig(int internalLogSegmentBytes, long logSegmentMillis, long retentionMaxBytes, long retentionMillis, int maxBatchSizeInBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { - this.logSegmentBytes = logSegmentBytes; - this.internalLogSegmentBytes = logSegmentBytes; + this.logSegmentBytes = internalLogSegmentBytes; + this.internalLogSegmentBytes = internalLogSegmentBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java index be85c9fd0af86..67203b8926642 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/PurgeRepartitionTopicIntegrationTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; @@ -114,7 +115,7 @@ public final boolean conditionMet() { .get(); return config.get(TopicConfig.CLEANUP_POLICY_CONFIG).value().equals(TopicConfig.CLEANUP_POLICY_DELETE) && config.get(TopicConfig.SEGMENT_MS_CONFIG).value().equals(PURGE_INTERVAL_MS.toString()) - && config.get("internal.segment.bytes").value().equals(PURGE_SEGMENT_BYTES.toString()); + && config.get(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG).value().equals(PURGE_SEGMENT_BYTES.toString()); } catch (final Exception e) { return false; } @@ -171,7 +172,7 @@ public void setup() { streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(APPLICATION_ID).getPath()); streamsConfiguration.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_MS_CONFIG), PURGE_INTERVAL_MS); - streamsConfiguration.put(StreamsConfig.topicPrefix("internal.segment.bytes"), PURGE_SEGMENT_BYTES); + streamsConfiguration.put(StreamsConfig.topicPrefix(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG), PURGE_SEGMENT_BYTES); streamsConfiguration.put(StreamsConfig.producerPrefix(ProducerConfig.BATCH_SIZE_CONFIG), PURGE_SEGMENT_BYTES / 2); // we cannot allow batch size larger than segment size final StreamsBuilder builder = new StreamsBuilder(); From f5aa91ac6e19369e007c8d56ea26b4144df66751 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 15 May 2025 23:29:47 +0800 Subject: [PATCH 60/74] addressed by comments --- .../main/scala/kafka/admin/ConfigCommand.scala | 2 +- .../main/scala/kafka/raft/KafkaMetadataLog.scala | 5 +---- .../unit/kafka/server/KafkaConfigTest.scala | 2 ++ .../org/apache/kafka/raft/MetadataLogConfig.java | 16 ++++------------ .../server/config/ServerTopicConfigSynonyms.java | 1 + .../kafka/storage/internals/log/LogConfig.java | 7 ++++++- .../org/apache/kafka/tools/TopicCommand.java | 2 +- 7 files changed, 16 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index ee1fbbca2da8f..f84ee1c66781e 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -529,7 +529,7 @@ object ConfigCommand extends Logging { private val nl: String = System.lineSeparator() val addConfig: OptionSpec[String] = parser.accepts("add-config", "Key Value pairs of configs to add. Square brackets can be used to group values which contain commas: 'k1=v1,k2=[v1,v2,v2],k3=v3'. The following is a list of valid configurations: " + - "For entity-type '" + TopicType + "': " + LogConfig.configNames.asScala.map("\t" + _).mkString(nl, nl, nl) + + "For entity-type '" + TopicType + "': " + LogConfig.excludeInternalConfigNames.asScala.map("\t" + _).mkString(nl, nl, nl) + "For entity-type '" + BrokerType + "': " + DynamicConfig.Broker.names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + "For entity-type '" + UserType + "': " + QuotaConfig.scramMechanismsPlusUserAndClientQuotaConfigs().names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + "For entity-type '" + ClientType + "': " + QuotaConfig.userAndClientQuotaConfigs().names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index a94bcbd78c8ef..a3c00f4f07eca 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -589,10 +589,7 @@ object KafkaMetadataLog extends Logging { ): KafkaMetadataLog = { val props = new Properties() props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString) - if (config.internalLogSegmentBytes != null) - props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) - else - props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) + props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT.toString) // Disable time and byte retention when deleting segments diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 50c988b4649c7..527397a7f64c7 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1132,6 +1132,8 @@ class KafkaConfigTest { assertDynamic(kafkaConfigProp, 10015L, () => config.remoteLogManagerConfig.logLocalRetentionMs) case TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 10016L, () => config.remoteLogManagerConfig.logLocalRetentionBytes) + case ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG => + // update metadata log config and topic config // not dynamically updatable case QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG => // topic only config diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index c86bbaa66b0d5..0c6fedee6a903 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -18,9 +18,9 @@ import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.record.Records; import org.apache.kafka.server.config.ServerLogConfigs; +import java.util.Objects; import java.util.concurrent.TimeUnit; import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH; @@ -80,14 +80,13 @@ public class MetadataLogConfig { .define(METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG, LONG, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_DOC) .define(METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC) .define(METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, METADATA_LOG_DIR_DOC) - .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(Records.LOG_OVERHEAD), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) + .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(1024 * 1024), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); private final int logSegmentBytes; - private final Integer internalLogSegmentBytes; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -113,7 +112,6 @@ public MetadataLogConfig(int internalLogSegmentBytes, int maxFetchSizeInBytes, long deleteDelayMillis) { this.logSegmentBytes = internalLogSegmentBytes; - this.internalLogSegmentBytes = internalLogSegmentBytes; this.logSegmentMillis = logSegmentMillis; this.retentionMaxBytes = retentionMaxBytes; this.retentionMillis = retentionMillis; @@ -123,8 +121,7 @@ public MetadataLogConfig(int internalLogSegmentBytes, } public MetadataLogConfig(AbstractConfig config) { - this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); - this.internalLogSegmentBytes = config.getInt(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG); + this.logSegmentBytes = Objects.requireNonNullElseGet(config.getInt(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG), () -> config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG)); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); @@ -134,12 +131,7 @@ public MetadataLogConfig(AbstractConfig config) { } public int logSegmentBytes() { - if (internalLogSegmentBytes == null) return logSegmentBytes; - return internalLogSegmentBytes; - } - - public Integer internalLogSegmentBytes() { - return internalLogSegmentBytes; + return logSegmentBytes; } public long logSegmentMillis() { diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index 5394d2d2c3a06..eaf2b0e1f6a68 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -50,6 +50,7 @@ public final class ServerTopicConfigSynonyms { // Topic configs with no mapping to a server config can be found in `LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS` public static final Map> ALL_TOPIC_CONFIG_SYNONYMS = Utils.mkMap( sameNameWithLogPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), + sameName("internal.log." + TopicConfig.SEGMENT_BYTES_CONFIG), listWithLogPrefix(TopicConfig.SEGMENT_MS_CONFIG, new ConfigSynonym("roll.ms"), new ConfigSynonym("roll.hours", ConfigSynonym.HOURS_TO_MILLISECONDS)), diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 5a1509ac7b1c2..6cb14b339ea78 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -139,7 +139,8 @@ public Optional serverConfigName(String configName) { public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs - public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; + private static final String INTERNAL_PREFIX = "internal."; + public static final String INTERNAL_SEGMENT_BYTES_CONFIG = INTERNAL_PREFIX + "segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; @@ -459,6 +460,10 @@ public static List configNames() { return CONFIG.names().stream().sorted().toList(); } + public static List excludeInternalConfigNames() { + return CONFIG.names().stream().filter(name -> !name.contains(INTERNAL_PREFIX)).sorted().toList(); + } + public static Optional serverConfigName(String configName) { return CONFIG.serverConfigName(configName); } diff --git a/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java index 9c67917d76a66..9907cbd62d335 100644 --- a/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java @@ -769,7 +769,7 @@ public TopicCommandOptions(String[] args) { .ofType(String.class); nl = System.lineSeparator(); - String logConfigNames = LogConfig.configNames().stream().map(config -> "\t" + config).collect(Collectors.joining(nl)); + String logConfigNames = LogConfig.excludeInternalConfigNames().stream().map(config -> "\t" + config).collect(Collectors.joining(nl)); configOpt = parser.accepts("config", "A topic configuration override for the topic being created." + " The following is a list of valid configurations: " + nl + logConfigNames + nl + "See the Kafka documentation for full details on the topic configs." + From d46fe99617048d8cccc7dec91e3af7d8141206cc Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 15 May 2025 23:39:10 +0800 Subject: [PATCH 61/74] addressed by comments --- core/src/main/scala/kafka/raft/KafkaMetadataLog.scala | 2 ++ core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala | 2 -- .../main/java/org/apache/kafka/metadata/KafkaConfigSchema.java | 1 + .../apache/kafka/server/config/ServerTopicConfigSynonyms.java | 1 - 4 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index a3c00f4f07eca..f54132067a2b3 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -589,6 +589,8 @@ object KafkaMetadataLog extends Logging { ): KafkaMetadataLog = { val props = new Properties() props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString) + // Since MetadataLogConfig validates the log segment size using a minimum allowed value, + // we can safely use `internal.segment.bytes` to configure it instead of relying on `segment.bytes`. props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT.toString) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 527397a7f64c7..50c988b4649c7 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1132,8 +1132,6 @@ class KafkaConfigTest { assertDynamic(kafkaConfigProp, 10015L, () => config.remoteLogManagerConfig.logLocalRetentionMs) case TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG => assertDynamic(kafkaConfigProp, 10016L, () => config.remoteLogManagerConfig.logLocalRetentionBytes) - case ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG => - // update metadata log config and topic config // not dynamically updatable case QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG => // topic only config diff --git a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java index 13c1743ee5560..a8cf8a47bb13c 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java @@ -166,6 +166,7 @@ public Map resolveEffectiveTopicConfigs( ConfigDef configDef = configDefs.getOrDefault(ConfigResource.Type.TOPIC, EMPTY_CONFIG_DEF); HashMap effectiveConfigs = new HashMap<>(); for (ConfigDef.ConfigKey configKey : configDef.configKeys().values()) { + // This config is internal; if the user hasn't set it explicitly, it should not be returned. if (configKey.internalConfig && !dynamicTopicConfigs.containsKey(configKey.name)) { continue; } diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index eaf2b0e1f6a68..5394d2d2c3a06 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -50,7 +50,6 @@ public final class ServerTopicConfigSynonyms { // Topic configs with no mapping to a server config can be found in `LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS` public static final Map> ALL_TOPIC_CONFIG_SYNONYMS = Utils.mkMap( sameNameWithLogPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), - sameName("internal.log." + TopicConfig.SEGMENT_BYTES_CONFIG), listWithLogPrefix(TopicConfig.SEGMENT_MS_CONFIG, new ConfigSynonym("roll.ms"), new ConfigSynonym("roll.hours", ConfigSynonym.HOURS_TO_MILLISECONDS)), From 8316923023d46e8b6a145d184e6c62d3556b8c1f Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 16 May 2025 23:27:29 +0800 Subject: [PATCH 62/74] addressed by comments --- .../java/org/apache/kafka/raft/MetadataLogConfig.java | 2 +- .../apache/kafka/storage/internals/log/LogConfig.java | 10 ++++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index 0c6fedee6a903..77dfcea3cf1bf 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -80,7 +80,7 @@ public class MetadataLogConfig { .define(METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG, LONG, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_DOC) .define(METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC) .define(METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, METADATA_LOG_DIR_DOC) - .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(1024 * 1024), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) + .define(METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, METADATA_LOG_SEGMENT_BYTES_DEFAULT, atLeast(8 * 1024 * 1024), HIGH, METADATA_LOG_SEGMENT_BYTES_DOC) .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 40afdffb6d4ef..480ec69d38ba3 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -139,8 +139,7 @@ public Optional serverConfigName(String configName) { public static final long DEFAULT_LOCAL_RETENTION_BYTES = -2; // It indicates the value to be derived from RetentionBytes public static final long DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates the value to be derived from RetentionMs - private static final String INTERNAL_PREFIX = "internal."; - public static final String INTERNAL_SEGMENT_BYTES_CONFIG = INTERNAL_PREFIX + "segment.bytes"; + public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; @@ -149,7 +148,6 @@ public Optional serverConfigName(String configName) { TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, - INTERNAL_SEGMENT_BYTES_CONFIG, QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG ); @@ -461,7 +459,11 @@ public static List configNames() { } public static List excludeInternalConfigNames() { - return CONFIG.names().stream().filter(name -> !name.contains(INTERNAL_PREFIX)).sorted().toList(); + return CONFIG.configKeys().entrySet() + .stream() + .filter(entry -> !entry.getValue().internalConfig) + .map(Map.Entry::getKey) + .sorted().toList(); } public static Optional serverConfigName(String configName) { From 85d77d3932215ba8e31fda98113a3fa58f61a452 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 17 May 2025 00:37:46 +0800 Subject: [PATCH 63/74] fix fail test --- .../src/test/scala/unit/kafka/log/LogConfigTest.scala | 11 ++++++++--- .../server/config/ServerTopicConfigSynonyms.java | 1 + 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 88e78539412f9..b5cda3321a500 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -28,7 +28,7 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import java.util.{Collections, Properties} -import org.apache.kafka.server.config.ServerLogConfigs +import org.apache.kafka.server.config.{ServerLogConfigs, ServerTopicConfigSynonyms} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.storage.internals.log.{LogConfig, ThrottledReplicaListValidator} import org.junit.jupiter.params.ParameterizedTest @@ -52,8 +52,13 @@ class LogConfigTest { assertTrue(LogConfig.configNames.asScala .filter(config => !LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS.contains(config)) .forall { config => - val serverConfigOpt = LogConfig.serverConfigName(config) - serverConfigOpt.isPresent && (serverConfigOpt.get != null) + // this internal config naming pattern is not as same as a default pattern + if (config.equals(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG)) { + ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.containsKey(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG) + } else { + val serverConfigOpt = LogConfig.serverConfigName(config) + serverConfigOpt.isPresent && (serverConfigOpt.get != null) + } }) } diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index 5394d2d2c3a06..eaf2b0e1f6a68 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -50,6 +50,7 @@ public final class ServerTopicConfigSynonyms { // Topic configs with no mapping to a server config can be found in `LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS` public static final Map> ALL_TOPIC_CONFIG_SYNONYMS = Utils.mkMap( sameNameWithLogPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), + sameName("internal.log." + TopicConfig.SEGMENT_BYTES_CONFIG), listWithLogPrefix(TopicConfig.SEGMENT_MS_CONFIG, new ConfigSynonym("roll.ms"), new ConfigSynonym("roll.hours", ConfigSynonym.HOURS_TO_MILLISECONDS)), From 872c08f75d20f7814a4e950d2a1ed2f4256095ae Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 17 May 2025 15:17:27 +0800 Subject: [PATCH 64/74] addressed by comment --- .../scala/unit/kafka/log/LogConfigTest.scala | 28 +------------------ .../config/ServerTopicConfigSynonyms.java | 7 ++++- .../storage/internals/log/LogConfig.java | 10 ------- 3 files changed, 7 insertions(+), 38 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index b5cda3321a500..296736fc67817 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -28,40 +28,14 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import java.util.{Collections, Properties} -import org.apache.kafka.server.config.{ServerLogConfigs, ServerTopicConfigSynonyms} +import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.storage.internals.log.{LogConfig, ThrottledReplicaListValidator} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource -import scala.jdk.CollectionConverters._ - class LogConfigTest { - /** - * This test verifies that KafkaConfig object initialization does not depend on - * LogConfig initialization. Bad things happen due to static initialization - * order dependencies. For example, LogConfig.configDef ends up adding null - * values in serverDefaultConfigNames. This test ensures that the mapping of - * keys from LogConfig to KafkaConfig are not missing values. - */ - @Test - def ensureNoStaticInitializationOrderDependency(): Unit = { - // Access any KafkaConfig val to load KafkaConfig object before LogConfig. - assertNotNull(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG) - assertTrue(LogConfig.configNames.asScala - .filter(config => !LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS.contains(config)) - .forall { config => - // this internal config naming pattern is not as same as a default pattern - if (config.equals(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG)) { - ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.containsKey(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG) - } else { - val serverConfigOpt = LogConfig.serverConfigName(config) - serverConfigOpt.isPresent && (serverConfigOpt.get != null) - } - }) - } - @Test def testKafkaConfigToProps(): Unit = { val millisInHour = 60L * 60L * 1000L diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index eaf2b0e1f6a68..e5e5a0c37ea3d 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -29,6 +29,7 @@ public final class ServerTopicConfigSynonyms { public static final String LOG_PREFIX = "log."; + public static final String INTERNAL_PREFIX = "internal."; public static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner."; /** @@ -50,7 +51,7 @@ public final class ServerTopicConfigSynonyms { // Topic configs with no mapping to a server config can be found in `LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS` public static final Map> ALL_TOPIC_CONFIG_SYNONYMS = Utils.mkMap( sameNameWithLogPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), - sameName("internal.log." + TopicConfig.SEGMENT_BYTES_CONFIG), + sameNameWithInternalLogPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), listWithLogPrefix(TopicConfig.SEGMENT_MS_CONFIG, new ConfigSynonym("roll.ms"), new ConfigSynonym("roll.hours", ConfigSynonym.HOURS_TO_MILLISECONDS)), @@ -137,6 +138,10 @@ private static Entry> listWithLogPrefix(String topic return Utils.mkEntry(topicConfigName, synonymsWithPrefix); } + private static Entry> sameNameWithInternalLogPrefix(String configName) { + return Utils.mkEntry(configName, List.of(new ConfigSynonym(INTERNAL_PREFIX + LOG_PREFIX + configName))); + } + private static Entry> single(String topicConfigName, String brokerConfigName) { return Utils.mkEntry(topicConfigName, List.of(new ConfigSynonym(brokerConfigName))); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 480ec69d38ba3..ce6f69d7dc0ba 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -142,16 +142,6 @@ public Optional serverConfigName(String configName) { public static final String INTERNAL_SEGMENT_BYTES_CONFIG = "internal.segment.bytes"; public static final String INTERNAL_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; - - // Visible for testing - public static final Set CONFIGS_WITH_NO_SERVER_DEFAULTS = Set.of( - TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, - TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, - TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, - QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, - QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG - ); - public static final ConfigDef SERVER_CONFIG_DEF = new ConfigDef() .define(ServerLogConfigs.NUM_PARTITIONS_CONFIG, INT, ServerLogConfigs.NUM_PARTITIONS_DEFAULT, atLeast(1), MEDIUM, ServerLogConfigs.NUM_PARTITIONS_DOC) .define(ServerLogConfigs.LOG_DIR_CONFIG, STRING, ServerLogConfigs.LOG_DIR_DEFAULT, HIGH, ServerLogConfigs.LOG_DIR_DOC) From 9a1afb07390b6b4b21a503f8d314926e1ee04cab Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 17 May 2025 19:27:05 +0800 Subject: [PATCH 65/74] revert broker setting --- .../kafka/server/config/ServerTopicConfigSynonyms.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index e5e5a0c37ea3d..5394d2d2c3a06 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -29,7 +29,6 @@ public final class ServerTopicConfigSynonyms { public static final String LOG_PREFIX = "log."; - public static final String INTERNAL_PREFIX = "internal."; public static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner."; /** @@ -51,7 +50,6 @@ public final class ServerTopicConfigSynonyms { // Topic configs with no mapping to a server config can be found in `LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS` public static final Map> ALL_TOPIC_CONFIG_SYNONYMS = Utils.mkMap( sameNameWithLogPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), - sameNameWithInternalLogPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), listWithLogPrefix(TopicConfig.SEGMENT_MS_CONFIG, new ConfigSynonym("roll.ms"), new ConfigSynonym("roll.hours", ConfigSynonym.HOURS_TO_MILLISECONDS)), @@ -138,10 +136,6 @@ private static Entry> listWithLogPrefix(String topic return Utils.mkEntry(topicConfigName, synonymsWithPrefix); } - private static Entry> sameNameWithInternalLogPrefix(String configName) { - return Utils.mkEntry(configName, List.of(new ConfigSynonym(INTERNAL_PREFIX + LOG_PREFIX + configName))); - } - private static Entry> single(String topicConfigName, String brokerConfigName) { return Utils.mkEntry(topicConfigName, List.of(new ConfigSynonym(brokerConfigName))); } From ee9711b523769780d05162c238cb362ff37ffaea Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 17 May 2025 19:36:10 +0800 Subject: [PATCH 66/74] using internal config --- .../kafka/server/config/ServerTopicConfigSynonyms.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index 5394d2d2c3a06..6e77e23810319 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -29,6 +29,7 @@ public final class ServerTopicConfigSynonyms { public static final String LOG_PREFIX = "log."; + public static final String INTERNAL_PREFIX = "internal."; public static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner."; /** @@ -50,6 +51,7 @@ public final class ServerTopicConfigSynonyms { // Topic configs with no mapping to a server config can be found in `LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS` public static final Map> ALL_TOPIC_CONFIG_SYNONYMS = Utils.mkMap( sameNameWithLogPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), + sameNameWithInternalLogPrefix("internal.segment.bytes"), listWithLogPrefix(TopicConfig.SEGMENT_MS_CONFIG, new ConfigSynonym("roll.ms"), new ConfigSynonym("roll.hours", ConfigSynonym.HOURS_TO_MILLISECONDS)), @@ -136,6 +138,10 @@ private static Entry> listWithLogPrefix(String topic return Utils.mkEntry(topicConfigName, synonymsWithPrefix); } + private static Entry> sameNameWithInternalLogPrefix(String configName) { + return Utils.mkEntry(configName, List.of(new ConfigSynonym(INTERNAL_PREFIX + LOG_PREFIX + configName.replaceFirst(INTERNAL_PREFIX, "")))); + } + private static Entry> single(String topicConfigName, String brokerConfigName) { return Utils.mkEntry(topicConfigName, List.of(new ConfigSynonym(brokerConfigName))); } From cece060ba4cfad62dfdfcaad9a8989a5a8d86c30 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 17 May 2025 20:15:37 +0800 Subject: [PATCH 67/74] fix testDynamicLogConfigs --- core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 5fabdbf1226a5..dafbb2478713b 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1137,6 +1137,8 @@ class KafkaConfigTest { // topic only config case QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG => // topic only config + case "internal.segment.bytes" => + // topic internal config case prop => fail(prop + " must be explicitly checked for dynamic updatability. Note that LogConfig(s) require that KafkaConfig value lookups are dynamic and not static values.") } From d8426bcd2955fc3b803db6937f51e83479e3e106 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 17 May 2025 22:41:37 +0800 Subject: [PATCH 68/74] add the comment --- .../apache/kafka/server/config/ServerTopicConfigSynonyms.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index 6e77e23810319..21b4821c487d8 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -48,9 +48,10 @@ public final class ServerTopicConfigSynonyms { * both the first and the second synonyms are configured, we will use only the value of * the first synonym and ignore the second. */ - // Topic configs with no mapping to a server config can be found in `LogConfig.CONFIGS_WITH_NO_SERVER_DEFAULTS` public static final Map> ALL_TOPIC_CONFIG_SYNONYMS = Utils.mkMap( sameNameWithLogPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), + // Due to the internal.segment.bytes is in storage module, thus we could not use the + // LogConfig#INTERNAL_SEGMENT_BYTES_CONFIG directly. We need to use the string value instead. sameNameWithInternalLogPrefix("internal.segment.bytes"), listWithLogPrefix(TopicConfig.SEGMENT_MS_CONFIG, new ConfigSynonym("roll.ms"), From b622e08c7f3bad6c576b91cd78ee7bed2ed7f97b Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 20 May 2025 21:11:24 +0800 Subject: [PATCH 69/74] addressed by comments --- .../test/scala/unit/kafka/server/KafkaConfigTest.scala | 2 -- .../java/org/apache/kafka/raft/MetadataLogConfig.java | 3 +-- .../org/apache/kafka/server/config/ServerLogConfigs.java | 3 --- .../kafka/server/config/ServerTopicConfigSynonyms.java | 8 -------- .../org/apache/kafka/storage/internals/log/LogConfig.java | 3 +-- 5 files changed, 2 insertions(+), 17 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index dafbb2478713b..080f4f7a6191a 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -822,8 +822,6 @@ class KafkaConfigTest { case ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG => // no op - case ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG => assertPropertyInvalid(baseProperties, name, "unknown_policy", "0") diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index 77dfcea3cf1bf..b43ab20b39844 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.server.config.ServerLogConfigs; -import java.util.Objects; import java.util.concurrent.TimeUnit; import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH; @@ -121,7 +120,7 @@ public MetadataLogConfig(int internalLogSegmentBytes, } public MetadataLogConfig(AbstractConfig config) { - this.logSegmentBytes = Objects.requireNonNullElseGet(config.getInt(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG), () -> config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG)); + this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 16b5c7a99a39c..2c2d44ef012e5 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -43,9 +43,6 @@ public class ServerLogConfigs { public static final String LOG_SEGMENT_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); public static final String LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file"; - public static final String INTERNAL_LOG_SEGMENT_BYTES_CONFIG = "internal." + ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); - public static final String INTERNAL_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single log file. This should be used for testing only."; - public static final String LOG_ROLL_TIME_MILLIS_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_MS_CONFIG); public static final String LOG_ROLL_TIME_HOURS_CONFIG = LOG_PREFIX + "roll.hours"; public static final String LOG_ROLL_TIME_MILLIS_DOC = "The maximum time before a new log segment is rolled out (in milliseconds). If not set, the value in " + LOG_ROLL_TIME_HOURS_CONFIG + " is used"; diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java index 21b4821c487d8..c05f9f2816ae0 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerTopicConfigSynonyms.java @@ -29,7 +29,6 @@ public final class ServerTopicConfigSynonyms { public static final String LOG_PREFIX = "log."; - public static final String INTERNAL_PREFIX = "internal."; public static final String LOG_CLEANER_PREFIX = LOG_PREFIX + "cleaner."; /** @@ -50,9 +49,6 @@ public final class ServerTopicConfigSynonyms { */ public static final Map> ALL_TOPIC_CONFIG_SYNONYMS = Utils.mkMap( sameNameWithLogPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), - // Due to the internal.segment.bytes is in storage module, thus we could not use the - // LogConfig#INTERNAL_SEGMENT_BYTES_CONFIG directly. We need to use the string value instead. - sameNameWithInternalLogPrefix("internal.segment.bytes"), listWithLogPrefix(TopicConfig.SEGMENT_MS_CONFIG, new ConfigSynonym("roll.ms"), new ConfigSynonym("roll.hours", ConfigSynonym.HOURS_TO_MILLISECONDS)), @@ -139,10 +135,6 @@ private static Entry> listWithLogPrefix(String topic return Utils.mkEntry(topicConfigName, synonymsWithPrefix); } - private static Entry> sameNameWithInternalLogPrefix(String configName) { - return Utils.mkEntry(configName, List.of(new ConfigSynonym(INTERNAL_PREFIX + LOG_PREFIX + configName.replaceFirst(INTERNAL_PREFIX, "")))); - } - private static Entry> single(String topicConfigName, String brokerConfigName) { return Utils.mkEntry(topicConfigName, List.of(new ConfigSynonym(brokerConfigName))); } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index ce6f69d7dc0ba..9b2db6dde882f 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -179,8 +179,7 @@ public Optional serverConfigName(String configName) { .define(ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, CLASS, null, LOW, ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_DOC) .define(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG, LONG, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DEFAULT, atLeast(1), LOW, ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_DOC) - .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC) - .defineInternal(ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, MEDIUM, ServerLogConfigs.INTERNAL_LOG_SEGMENT_BYTES_DOC); + .defineInternal(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, LONG, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT, atLeast(0), LOW, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DOC); private static final LogConfigDef CONFIG = new LogConfigDef(); static { From 68657e6c7e057e011a4be4f57eb97ab5d0a5d319 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 22 May 2025 21:20:57 +0800 Subject: [PATCH 70/74] addressed by comments --- .../scala/kafka/admin/ConfigCommand.scala | 2 +- .../scala/kafka/raft/KafkaMetadataLog.scala | 7 ++- .../kafka/raft/KafkaMetadataLogTest.scala | 35 ++++++++--- .../kafka/tools/DumpLogSegmentsTest.scala | 25 +++++++- .../apache/kafka/raft/MetadataLogConfig.java | 59 +++++++++---------- .../storage/internals/log/LogConfig.java | 2 +- .../org/apache/kafka/tools/TopicCommand.java | 2 +- 7 files changed, 86 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 0ba3537818e42..8fb37665c3ae6 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -529,7 +529,7 @@ object ConfigCommand extends Logging { private val nl: String = System.lineSeparator() val addConfig: OptionSpec[String] = parser.accepts("add-config", "Key Value pairs of configs to add. Square brackets can be used to group values which contain commas: 'k1=v1,k2=[v1,v2,v2],k3=v3'. The following is a list of valid configurations: " + - "For entity-type '" + TopicType + "': " + LogConfig.excludeInternalConfigNames.asScala.map("\t" + _).mkString(nl, nl, nl) + + "For entity-type '" + TopicType + "': " + LogConfig.nonInternalConfigNames.asScala.map("\t" + _).mkString(nl, nl, nl) + "For entity-type '" + BrokerType + "': " + DynamicConfig.Broker.names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + "For entity-type '" + UserType + "': " + QuotaConfig.scramMechanismsPlusUserAndClientQuotaConfigs().names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + "For entity-type '" + ClientType + "': " + QuotaConfig.userAndClientQuotaConfigs().names.asScala.toSeq.sorted.map("\t" + _).mkString(nl, nl, nl) + diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index f54132067a2b3..e21ce821d5595 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -589,9 +589,10 @@ object KafkaMetadataLog extends Logging { ): KafkaMetadataLog = { val props = new Properties() props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString) - // Since MetadataLogConfig validates the log segment size using a minimum allowed value, - // we can safely use `internal.segment.bytes` to configure it instead of relying on `segment.bytes`. - props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) + if (config.internalSegmentBytes() != null) + props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) + else + props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT.toString) // Disable time and byte retention when deleting segments diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index 12c06fe0b9c71..f51d2d4e32942 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -43,7 +43,7 @@ import org.junit.jupiter.params.provider.ArgumentsSource import net.jqwik.api.AfterFailureMode import net.jqwik.api.ForAll import net.jqwik.api.Property -import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.config.{AbstractConfig, ConfigException} import org.apache.kafka.server.common.OffsetAndEpoch import java.io.File @@ -688,7 +688,7 @@ final class KafkaMetadataLogTest { val leaderEpoch = 5 val maxBatchSizeInBytes = 16384 val recordSize = 64 - val config = new MetadataLogConfig( + val config = createMetadataLogConfig( DefaultMetadataLogConfig.logSegmentBytes, DefaultMetadataLogConfig.logSegmentMillis, DefaultMetadataLogConfig.retentionMaxBytes, @@ -907,7 +907,7 @@ final class KafkaMetadataLogTest { @Test def testAdvanceLogStartOffsetAfterCleaning(): Unit = { - val config = new MetadataLogConfig( + val config = createMetadataLogConfig( 512, 10 * 1000, 256, @@ -943,7 +943,7 @@ final class KafkaMetadataLogTest { @Test def testDeleteSnapshots(): Unit = { // Generate some logs and a few snapshots, set retention low and verify that cleaning occurs - val config = new MetadataLogConfig( + val config = createMetadataLogConfig( 1024, 10 * 1000, 1024, @@ -976,7 +976,7 @@ final class KafkaMetadataLogTest { @Test def testSoftRetentionLimit(): Unit = { // Set retention equal to the segment size and generate slightly more than one segment of logs - val config = new MetadataLogConfig( + val config = createMetadataLogConfig( 10240, 10 * 1000, 10240, @@ -1019,7 +1019,7 @@ final class KafkaMetadataLogTest { @Test def testSegmentsLessThanLatestSnapshot(): Unit = { - val config = new MetadataLogConfig( + val config = createMetadataLogConfig( 10240, 10 * 1000, 10240, @@ -1077,7 +1077,7 @@ object KafkaMetadataLogTest { override def read(input: protocol.Readable, size: Int): Array[Byte] = input.readArray(size) } - val DefaultMetadataLogConfig = new MetadataLogConfig( + val DefaultMetadataLogConfig = createMetadataLogConfig( 100 * 1024, 10 * 1000, 100 * 1024, @@ -1161,4 +1161,25 @@ object KafkaMetadataLogTest { } dir } + + private def createMetadataLogConfig( + internalLogSegmentBytes: Int, + logSegmentMillis: Long, + retentionMaxBytes: Long, + retentionMillis: Long, + maxBatchSizeInBytes: Int, + maxFetchSizeInBytes: Int, + deleteDelayMillis: Long + ): MetadataLogConfig = { + val config = util.Map.of( + MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, internalLogSegmentBytes, + MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, logSegmentMillis, + MetadataLogConfig.METADATA_MAX_RETENTION_BYTES_CONFIG, retentionMaxBytes, + MetadataLogConfig.METADATA_MAX_RETENTION_MILLIS_CONFIG, retentionMillis, + MetadataLogConfig.INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG, maxBatchSizeInBytes, + MetadataLogConfig.INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG, maxFetchSizeInBytes, + MetadataLogConfig.INTERNAL_DELETE_DELAY_MILLIS_CONFIG, deleteDelayMillis, + ) + new MetadataLogConfig(new AbstractConfig(MetadataLogConfig.CONFIG_DEF, config, false)) + } } diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index fd199504030f9..4f7498e9158cd 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -33,7 +33,7 @@ import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.{Assignment, import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.compress.Compression -import org.apache.kafka.common.config.TopicConfig +import org.apache.kafka.common.config.{AbstractConfig, TopicConfig} import org.apache.kafka.common.metadata.{PartitionChangeRecord, RegisterBrokerRecord, TopicRecord} import org.apache.kafka.common.protocol.{ApiMessage, ByteBufferAccessor, MessageUtil, ObjectSerializationCache} import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, MemoryRecords, Record, RecordVersion, SimpleRecord} @@ -544,7 +544,7 @@ class DumpLogSegmentsTest { logDir, time, time.scheduler, - new MetadataLogConfig( + createMetadataLogConfig( 100 * 1024, 10 * 1000, 100 * 1024, @@ -1154,4 +1154,25 @@ class DumpLogSegmentsTest { )) ) } + + private def createMetadataLogConfig( + internalLogSegmentBytes: Int, + logSegmentMillis: Long, + retentionMaxBytes: Long, + retentionMillis: Long, + maxBatchSizeInBytes: Int, + maxFetchSizeInBytes: Int, + deleteDelayMillis: Long + ): MetadataLogConfig = { + val config = util.Map.of( + MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, internalLogSegmentBytes, + MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, logSegmentMillis, + MetadataLogConfig.METADATA_MAX_RETENTION_BYTES_CONFIG, retentionMaxBytes, + MetadataLogConfig.METADATA_MAX_RETENTION_MILLIS_CONFIG, retentionMillis, + MetadataLogConfig.INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG, maxBatchSizeInBytes, + MetadataLogConfig.INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG, maxFetchSizeInBytes, + MetadataLogConfig.INTERNAL_DELETE_DELAY_MILLIS_CONFIG, deleteDelayMillis, + ) + new MetadataLogConfig(new AbstractConfig(MetadataLogConfig.CONFIG_DEF, config, false)) + } } diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index b43ab20b39844..f10fa0d98ef1e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.server.config.ServerLogConfigs; +import java.util.Objects; import java.util.concurrent.TimeUnit; import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH; @@ -55,6 +56,9 @@ public class MetadataLogConfig { public static final String METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file."; public static final int METADATA_LOG_SEGMENT_BYTES_DEFAULT = 1024 * 1024 * 1024; + public static final String INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG = "internal.metadata.log.segment.bytes"; + public static final String INTERNAL_METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file, only for testing."; + public static final String METADATA_LOG_SEGMENT_MILLIS_CONFIG = "metadata.log.segment.ms"; public static final String METADATA_LOG_SEGMENT_MILLIS_DOC = "The maximum time before a new metadata log file is rolled out (in milliseconds)."; public static final long METADATA_LOG_SEGMENT_MILLIS_DEFAULT = 24 * 7 * 60 * 60 * 1000L; @@ -75,6 +79,15 @@ public class MetadataLogConfig { "controller should write no-op records to the metadata partition. If the value is 0, no-op records " + "are not appended to the metadata partition. The default value is " + METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT; + public static final String INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG = "internal.max.batch.size.in.bytes"; + public static final String INTERNAL_MAX_BATCH_SIZE_IN_BYTES_DOC = "The largest record batch size allowed in the metadata log, only for testing."; + + public static final String INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG = "internal.max.fetch.size.in.bytes"; + public static final String INTERNAL_MAX_FETCH_SIZE_IN_BYTES_DOC = "The maximum number of bytes to read when fetching from the metadata log, only for testing."; + + public static final String INTERNAL_DELETE_DELAY_MILLIS_CONFIG = "internal.delete.delay.millis"; + public static final String INTERNAL_DELETE_DELAY_MILLIS_DOC = "The amount of time to wait before deleting a file from the filesystem, only for testing."; + public static final ConfigDef CONFIG_DEF = new ConfigDef() .define(METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG, LONG, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_DOC) .define(METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC) @@ -83,9 +96,14 @@ public class MetadataLogConfig { .define(METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, METADATA_LOG_SEGMENT_MILLIS_DEFAULT, null, HIGH, METADATA_LOG_SEGMENT_MILLIS_DOC) .define(METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_BYTES_DOC) .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) - .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC); + .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC) + .defineInternal(INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_METADATA_LOG_SEGMENT_BYTES_DOC) + .defineInternal(INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_MAX_BATCH_SIZE_IN_BYTES_DOC) + .defineInternal(INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_MAX_FETCH_SIZE_IN_BYTES_DOC) + .defineInternal(INTERNAL_DELETE_DELAY_MILLIS_CONFIG, LONG, null, null, LOW, INTERNAL_DELETE_DELAY_MILLIS_DOC); private final int logSegmentBytes; + private final Integer internalSegmentBytes; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; @@ -93,44 +111,23 @@ public class MetadataLogConfig { private final int maxFetchSizeInBytes; private final long deleteDelayMillis; - /** - * Configuration for the metadata log, the constructor only for testing. - * @param internalLogSegmentBytes The maximum size of a single metadata log file - * @param logSegmentMillis The maximum time before a new metadata log file is rolled out - * @param retentionMaxBytes The size of the metadata log and snapshots before deleting old snapshots and log files - * @param retentionMillis The time to keep a metadata log file or snapshot before deleting it - * @param maxBatchSizeInBytes The largest record batch size allowed in the metadata log - * @param maxFetchSizeInBytes The maximum number of bytes to read when fetching from the metadata log - * @param deleteDelayMillis The amount of time to wait before deleting a file from the filesystem - */ - public MetadataLogConfig(int internalLogSegmentBytes, - long logSegmentMillis, - long retentionMaxBytes, - long retentionMillis, - int maxBatchSizeInBytes, - int maxFetchSizeInBytes, - long deleteDelayMillis) { - this.logSegmentBytes = internalLogSegmentBytes; - this.logSegmentMillis = logSegmentMillis; - this.retentionMaxBytes = retentionMaxBytes; - this.retentionMillis = retentionMillis; - this.maxBatchSizeInBytes = maxBatchSizeInBytes; - this.maxFetchSizeInBytes = maxFetchSizeInBytes; - this.deleteDelayMillis = deleteDelayMillis; - } - public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); + this.internalSegmentBytes = config.getInt(INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG); this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); - this.maxBatchSizeInBytes = KafkaRaftClient.MAX_BATCH_SIZE_BYTES; - this.maxFetchSizeInBytes = KafkaRaftClient.MAX_FETCH_SIZE_BYTES; - this.deleteDelayMillis = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT; + this.maxBatchSizeInBytes = Objects.requireNonNullElse(config.getInt(INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG), KafkaRaftClient.MAX_BATCH_SIZE_BYTES); + this.maxFetchSizeInBytes = Objects.requireNonNullElse(config.getInt(INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG), KafkaRaftClient.MAX_FETCH_SIZE_BYTES); + this.deleteDelayMillis = Objects.requireNonNullElse(config.getLong(INTERNAL_DELETE_DELAY_MILLIS_CONFIG), ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT); } public int logSegmentBytes() { - return logSegmentBytes; + return Objects.requireNonNullElse(internalSegmentBytes, logSegmentBytes); + } + + public Integer internalSegmentBytes() { + return internalSegmentBytes; } public long logSegmentMillis() { diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 9b2db6dde882f..3c8b42c6258ad 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -447,7 +447,7 @@ public static List configNames() { return CONFIG.names().stream().sorted().toList(); } - public static List excludeInternalConfigNames() { + public static List nonInternalConfigNames() { return CONFIG.configKeys().entrySet() .stream() .filter(entry -> !entry.getValue().internalConfig) diff --git a/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java index 9907cbd62d335..4eee239069cfe 100644 --- a/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/TopicCommand.java @@ -769,7 +769,7 @@ public TopicCommandOptions(String[] args) { .ofType(String.class); nl = System.lineSeparator(); - String logConfigNames = LogConfig.excludeInternalConfigNames().stream().map(config -> "\t" + config).collect(Collectors.joining(nl)); + String logConfigNames = LogConfig.nonInternalConfigNames().stream().map(config -> "\t" + config).collect(Collectors.joining(nl)); configOpt = parser.accepts("config", "A topic configuration override for the topic being created." + " The following is a list of valid configurations: " + nl + logConfigNames + nl + "See the Kafka documentation for full details on the topic configs." + From 6ec200963c213b90d11ea18cb49d232e40432724 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 22 May 2025 21:57:19 +0800 Subject: [PATCH 71/74] fix fail test --- .../scala/kafka/raft/KafkaMetadataLogTest.scala | 16 ++++++++-------- .../unit/kafka/tools/DumpLogSegmentsTest.scala | 2 +- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index f51d2d4e32942..cf7ff56c0a30e 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -1171,14 +1171,14 @@ object KafkaMetadataLogTest { maxFetchSizeInBytes: Int, deleteDelayMillis: Long ): MetadataLogConfig = { - val config = util.Map.of( - MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, internalLogSegmentBytes, - MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, logSegmentMillis, - MetadataLogConfig.METADATA_MAX_RETENTION_BYTES_CONFIG, retentionMaxBytes, - MetadataLogConfig.METADATA_MAX_RETENTION_MILLIS_CONFIG, retentionMillis, - MetadataLogConfig.INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG, maxBatchSizeInBytes, - MetadataLogConfig.INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG, maxFetchSizeInBytes, - MetadataLogConfig.INTERNAL_DELETE_DELAY_MILLIS_CONFIG, deleteDelayMillis, + val config: util.Map[String, Any] = util.Map.of( + MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, internalLogSegmentBytes, + MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, logSegmentMillis, + MetadataLogConfig.METADATA_MAX_RETENTION_BYTES_CONFIG, retentionMaxBytes, + MetadataLogConfig.METADATA_MAX_RETENTION_MILLIS_CONFIG, retentionMillis, + MetadataLogConfig.INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG, maxBatchSizeInBytes, + MetadataLogConfig.INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG, maxFetchSizeInBytes, + MetadataLogConfig.INTERNAL_DELETE_DELAY_MILLIS_CONFIG, deleteDelayMillis, ) new MetadataLogConfig(new AbstractConfig(MetadataLogConfig.CONFIG_DEF, config, false)) } diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 4f7498e9158cd..68d3a72d998f1 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -1164,7 +1164,7 @@ class DumpLogSegmentsTest { maxFetchSizeInBytes: Int, deleteDelayMillis: Long ): MetadataLogConfig = { - val config = util.Map.of( + val config: util.Map[String, Any] = util.Map.of( MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, internalLogSegmentBytes, MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, logSegmentMillis, MetadataLogConfig.METADATA_MAX_RETENTION_BYTES_CONFIG, retentionMaxBytes, From d01cfb6a25e47f08d2049a0bcb0623c5af6ad0d8 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 22 May 2025 23:17:41 +0800 Subject: [PATCH 72/74] fix fail test `testFromPropsInvalid` --- core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 080f4f7a6191a..fb9b2939f3220 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -793,6 +793,10 @@ class KafkaConfigTest { case MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case MetadataLogConfig.METADATA_MAX_RETENTION_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case MetadataLogConfig.METADATA_MAX_RETENTION_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG => // no op + case MetadataLogConfig.INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG => // no op + case MetadataLogConfig.INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG => // no op + case MetadataLogConfig.INTERNAL_DELETE_DELAY_MILLIS_CONFIG => // no op case KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG => // ignore string case MetadataLogConfig.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") From 2c569cd6ef380c4038fd3ef8c43a6cce1ade1b7f Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 23 May 2025 20:19:20 +0800 Subject: [PATCH 73/74] addressed by comments --- .../scala/kafka/raft/KafkaMetadataLog.scala | 8 ++-- .../kafka/raft/KafkaMetadataLogTest.scala | 42 +++++++++---------- .../kafka/tools/DumpLogSegmentsTest.scala | 16 ++----- .../apache/kafka/raft/MetadataLogConfig.java | 33 +++++++-------- 4 files changed, 44 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index e21ce821d5595..dc5f3ca11d502 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -73,7 +73,7 @@ final class KafkaMetadataLog private ( case _ => throw new IllegalArgumentException(s"Unhandled read isolation $readIsolation") } - val fetchInfo = log.read(startOffset, config.maxFetchSizeInBytes, isolation, true) + val fetchInfo = log.read(startOffset, config.internalMaxFetchSizeInBytes, isolation, true) new LogFetchInfo( fetchInfo.records, @@ -557,7 +557,7 @@ final class KafkaMetadataLog private ( scheduler.scheduleOnce( "delete-snapshot-files", () => KafkaMetadataLog.deleteSnapshotFiles(log.dir.toPath, expiredSnapshots), - config.deleteDelayMillis + config.internalDeleteDelayMillis ) } } @@ -588,9 +588,9 @@ object KafkaMetadataLog extends Logging { nodeId: Int ): KafkaMetadataLog = { val props = new Properties() - props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.maxBatchSizeInBytes.toString) + props.setProperty(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, config.internalMaxBatchSizeInBytes.toString) if (config.internalSegmentBytes() != null) - props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) + props.setProperty(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, config.internalSegmentBytes().toString) else props.setProperty(TopicConfig.SEGMENT_BYTES_CONFIG, config.logSegmentBytes.toString) props.setProperty(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT.toString) diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index cf7ff56c0a30e..e3ba5b7b2a838 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -479,7 +479,7 @@ final class KafkaMetadataLogTest { assertEquals(log.earliestSnapshotId(), log.latestSnapshotId()) log.close() - mockTime.sleep(config.deleteDelayMillis) + mockTime.sleep(config.internalDeleteDelayMillis) // Assert that the log dir doesn't contain any older snapshots Files .walk(logDir, 1) @@ -650,7 +650,7 @@ final class KafkaMetadataLogTest { assertEquals(greaterSnapshotId, secondLog.latestSnapshotId().get) assertEquals(3 * numberOfRecords, secondLog.startOffset) assertEquals(epoch, secondLog.lastFetchedEpoch) - mockTime.sleep(config.deleteDelayMillis) + mockTime.sleep(config.internalDeleteDelayMillis) // Assert that the log dir doesn't contain any older snapshots Files @@ -694,8 +694,8 @@ final class KafkaMetadataLogTest { DefaultMetadataLogConfig.retentionMaxBytes, DefaultMetadataLogConfig.retentionMillis, maxBatchSizeInBytes, - DefaultMetadataLogConfig.maxFetchSizeInBytes, - DefaultMetadataLogConfig.deleteDelayMillis + DefaultMetadataLogConfig.internalMaxFetchSizeInBytes, + DefaultMetadataLogConfig.internalDeleteDelayMillis ) val log = buildMetadataLog(tempDir, mockTime, config) @@ -913,7 +913,7 @@ final class KafkaMetadataLogTest { 256, 60 * 1000, 512, - DefaultMetadataLogConfig.maxFetchSizeInBytes, + DefaultMetadataLogConfig.internalMaxFetchSizeInBytes, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT ) val log = buildMetadataLog(tempDir, mockTime, config) @@ -949,8 +949,8 @@ final class KafkaMetadataLogTest { 1024, 60 * 1000, 100, - DefaultMetadataLogConfig.maxBatchSizeInBytes, - DefaultMetadataLogConfig.maxFetchSizeInBytes + DefaultMetadataLogConfig.internalMaxBatchSizeInBytes, + DefaultMetadataLogConfig.internalMaxFetchSizeInBytes ) val log = buildMetadataLog(tempDir, mockTime, config) @@ -982,8 +982,8 @@ final class KafkaMetadataLogTest { 10240, 60 * 1000, 100, - DefaultMetadataLogConfig.maxFetchSizeInBytes, - DefaultMetadataLogConfig.deleteDelayMillis + DefaultMetadataLogConfig.internalMaxFetchSizeInBytes, + DefaultMetadataLogConfig.internalDeleteDelayMillis ) val log = buildMetadataLog(tempDir, mockTime, config) @@ -1025,8 +1025,8 @@ final class KafkaMetadataLogTest { 10240, 60 * 1000, 200, - DefaultMetadataLogConfig.maxFetchSizeInBytes, - DefaultMetadataLogConfig.deleteDelayMillis + DefaultMetadataLogConfig.internalMaxFetchSizeInBytes, + DefaultMetadataLogConfig.internalDeleteDelayMillis ) val log = buildMetadataLog(tempDir, mockTime, config) @@ -1163,22 +1163,22 @@ object KafkaMetadataLogTest { } private def createMetadataLogConfig( - internalLogSegmentBytes: Int, - logSegmentMillis: Long, - retentionMaxBytes: Long, - retentionMillis: Long, - maxBatchSizeInBytes: Int, - maxFetchSizeInBytes: Int, - deleteDelayMillis: Long + internalLogSegmentBytes: Int, + logSegmentMillis: Long, + retentionMaxBytes: Long, + retentionMillis: Long, + internalMaxBatchSizeInBytes: Int, + internalMaxFetchSizeInBytes: Int, + internalDeleteDelayMillis: Long ): MetadataLogConfig = { val config: util.Map[String, Any] = util.Map.of( MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, internalLogSegmentBytes, MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, logSegmentMillis, MetadataLogConfig.METADATA_MAX_RETENTION_BYTES_CONFIG, retentionMaxBytes, MetadataLogConfig.METADATA_MAX_RETENTION_MILLIS_CONFIG, retentionMillis, - MetadataLogConfig.INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG, maxBatchSizeInBytes, - MetadataLogConfig.INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG, maxFetchSizeInBytes, - MetadataLogConfig.INTERNAL_DELETE_DELAY_MILLIS_CONFIG, deleteDelayMillis, + MetadataLogConfig.INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG, internalMaxBatchSizeInBytes, + MetadataLogConfig.INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG, internalMaxFetchSizeInBytes, + MetadataLogConfig.INTERNAL_DELETE_DELAY_MILLIS_CONFIG, internalDeleteDelayMillis, ) new MetadataLogConfig(new AbstractConfig(MetadataLogConfig.CONFIG_DEF, config, false)) } diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 951a6c67a6c18..67df4f1a7e6fb 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -44,9 +44,8 @@ import org.apache.kafka.coordinator.share.generated.{ShareSnapshotKey, ShareSnap import org.apache.kafka.coordinator.transaction.generated.{TransactionLogKey, TransactionLogValue} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.MetadataRecordSerde -import org.apache.kafka.raft.{KafkaRaftClient, MetadataLogConfig, VoterSetTest} +import org.apache.kafka.raft.{MetadataLogConfig, VoterSetTest} import org.apache.kafka.server.common.{ApiMessageAndVersion, KRaftVersion, OffsetAndEpoch} -import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde import org.apache.kafka.server.log.remote.storage.{RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteLogSegmentMetadataUpdate, RemoteLogSegmentState, RemotePartitionDeleteMetadata, RemotePartitionDeleteState} import org.apache.kafka.server.storage.log.FetchIsolation @@ -588,10 +587,7 @@ class DumpLogSegmentsTest { 100 * 1024, 10 * 1000, 100 * 1024, - 60 * 1000, - KafkaRaftClient.MAX_BATCH_SIZE_BYTES, - KafkaRaftClient.MAX_FETCH_SIZE_BYTES, - ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT + 60 * 1000 ), 1 ) @@ -1199,19 +1195,13 @@ class DumpLogSegmentsTest { internalLogSegmentBytes: Int, logSegmentMillis: Long, retentionMaxBytes: Long, - retentionMillis: Long, - maxBatchSizeInBytes: Int, - maxFetchSizeInBytes: Int, - deleteDelayMillis: Long + retentionMillis: Long ): MetadataLogConfig = { val config: util.Map[String, Any] = util.Map.of( MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, internalLogSegmentBytes, MetadataLogConfig.METADATA_LOG_SEGMENT_MILLIS_CONFIG, logSegmentMillis, MetadataLogConfig.METADATA_MAX_RETENTION_BYTES_CONFIG, retentionMaxBytes, MetadataLogConfig.METADATA_MAX_RETENTION_MILLIS_CONFIG, retentionMillis, - MetadataLogConfig.INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG, maxBatchSizeInBytes, - MetadataLogConfig.INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG, maxFetchSizeInBytes, - MetadataLogConfig.INTERNAL_DELETE_DELAY_MILLIS_CONFIG, deleteDelayMillis, ) new MetadataLogConfig(new AbstractConfig(MetadataLogConfig.CONFIG_DEF, config, false)) } diff --git a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java index f10fa0d98ef1e..529a9ded415df 100644 --- a/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/MetadataLogConfig.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.server.config.ServerLogConfigs; -import java.util.Objects; import java.util.concurrent.TimeUnit; import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH; @@ -98,18 +97,18 @@ public class MetadataLogConfig { .define(METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, METADATA_MAX_RETENTION_MILLIS_DEFAULT, null, HIGH, METADATA_MAX_RETENTION_MILLIS_DOC) .define(METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, METADATA_MAX_IDLE_INTERVAL_MS_DOC) .defineInternal(INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_METADATA_LOG_SEGMENT_BYTES_DOC) - .defineInternal(INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_MAX_BATCH_SIZE_IN_BYTES_DOC) - .defineInternal(INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG, INT, null, null, LOW, INTERNAL_MAX_FETCH_SIZE_IN_BYTES_DOC) - .defineInternal(INTERNAL_DELETE_DELAY_MILLIS_CONFIG, LONG, null, null, LOW, INTERNAL_DELETE_DELAY_MILLIS_DOC); + .defineInternal(INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG, INT, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, null, LOW, INTERNAL_MAX_BATCH_SIZE_IN_BYTES_DOC) + .defineInternal(INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG, INT, KafkaRaftClient.MAX_FETCH_SIZE_BYTES, null, LOW, INTERNAL_MAX_FETCH_SIZE_IN_BYTES_DOC) + .defineInternal(INTERNAL_DELETE_DELAY_MILLIS_CONFIG, LONG, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, null, LOW, INTERNAL_DELETE_DELAY_MILLIS_DOC); private final int logSegmentBytes; private final Integer internalSegmentBytes; private final long logSegmentMillis; private final long retentionMaxBytes; private final long retentionMillis; - private final int maxBatchSizeInBytes; - private final int maxFetchSizeInBytes; - private final long deleteDelayMillis; + private final int internalMaxBatchSizeInBytes; + private final int internalMaxFetchSizeInBytes; + private final long internalDeleteDelayMillis; public MetadataLogConfig(AbstractConfig config) { this.logSegmentBytes = config.getInt(METADATA_LOG_SEGMENT_BYTES_CONFIG); @@ -117,13 +116,13 @@ public MetadataLogConfig(AbstractConfig config) { this.logSegmentMillis = config.getLong(METADATA_LOG_SEGMENT_MILLIS_CONFIG); this.retentionMaxBytes = config.getLong(METADATA_MAX_RETENTION_BYTES_CONFIG); this.retentionMillis = config.getLong(METADATA_MAX_RETENTION_MILLIS_CONFIG); - this.maxBatchSizeInBytes = Objects.requireNonNullElse(config.getInt(INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG), KafkaRaftClient.MAX_BATCH_SIZE_BYTES); - this.maxFetchSizeInBytes = Objects.requireNonNullElse(config.getInt(INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG), KafkaRaftClient.MAX_FETCH_SIZE_BYTES); - this.deleteDelayMillis = Objects.requireNonNullElse(config.getLong(INTERNAL_DELETE_DELAY_MILLIS_CONFIG), ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT); + this.internalMaxBatchSizeInBytes = config.getInt(INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG); + this.internalMaxFetchSizeInBytes = config.getInt(INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG); + this.internalDeleteDelayMillis = config.getLong(INTERNAL_DELETE_DELAY_MILLIS_CONFIG); } public int logSegmentBytes() { - return Objects.requireNonNullElse(internalSegmentBytes, logSegmentBytes); + return logSegmentBytes; } public Integer internalSegmentBytes() { @@ -142,15 +141,15 @@ public long retentionMillis() { return retentionMillis; } - public int maxBatchSizeInBytes() { - return maxBatchSizeInBytes; + public int internalMaxBatchSizeInBytes() { + return internalMaxBatchSizeInBytes; } - public int maxFetchSizeInBytes() { - return maxFetchSizeInBytes; + public int internalMaxFetchSizeInBytes() { + return internalMaxFetchSizeInBytes; } - public long deleteDelayMillis() { - return deleteDelayMillis; + public long internalDeleteDelayMillis() { + return internalDeleteDelayMillis; } } From 0b4ef666e8397b9315e9d2f7c97a7da5f4d85a71 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 25 May 2025 10:48:24 +0800 Subject: [PATCH 74/74] addressed by comments --- .../kafka/raft/KafkaMetadataLogTest.scala | 20 +++---------- .../server/DynamicConfigChangeTest.scala | 4 +-- .../kafka/metadata/KafkaConfigSchemaTest.java | 29 ++++++++++++++++++- .../storage/internals/log/LogConfig.java | 4 --- .../storage/internals/log/LocalLogTest.java | 3 +- .../kafka/streams/StreamsConfigTest.java | 5 ++-- .../apache/kafka/tools/TopicCommandTest.java | 24 +++++++++++++++ 7 files changed, 63 insertions(+), 26 deletions(-) diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index e3ba5b7b2a838..4b40118e22ad2 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -693,9 +693,7 @@ final class KafkaMetadataLogTest { DefaultMetadataLogConfig.logSegmentMillis, DefaultMetadataLogConfig.retentionMaxBytes, DefaultMetadataLogConfig.retentionMillis, - maxBatchSizeInBytes, - DefaultMetadataLogConfig.internalMaxFetchSizeInBytes, - DefaultMetadataLogConfig.internalDeleteDelayMillis + maxBatchSizeInBytes ) val log = buildMetadataLog(tempDir, mockTime, config) @@ -914,7 +912,6 @@ final class KafkaMetadataLogTest { 60 * 1000, 512, DefaultMetadataLogConfig.internalMaxFetchSizeInBytes, - ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT ) val log = buildMetadataLog(tempDir, mockTime, config) @@ -949,8 +946,6 @@ final class KafkaMetadataLogTest { 1024, 60 * 1000, 100, - DefaultMetadataLogConfig.internalMaxBatchSizeInBytes, - DefaultMetadataLogConfig.internalMaxFetchSizeInBytes ) val log = buildMetadataLog(tempDir, mockTime, config) @@ -982,8 +977,6 @@ final class KafkaMetadataLogTest { 10240, 60 * 1000, 100, - DefaultMetadataLogConfig.internalMaxFetchSizeInBytes, - DefaultMetadataLogConfig.internalDeleteDelayMillis ) val log = buildMetadataLog(tempDir, mockTime, config) @@ -1025,8 +1018,6 @@ final class KafkaMetadataLogTest { 10240, 60 * 1000, 200, - DefaultMetadataLogConfig.internalMaxFetchSizeInBytes, - DefaultMetadataLogConfig.internalDeleteDelayMillis ) val log = buildMetadataLog(tempDir, mockTime, config) @@ -1082,9 +1073,6 @@ object KafkaMetadataLogTest { 10 * 1000, 100 * 1024, 60 * 1000, - KafkaRaftClient.MAX_BATCH_SIZE_BYTES, - KafkaRaftClient.MAX_FETCH_SIZE_BYTES, - ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT ) def buildMetadataLogAndDir( @@ -1167,9 +1155,9 @@ object KafkaMetadataLogTest { logSegmentMillis: Long, retentionMaxBytes: Long, retentionMillis: Long, - internalMaxBatchSizeInBytes: Int, - internalMaxFetchSizeInBytes: Int, - internalDeleteDelayMillis: Long + internalMaxBatchSizeInBytes: Int = KafkaRaftClient.MAX_BATCH_SIZE_BYTES, + internalMaxFetchSizeInBytes: Int = KafkaRaftClient.MAX_FETCH_SIZE_BYTES, + internalDeleteDelayMillis: Long = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT ): MetadataLogConfig = { val config: util.Map[String, Any] = util.Map.of( MetadataLogConfig.INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG, internalLogSegmentBytes, diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index a74e25e6c17b9..519a7d951a381 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -105,11 +105,11 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { assertEquals(oldSegmentSize, logOpt.get.config.segmentSize()) } - val newSegmentSize = 2000 + val newSegmentSize = 2 * 1024 * 1024 val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()) - val op = new AlterConfigOp(new ConfigEntry(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, newSegmentSize.toString), + val op = new AlterConfigOp(new ConfigEntry(TopicConfig.SEGMENT_BYTES_CONFIG, newSegmentSize.toString), OpType.SET) admin.incrementalAlterConfigs(Map(resource -> List(op).asJavaCollection).asJava).all.get } finally { diff --git a/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java b/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java index 27683ce8933d8..92c6e87a0189b 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java @@ -54,7 +54,8 @@ public class KafkaConfigSchemaTest { define("abc", ConfigDef.Type.LIST, ConfigDef.Importance.HIGH, "abc doc"). define("def", ConfigDef.Type.LONG, ConfigDef.Importance.HIGH, "def doc"). define("ghi", ConfigDef.Type.BOOLEAN, true, ConfigDef.Importance.HIGH, "ghi doc"). - define("xyz", ConfigDef.Type.PASSWORD, "thedefault", ConfigDef.Importance.HIGH, "xyz doc")); + define("xyz", ConfigDef.Type.PASSWORD, "thedefault", ConfigDef.Importance.HIGH, "xyz doc"). + defineInternal("internal", ConfigDef.Type.STRING, "internalValue", null, ConfigDef.Importance.HIGH, "internal doc")); } public static final Map> SYNONYMS = new HashMap<>(); @@ -167,4 +168,30 @@ public void testResolveEffectiveTopicConfig() { dynamicNodeConfigs, dynamicTopicConfigs)); } + + @Test + public void testResolveEffectiveDynamicInternalTopicConfig() { + Map dynamicTopicConfigs = Map.of( + "ghi", "true", + "internal", "internal,change" + ); + Map expected = Map.of( + "abc", new ConfigEntry("abc", null, + ConfigEntry.ConfigSource.DEFAULT_CONFIG, false, false, List.of(), + ConfigEntry.ConfigType.LIST, "abc doc"), + "def", new ConfigEntry("def", null, + ConfigEntry.ConfigSource.DEFAULT_CONFIG, false, false, List.of(), + ConfigEntry.ConfigType.LONG, "def doc"), + "ghi", new ConfigEntry("ghi", "true", + ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, false, false, List.of(), + ConfigEntry.ConfigType.BOOLEAN, "ghi doc"), + "xyz", new ConfigEntry("xyz", "thedefault", + ConfigEntry.ConfigSource.DEFAULT_CONFIG, true, false, List.of(), + ConfigEntry.ConfigType.PASSWORD, "xyz doc"), + "internal", new ConfigEntry("internal", "internal,change", + ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, false, false, List.of(), + ConfigEntry.ConfigType.STRING, "internal doc") + ); + assertEquals(expected, SCHEMA.resolveEffectiveTopicConfigs(Map.of(), Map.of(), Map.of(), dynamicTopicConfigs)); + } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 3c8b42c6258ad..eb7d0eaa5d4d5 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -455,10 +455,6 @@ public static List nonInternalConfigNames() { .sorted().toList(); } - public static Optional serverConfigName(String configName) { - return CONFIG.serverConfigName(configName); - } - public static Map configKeys() { return Collections.unmodifiableMap(CONFIG.configKeys()); } diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java index d58c520af0972..86ff278012e87 100644 --- a/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LocalLogTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -197,7 +198,7 @@ public void testUpdateConfig() { assertEquals(oldConfig, log.config()); Properties props = new Properties(); - props.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); + props.put(TopicConfig.SEGMENT_BYTES_CONFIG, oldConfig.segmentSize() + 1); LogConfig newConfig = new LogConfig(props); log.updateConfig(newConfig); assertEquals(newConfig, log.config()); diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index 2db5cc682d9f2..f1ee8df371dbe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; @@ -247,7 +248,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { props.put(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, 99_999L); props.put(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, 7L); props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "dummy:host"); - props.put(StreamsConfig.topicPrefix("internal.segment.bytes"), 100); + props.put(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG), 1024 * 1024); final StreamsConfig streamsConfig = new StreamsConfig(props); final Map returnedProps = streamsConfig.getMainConsumerConfigs(groupId, clientId, threadIdx); @@ -262,7 +263,7 @@ public void consumerConfigMustContainStreamPartitionAssignorConfig() { ); assertEquals(7L, returnedProps.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG)); assertEquals("dummy:host", returnedProps.get(StreamsConfig.APPLICATION_SERVER_CONFIG)); - assertEquals(100, returnedProps.get(StreamsConfig.topicPrefix("internal.segment.bytes"))); + assertEquals(1024 * 1024, returnedProps.get(StreamsConfig.topicPrefix(TopicConfig.SEGMENT_BYTES_CONFIG))); } @Test diff --git a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java index deb57ae8cbc90..dcfe861b7579a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClientTestUtils; import org.apache.kafka.clients.admin.Config; +import org.apache.kafka.clients.admin.ConfigEntry; import org.apache.kafka.clients.admin.CreatePartitionsResult; import org.apache.kafka.clients.admin.CreateTopicsResult; import org.apache.kafka.clients.admin.DeleteTopicsOptions; @@ -56,6 +57,7 @@ import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.metadata.LeaderAndIsr; +import org.apache.kafka.storage.internals.log.LogConfig; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Assertions; @@ -82,6 +84,7 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -1400,6 +1403,27 @@ public void testCreateWithTopicNameCollision(ClusterInstance clusterInstance) th } } + @ClusterTest + public void testCreateWithInternalConfig(ClusterInstance cluster) throws InterruptedException, ExecutionException { + String internalConfigTopicName = TestUtils.randomString(10); + String testTopicName = TestUtils.randomString(10); + + try (Admin adminClient = cluster.admin()) { + CreateTopicsResult internalResult = adminClient.createTopics(List.of(new NewTopic(internalConfigTopicName, defaultNumPartitions, defaultReplicationFactor).configs( + Map.of(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, "1000") + ))); + + ConfigEntry internalConfigEntry = internalResult.config(internalConfigTopicName).get().get(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG); + assertNotNull(internalConfigEntry, "Internal config entry should not be null"); + assertEquals("1000", internalConfigEntry.value()); + + CreateTopicsResult nonInternalResult = adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); + + ConfigEntry nonInternalConfigEntry = nonInternalResult.config(testTopicName).get().get(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG); + assertNull(nonInternalConfigEntry, "Non-internal config entry should be null"); + } + } + private void checkReplicaDistribution(Map> assignment, Map brokerRackMapping, Integer numBrokers,