mirror of https://github.com/apache/kafka.git
				
				
				
			KAFKA-15599 Move MetadataLogConfig to raft module (#19246)
Rewrite the class in Java and move it to the raft module. Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
		
							parent
							
								
									1c582a4a35
								
							
						
					
					
						commit
						121ec2a662
					
				|  | @ -488,6 +488,7 @@ | ||||||
|     <allow pkg="org.apache.kafka.common.requests" /> |     <allow pkg="org.apache.kafka.common.requests" /> | ||||||
|     <allow pkg="org.apache.kafka.server.common" /> |     <allow pkg="org.apache.kafka.server.common" /> | ||||||
|     <allow pkg="org.apache.kafka.server.common.serialization" /> |     <allow pkg="org.apache.kafka.server.common.serialization" /> | ||||||
|  |     <allow pkg="org.apache.kafka.server.config" /> | ||||||
|     <allow pkg="org.apache.kafka.server.fault"/> |     <allow pkg="org.apache.kafka.server.fault"/> | ||||||
|     <allow pkg="org.apache.kafka.server.util" /> |     <allow pkg="org.apache.kafka.server.util" /> | ||||||
|     <allow pkg="org.apache.kafka.test"/> |     <allow pkg="org.apache.kafka.test"/> | ||||||
|  |  | ||||||
|  | @ -1,48 +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.raft |  | ||||||
| 
 |  | ||||||
| import org.apache.kafka.server.config.ServerLogConfigs |  | ||||||
| import kafka.server.KafkaConfig |  | ||||||
| 
 |  | ||||||
| final case class MetadataLogConfig( |  | ||||||
|   logSegmentBytes: Int, |  | ||||||
|   logSegmentMinBytes: Int, |  | ||||||
|   logSegmentMillis: Long, |  | ||||||
|   retentionMaxBytes: Long, |  | ||||||
|   retentionMillis: Long, |  | ||||||
|   maxBatchSizeInBytes: Int, |  | ||||||
|   maxFetchSizeInBytes: Int, |  | ||||||
|   fileDeleteDelayMs: Long, |  | ||||||
|   nodeId: Int |  | ||||||
| ) |  | ||||||
| 
 |  | ||||||
| object MetadataLogConfig { |  | ||||||
|   def apply(config: KafkaConfig, maxBatchSizeInBytes: Int, maxFetchSizeInBytes: Int): MetadataLogConfig = { |  | ||||||
|     new MetadataLogConfig( |  | ||||||
|       config.metadataLogSegmentBytes, |  | ||||||
|       config.metadataLogSegmentMinBytes, |  | ||||||
|       config.metadataLogSegmentMillis, |  | ||||||
|       config.metadataRetentionBytes, |  | ||||||
|       config.metadataRetentionMillis, |  | ||||||
|       maxBatchSizeInBytes, |  | ||||||
|       maxFetchSizeInBytes, |  | ||||||
|       ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, |  | ||||||
|       config.metadataNodeIDConfig |  | ||||||
|     ) |  | ||||||
|   } |  | ||||||
| } |  | ||||||
|  | @ -28,7 +28,7 @@ import org.apache.kafka.common.errors.CorruptRecordException | ||||||
| import org.apache.kafka.common.record.{MemoryRecords, Records} | import org.apache.kafka.common.record.{MemoryRecords, Records} | ||||||
| import org.apache.kafka.common.utils.{Time, Utils} | import org.apache.kafka.common.utils.{Time, Utils} | ||||||
| import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} | import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} | ||||||
| import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, 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.{KRaftConfigs, ServerLogConfigs} | import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} | ||||||
| import org.apache.kafka.server.storage.log.FetchIsolation | import org.apache.kafka.server.storage.log.FetchIsolation | ||||||
| import org.apache.kafka.server.util.Scheduler | import org.apache.kafka.server.util.Scheduler | ||||||
|  | @ -554,7 +554,7 @@ final class KafkaMetadataLog private ( | ||||||
|       scheduler.scheduleOnce( |       scheduler.scheduleOnce( | ||||||
|         "delete-snapshot-files", |         "delete-snapshot-files", | ||||||
|         () => KafkaMetadataLog.deleteSnapshotFiles(log.dir.toPath, expiredSnapshots, this), |         () => KafkaMetadataLog.deleteSnapshotFiles(log.dir.toPath, expiredSnapshots, this), | ||||||
|         config.fileDeleteDelayMs |         config.deleteDelayMillis | ||||||
|       ) |       ) | ||||||
|     } |     } | ||||||
|   } |   } | ||||||
|  |  | ||||||
|  | @ -41,10 +41,11 @@ import org.apache.kafka.common.requests.RequestHeader | ||||||
| import org.apache.kafka.common.security.JaasContext | import org.apache.kafka.common.security.JaasContext | ||||||
| import org.apache.kafka.common.security.auth.SecurityProtocol | import org.apache.kafka.common.security.auth.SecurityProtocol | ||||||
| import org.apache.kafka.common.utils.{LogContext, Time, Utils} | import org.apache.kafka.common.utils.{LogContext, Time, Utils} | ||||||
| import org.apache.kafka.raft.{ExternalKRaftMetrics, Endpoints, FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, QuorumConfig, RaftClient, ReplicatedLog, TimingWheelExpirationService} | import org.apache.kafka.raft.{Endpoints, ExternalKRaftMetrics, FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, MetadataLogConfig, QuorumConfig, RaftClient, ReplicatedLog, TimingWheelExpirationService} | ||||||
| import org.apache.kafka.server.ProcessRole | import org.apache.kafka.server.ProcessRole | ||||||
| import org.apache.kafka.server.common.Feature | import org.apache.kafka.server.common.Feature | ||||||
| import org.apache.kafka.server.common.serialization.RecordSerde | import org.apache.kafka.server.common.serialization.RecordSerde | ||||||
|  | import org.apache.kafka.server.config.ServerLogConfigs | ||||||
| import org.apache.kafka.server.util.{FileLock, KafkaScheduler} | import org.apache.kafka.server.util.{FileLock, KafkaScheduler} | ||||||
| import org.apache.kafka.server.fault.FaultHandler | import org.apache.kafka.server.fault.FaultHandler | ||||||
| import org.apache.kafka.server.util.timer.SystemTimer | import org.apache.kafka.server.util.timer.SystemTimer | ||||||
|  | @ -230,7 +231,15 @@ class KafkaRaftManager[T]( | ||||||
|       dataDir, |       dataDir, | ||||||
|       time, |       time, | ||||||
|       scheduler, |       scheduler, | ||||||
|       config = MetadataLogConfig(config, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES) |       config = new MetadataLogConfig(config.metadataLogSegmentBytes, | ||||||
|  |         config.metadataLogSegmentMinBytes, | ||||||
|  |         config.metadataLogSegmentMillis, | ||||||
|  |         config.metadataRetentionBytes, | ||||||
|  |         config.metadataRetentionMillis, | ||||||
|  |         KafkaRaftClient.MAX_BATCH_SIZE_BYTES, | ||||||
|  |         KafkaRaftClient.MAX_FETCH_SIZE_BYTES, | ||||||
|  |         ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, | ||||||
|  |         config.metadataNodeIDConfig) | ||||||
|     ) |     ) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|  |  | ||||||
|  | @ -27,7 +27,7 @@ import org.apache.kafka.common.record.ArbitraryMemoryRecords | ||||||
| import org.apache.kafka.common.record.InvalidMemoryRecordsProvider | import org.apache.kafka.common.record.InvalidMemoryRecordsProvider | ||||||
| import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} | import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} | ||||||
| import org.apache.kafka.common.utils.Utils | import org.apache.kafka.common.utils.Utils | ||||||
| import org.apache.kafka.raft._ | import org.apache.kafka.raft.{KafkaRaftClient, LogAppendInfo, LogOffsetMetadata, MetadataLogConfig, OffsetAndEpoch, QuorumConfig, ReplicatedLog, SegmentPosition, ValidOffsetAndEpoch} | ||||||
| import org.apache.kafka.raft.internals.BatchBuilder | import org.apache.kafka.raft.internals.BatchBuilder | ||||||
| import org.apache.kafka.server.common.serialization.RecordSerde | import org.apache.kafka.server.common.serialization.RecordSerde | ||||||
| import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} | import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} | ||||||
|  | @ -80,13 +80,31 @@ final class KafkaMetadataLogTest { | ||||||
|     props.put(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024)) |     props.put(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024)) | ||||||
|     assertThrows(classOf[InvalidConfigurationException], () => { |     assertThrows(classOf[InvalidConfigurationException], () => { | ||||||
|       val kafkaConfig = KafkaConfig.fromProps(props) |       val kafkaConfig = KafkaConfig.fromProps(props) | ||||||
|       val metadataConfig = MetadataLogConfig(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES) |       val metadataConfig = new MetadataLogConfig( | ||||||
|  |         kafkaConfig.metadataLogSegmentBytes, | ||||||
|  |         kafkaConfig.metadataLogSegmentMinBytes, | ||||||
|  |         kafkaConfig.metadataLogSegmentMillis, | ||||||
|  |         kafkaConfig.metadataRetentionBytes, | ||||||
|  |         kafkaConfig.metadataRetentionMillis, | ||||||
|  |         KafkaRaftClient.MAX_BATCH_SIZE_BYTES, | ||||||
|  |         KafkaRaftClient.MAX_FETCH_SIZE_BYTES, | ||||||
|  |         ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, | ||||||
|  |         kafkaConfig.metadataNodeIDConfig) | ||||||
|       buildMetadataLog(tempDir, mockTime, metadataConfig) |       buildMetadataLog(tempDir, mockTime, metadataConfig) | ||||||
|     }) |     }) | ||||||
| 
 | 
 | ||||||
|     props.put(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) |     props.put(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240)) | ||||||
|     val kafkaConfig = KafkaConfig.fromProps(props) |     val kafkaConfig = KafkaConfig.fromProps(props) | ||||||
|     val metadataConfig = MetadataLogConfig(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES) |     val metadataConfig = new MetadataLogConfig( | ||||||
|  |       kafkaConfig.metadataLogSegmentBytes, | ||||||
|  |       kafkaConfig.metadataLogSegmentMinBytes, | ||||||
|  |       kafkaConfig.metadataLogSegmentMillis, | ||||||
|  |       kafkaConfig.metadataRetentionBytes, | ||||||
|  |       kafkaConfig.metadataRetentionMillis, | ||||||
|  |       KafkaRaftClient.MAX_BATCH_SIZE_BYTES, | ||||||
|  |       KafkaRaftClient.MAX_FETCH_SIZE_BYTES, | ||||||
|  |       ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, | ||||||
|  |       kafkaConfig.metadataNodeIDConfig) | ||||||
|     buildMetadataLog(tempDir, mockTime, metadataConfig) |     buildMetadataLog(tempDir, mockTime, metadataConfig) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|  | @ -129,8 +147,8 @@ final class KafkaMetadataLogTest { | ||||||
|   def testEmptyAppendNotAllowed(): Unit = { |   def testEmptyAppendNotAllowed(): Unit = { | ||||||
|     val log = buildMetadataLog(tempDir, mockTime) |     val log = buildMetadataLog(tempDir, mockTime) | ||||||
| 
 | 
 | ||||||
|     assertThrows(classOf[IllegalArgumentException], () => log.appendAsFollower(MemoryRecords.EMPTY, 1)); |     assertThrows(classOf[IllegalArgumentException], () => log.appendAsFollower(MemoryRecords.EMPTY, 1)) | ||||||
|     assertThrows(classOf[IllegalArgumentException], () => log.appendAsLeader(MemoryRecords.EMPTY, 1)); |     assertThrows(classOf[IllegalArgumentException], () => log.appendAsLeader(MemoryRecords.EMPTY, 1)) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   @ParameterizedTest |   @ParameterizedTest | ||||||
|  | @ -140,7 +158,7 @@ final class KafkaMetadataLogTest { | ||||||
|     val previousEndOffset = log.endOffset().offset() |     val previousEndOffset = log.endOffset().offset() | ||||||
| 
 | 
 | ||||||
|     val action: Executable = () => log.appendAsFollower(records, Int.MaxValue) |     val action: Executable = () => log.appendAsFollower(records, Int.MaxValue) | ||||||
|     if (expectedException.isPresent()) { |     if (expectedException.isPresent) { | ||||||
|       assertThrows(expectedException.get, action) |       assertThrows(expectedException.get, action) | ||||||
|     } else { |     } else { | ||||||
|       assertThrows(classOf[CorruptRecordException], action) |       assertThrows(classOf[CorruptRecordException], action) | ||||||
|  | @ -478,7 +496,7 @@ final class KafkaMetadataLogTest { | ||||||
|     assertEquals(log.earliestSnapshotId(), log.latestSnapshotId()) |     assertEquals(log.earliestSnapshotId(), log.latestSnapshotId()) | ||||||
|     log.close() |     log.close() | ||||||
| 
 | 
 | ||||||
|     mockTime.sleep(config.fileDeleteDelayMs) |     mockTime.sleep(config.deleteDelayMillis) | ||||||
|     // Assert that the log dir doesn't contain any older snapshots |     // Assert that the log dir doesn't contain any older snapshots | ||||||
|     Files |     Files | ||||||
|       .walk(logDir, 1) |       .walk(logDir, 1) | ||||||
|  | @ -649,7 +667,7 @@ final class KafkaMetadataLogTest { | ||||||
|     assertEquals(greaterSnapshotId, secondLog.latestSnapshotId().get) |     assertEquals(greaterSnapshotId, secondLog.latestSnapshotId().get) | ||||||
|     assertEquals(3 * numberOfRecords, secondLog.startOffset) |     assertEquals(3 * numberOfRecords, secondLog.startOffset) | ||||||
|     assertEquals(epoch, secondLog.lastFetchedEpoch) |     assertEquals(epoch, secondLog.lastFetchedEpoch) | ||||||
|     mockTime.sleep(config.fileDeleteDelayMs) |     mockTime.sleep(config.deleteDelayMillis) | ||||||
| 
 | 
 | ||||||
|     // Assert that the log dir doesn't contain any older snapshots |     // Assert that the log dir doesn't contain any older snapshots | ||||||
|     Files |     Files | ||||||
|  | @ -687,7 +705,18 @@ final class KafkaMetadataLogTest { | ||||||
|     val leaderEpoch = 5 |     val leaderEpoch = 5 | ||||||
|     val maxBatchSizeInBytes = 16384 |     val maxBatchSizeInBytes = 16384 | ||||||
|     val recordSize = 64 |     val recordSize = 64 | ||||||
|     val log = buildMetadataLog(tempDir, mockTime, DefaultMetadataLogConfig.copy(maxBatchSizeInBytes = maxBatchSizeInBytes)) |     val config = new MetadataLogConfig( | ||||||
|  |       DefaultMetadataLogConfig.logSegmentBytes, | ||||||
|  |       DefaultMetadataLogConfig.logSegmentMinBytes, | ||||||
|  |       DefaultMetadataLogConfig.logSegmentMillis, | ||||||
|  |       DefaultMetadataLogConfig.retentionMaxBytes, | ||||||
|  |       DefaultMetadataLogConfig.retentionMillis, | ||||||
|  |       maxBatchSizeInBytes, | ||||||
|  |       DefaultMetadataLogConfig.maxFetchSizeInBytes, | ||||||
|  |       DefaultMetadataLogConfig.deleteDelayMillis, | ||||||
|  |       DefaultMetadataLogConfig.nodeId | ||||||
|  |     ) | ||||||
|  |     val log = buildMetadataLog(tempDir, mockTime, config) | ||||||
| 
 | 
 | ||||||
|     val oversizeBatch = buildFullBatch(leaderEpoch, recordSize, maxBatchSizeInBytes + recordSize) |     val oversizeBatch = buildFullBatch(leaderEpoch, recordSize, maxBatchSizeInBytes + recordSize) | ||||||
|     assertThrows(classOf[RecordTooLargeException], () => { |     assertThrows(classOf[RecordTooLargeException], () => { | ||||||
|  | @ -897,18 +926,17 @@ final class KafkaMetadataLogTest { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testAdvanceLogStartOffsetAfterCleaning(): Unit = { |   def testAdvanceLogStartOffsetAfterCleaning(): Unit = { | ||||||
|     val config = MetadataLogConfig( |     val config = new MetadataLogConfig( | ||||||
|       logSegmentBytes = 512, |       512, | ||||||
|       logSegmentMinBytes = 512, |       512, | ||||||
|       logSegmentMillis = 10 * 1000, |       10 * 1000, | ||||||
|       retentionMaxBytes = 256, |       256, | ||||||
|       retentionMillis = 60 * 1000, |       60 * 1000, | ||||||
|       maxBatchSizeInBytes = 512, |       512, | ||||||
|       maxFetchSizeInBytes = DefaultMetadataLogConfig.maxFetchSizeInBytes, |       DefaultMetadataLogConfig.maxFetchSizeInBytes, | ||||||
|       fileDeleteDelayMs = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, |       ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, | ||||||
|       nodeId = 1 |       1 | ||||||
|     ) |     ) | ||||||
|     config.copy() |  | ||||||
|     val log = buildMetadataLog(tempDir, mockTime, config) |     val log = buildMetadataLog(tempDir, mockTime, config) | ||||||
| 
 | 
 | ||||||
|     // Generate some segments |     // Generate some segments | ||||||
|  | @ -936,13 +964,16 @@ final class KafkaMetadataLogTest { | ||||||
|   @Test |   @Test | ||||||
|   def testDeleteSnapshots(): Unit = { |   def testDeleteSnapshots(): Unit = { | ||||||
|     // Generate some logs and a few snapshots, set retention low and verify that cleaning occurs |     // Generate some logs and a few snapshots, set retention low and verify that cleaning occurs | ||||||
|     val config = DefaultMetadataLogConfig.copy( |     val config = new MetadataLogConfig( | ||||||
|       logSegmentBytes = 1024, |       1024, | ||||||
|       logSegmentMinBytes = 1024, |       1024, | ||||||
|       logSegmentMillis = 10 * 1000, |       10 * 1000, | ||||||
|       retentionMaxBytes = 1024, |       1024, | ||||||
|       retentionMillis = 60 * 1000, |       60 * 1000, | ||||||
|       maxBatchSizeInBytes = 100 |       100, | ||||||
|  |       DefaultMetadataLogConfig.maxBatchSizeInBytes, | ||||||
|  |       DefaultMetadataLogConfig.maxFetchSizeInBytes, | ||||||
|  |       DefaultMetadataLogConfig.nodeId | ||||||
|     ) |     ) | ||||||
|     val log = buildMetadataLog(tempDir, mockTime, config) |     val log = buildMetadataLog(tempDir, mockTime, config) | ||||||
| 
 | 
 | ||||||
|  | @ -968,13 +999,16 @@ final class KafkaMetadataLogTest { | ||||||
|   @Test |   @Test | ||||||
|   def testSoftRetentionLimit(): Unit = { |   def testSoftRetentionLimit(): Unit = { | ||||||
|     // Set retention equal to the segment size and generate slightly more than one segment of logs |     // Set retention equal to the segment size and generate slightly more than one segment of logs | ||||||
|     val config = DefaultMetadataLogConfig.copy( |     val config = new MetadataLogConfig( | ||||||
|       logSegmentBytes = 10240, |       10240, | ||||||
|       logSegmentMinBytes = 10240, |       10240, | ||||||
|       logSegmentMillis = 10 * 1000, |       10 * 1000, | ||||||
|       retentionMaxBytes = 10240, |       10240, | ||||||
|       retentionMillis = 60 * 1000, |       60 * 1000, | ||||||
|       maxBatchSizeInBytes = 100 |       100, | ||||||
|  |       DefaultMetadataLogConfig.maxFetchSizeInBytes, | ||||||
|  |       DefaultMetadataLogConfig.deleteDelayMillis, | ||||||
|  |       DefaultMetadataLogConfig.nodeId | ||||||
|     ) |     ) | ||||||
|     val log = buildMetadataLog(tempDir, mockTime, config) |     val log = buildMetadataLog(tempDir, mockTime, config) | ||||||
| 
 | 
 | ||||||
|  | @ -1010,13 +1044,16 @@ final class KafkaMetadataLogTest { | ||||||
| 
 | 
 | ||||||
|   @Test |   @Test | ||||||
|   def testSegmentsLessThanLatestSnapshot(): Unit = { |   def testSegmentsLessThanLatestSnapshot(): Unit = { | ||||||
|     val config = DefaultMetadataLogConfig.copy( |     val config = new MetadataLogConfig( | ||||||
|       logSegmentBytes = 10240, |       10240, | ||||||
|       logSegmentMinBytes = 10240, |       10240, | ||||||
|       logSegmentMillis = 10 * 1000, |       10 * 1000, | ||||||
|       retentionMaxBytes = 10240, |       10240, | ||||||
|       retentionMillis = 60 * 1000, |       60 * 1000, | ||||||
|       maxBatchSizeInBytes = 200 |       200, | ||||||
|  |       DefaultMetadataLogConfig.maxFetchSizeInBytes, | ||||||
|  |       DefaultMetadataLogConfig.deleteDelayMillis, | ||||||
|  |       DefaultMetadataLogConfig.nodeId | ||||||
|     ) |     ) | ||||||
|     val log = buildMetadataLog(tempDir, mockTime, config) |     val log = buildMetadataLog(tempDir, mockTime, config) | ||||||
| 
 | 
 | ||||||
|  | @ -1067,16 +1104,16 @@ object KafkaMetadataLogTest { | ||||||
|     override def read(input: protocol.Readable, size: Int): Array[Byte] = input.readArray(size) |     override def read(input: protocol.Readable, size: Int): Array[Byte] = input.readArray(size) | ||||||
|   } |   } | ||||||
| 
 | 
 | ||||||
|   val DefaultMetadataLogConfig = MetadataLogConfig( |   val DefaultMetadataLogConfig = new MetadataLogConfig( | ||||||
|     logSegmentBytes = 100 * 1024, |     100 * 1024, | ||||||
|     logSegmentMinBytes = 100 * 1024, |     100 * 1024, | ||||||
|     logSegmentMillis = 10 * 1000, |     10 * 1000, | ||||||
|     retentionMaxBytes = 100 * 1024, |     100 * 1024, | ||||||
|     retentionMillis = 60 * 1000, |     60 * 1000, | ||||||
|     maxBatchSizeInBytes = KafkaRaftClient.MAX_BATCH_SIZE_BYTES, |     KafkaRaftClient.MAX_BATCH_SIZE_BYTES, | ||||||
|     maxFetchSizeInBytes = KafkaRaftClient.MAX_FETCH_SIZE_BYTES, |     KafkaRaftClient.MAX_FETCH_SIZE_BYTES, | ||||||
|     fileDeleteDelayMs = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, |     ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, | ||||||
|     nodeId = 1 |     1 | ||||||
|   ) |   ) | ||||||
| 
 | 
 | ||||||
|   def buildMetadataLogAndDir( |   def buildMetadataLogAndDir( | ||||||
|  |  | ||||||
|  | @ -25,7 +25,7 @@ import java.util.Optional | ||||||
| import java.util.Properties | import java.util.Properties | ||||||
| import java.util.stream.IntStream | import java.util.stream.IntStream | ||||||
| import kafka.log.LogTestUtils | import kafka.log.LogTestUtils | ||||||
| import kafka.raft.{KafkaMetadataLog, MetadataLogConfig} | import kafka.raft.KafkaMetadataLog | ||||||
| import kafka.server.KafkaRaftServer | import kafka.server.KafkaRaftServer | ||||||
| import kafka.tools.DumpLogSegments.{OffsetsMessageParser, ShareGroupStateMessageParser, TimeIndexDumpErrors, TransactionLogMessageParser} | import kafka.tools.DumpLogSegments.{OffsetsMessageParser, ShareGroupStateMessageParser, TimeIndexDumpErrors, TransactionLogMessageParser} | ||||||
| import kafka.utils.TestUtils | import kafka.utils.TestUtils | ||||||
|  | @ -43,7 +43,7 @@ import org.apache.kafka.coordinator.share.generated.{ShareSnapshotKey, ShareSnap | ||||||
| import org.apache.kafka.coordinator.transaction.generated.{TransactionLogKey, TransactionLogValue} | import org.apache.kafka.coordinator.transaction.generated.{TransactionLogKey, TransactionLogValue} | ||||||
| import org.apache.kafka.coordinator.transaction.TransactionLogConfig | import org.apache.kafka.coordinator.transaction.TransactionLogConfig | ||||||
| import org.apache.kafka.metadata.MetadataRecordSerde | import org.apache.kafka.metadata.MetadataRecordSerde | ||||||
| import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch, VoterSetTest} | import org.apache.kafka.raft.{KafkaRaftClient, MetadataLogConfig, OffsetAndEpoch, VoterSetTest} | ||||||
| import org.apache.kafka.server.common.{ApiMessageAndVersion, KRaftVersion} | import org.apache.kafka.server.common.{ApiMessageAndVersion, KRaftVersion} | ||||||
| import org.apache.kafka.server.config.ServerLogConfigs | 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.metadata.storage.serialization.RemoteLogMetadataSerde | ||||||
|  | @ -544,16 +544,16 @@ class DumpLogSegmentsTest { | ||||||
|       logDir, |       logDir, | ||||||
|       time, |       time, | ||||||
|       time.scheduler, |       time.scheduler, | ||||||
|       MetadataLogConfig( |       new MetadataLogConfig( | ||||||
|         logSegmentBytes = 100 * 1024, |         100 * 1024, | ||||||
|         logSegmentMinBytes = 100 * 1024, |         100 * 1024, | ||||||
|         logSegmentMillis = 10 * 1000, |         10 * 1000, | ||||||
|         retentionMaxBytes = 100 * 1024, |         100 * 1024, | ||||||
|         retentionMillis = 60 * 1000, |         60 * 1000, | ||||||
|         maxBatchSizeInBytes = KafkaRaftClient.MAX_BATCH_SIZE_BYTES, |         KafkaRaftClient.MAX_BATCH_SIZE_BYTES, | ||||||
|         maxFetchSizeInBytes = KafkaRaftClient.MAX_FETCH_SIZE_BYTES, |         KafkaRaftClient.MAX_FETCH_SIZE_BYTES, | ||||||
|         fileDeleteDelayMs = ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, |         ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, | ||||||
|         nodeId = 1 |         1 | ||||||
|       ) |       ) | ||||||
|     ) |     ) | ||||||
| 
 | 
 | ||||||
|  |  | ||||||
|  | @ -0,0 +1,40 @@ | ||||||
|  | /* | ||||||
|  |  * 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; | ||||||
|  | 
 | ||||||
|  | /** | ||||||
|  |  * 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 | ||||||
|  |  * @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 | ||||||
|  |  * @param nodeId The node id | ||||||
|  |  */ | ||||||
|  | public record MetadataLogConfig(int logSegmentBytes, | ||||||
|  |                                 int logSegmentMinBytes, | ||||||
|  |                                 long logSegmentMillis, | ||||||
|  |                                 long retentionMaxBytes, | ||||||
|  |                                 long retentionMillis, | ||||||
|  |                                 int maxBatchSizeInBytes, | ||||||
|  |                                 int maxFetchSizeInBytes, | ||||||
|  |                                 long deleteDelayMillis, | ||||||
|  |                                 int nodeId) { | ||||||
|  | } | ||||||
		Loading…
	
		Reference in New Issue