@@ -27,7 +27,7 @@ import org.apache.kafka.common.record.ArbitraryMemoryRecords
27
27
import org .apache .kafka .common .record .InvalidMemoryRecordsProvider
28
28
import org .apache .kafka .common .record .{MemoryRecords , SimpleRecord }
29
29
import org .apache .kafka .common .utils .Utils
30
- import org .apache .kafka .raft ._
30
+ import org .apache .kafka .raft .{ KafkaRaftClient , LogAppendInfo , LogOffsetMetadata , MetadataLogConfig , OffsetAndEpoch , QuorumConfig , ReplicatedLog , SegmentPosition , ValidOffsetAndEpoch }
31
31
import org .apache .kafka .raft .internals .BatchBuilder
32
32
import org .apache .kafka .server .common .serialization .RecordSerde
33
33
import org .apache .kafka .server .config .{KRaftConfigs , ServerLogConfigs }
@@ -80,13 +80,31 @@ final class KafkaMetadataLogTest {
80
80
props.put(KRaftConfigs .METADATA_LOG_SEGMENT_MILLIS_CONFIG , Int .box(10 * 1024 ))
81
81
assertThrows(classOf [InvalidConfigurationException ], () => {
82
82
val kafkaConfig = KafkaConfig .fromProps(props)
83
- val metadataConfig = MetadataLogConfig (kafkaConfig, KafkaRaftClient .MAX_BATCH_SIZE_BYTES , KafkaRaftClient .MAX_FETCH_SIZE_BYTES )
83
+ val metadataConfig = new MetadataLogConfig (
84
+ kafkaConfig.metadataLogSegmentBytes,
85
+ kafkaConfig.metadataLogSegmentMinBytes,
86
+ kafkaConfig.metadataLogSegmentMillis,
87
+ kafkaConfig.metadataRetentionBytes,
88
+ kafkaConfig.metadataRetentionMillis,
89
+ KafkaRaftClient .MAX_BATCH_SIZE_BYTES ,
90
+ KafkaRaftClient .MAX_FETCH_SIZE_BYTES ,
91
+ ServerLogConfigs .LOG_DELETE_DELAY_MS_DEFAULT ,
92
+ kafkaConfig.metadataNodeIDConfig)
84
93
buildMetadataLog(tempDir, mockTime, metadataConfig)
85
94
})
86
95
87
96
props.put(KRaftConfigs .METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG , Int .box(10240 ))
88
97
val kafkaConfig = KafkaConfig .fromProps(props)
89
- val metadataConfig = MetadataLogConfig (kafkaConfig, KafkaRaftClient .MAX_BATCH_SIZE_BYTES , KafkaRaftClient .MAX_FETCH_SIZE_BYTES )
98
+ val metadataConfig = new MetadataLogConfig (
99
+ kafkaConfig.metadataLogSegmentBytes,
100
+ kafkaConfig.metadataLogSegmentMinBytes,
101
+ kafkaConfig.metadataLogSegmentMillis,
102
+ kafkaConfig.metadataRetentionBytes,
103
+ kafkaConfig.metadataRetentionMillis,
104
+ KafkaRaftClient .MAX_BATCH_SIZE_BYTES ,
105
+ KafkaRaftClient .MAX_FETCH_SIZE_BYTES ,
106
+ ServerLogConfigs .LOG_DELETE_DELAY_MS_DEFAULT ,
107
+ kafkaConfig.metadataNodeIDConfig)
90
108
buildMetadataLog(tempDir, mockTime, metadataConfig)
91
109
}
92
110
@@ -129,8 +147,8 @@ final class KafkaMetadataLogTest {
129
147
def testEmptyAppendNotAllowed (): Unit = {
130
148
val log = buildMetadataLog(tempDir, mockTime)
131
149
132
- assertThrows(classOf [IllegalArgumentException ], () => log.appendAsFollower(MemoryRecords .EMPTY , 1 ));
133
- assertThrows(classOf [IllegalArgumentException ], () => log.appendAsLeader(MemoryRecords .EMPTY , 1 ));
150
+ assertThrows(classOf [IllegalArgumentException ], () => log.appendAsFollower(MemoryRecords .EMPTY , 1 ))
151
+ assertThrows(classOf [IllegalArgumentException ], () => log.appendAsLeader(MemoryRecords .EMPTY , 1 ))
134
152
}
135
153
136
154
@ ParameterizedTest
@@ -140,7 +158,7 @@ final class KafkaMetadataLogTest {
140
158
val previousEndOffset = log.endOffset().offset()
141
159
142
160
val action : Executable = () => log.appendAsFollower(records, Int .MaxValue )
143
- if (expectedException.isPresent() ) {
161
+ if (expectedException.isPresent) {
144
162
assertThrows(expectedException.get, action)
145
163
} else {
146
164
assertThrows(classOf [CorruptRecordException ], action)
@@ -478,7 +496,7 @@ final class KafkaMetadataLogTest {
478
496
assertEquals(log.earliestSnapshotId(), log.latestSnapshotId())
479
497
log.close()
480
498
481
- mockTime.sleep(config.fileDeleteDelayMs )
499
+ mockTime.sleep(config.deleteDelayMillis )
482
500
// Assert that the log dir doesn't contain any older snapshots
483
501
Files
484
502
.walk(logDir, 1 )
@@ -649,7 +667,7 @@ final class KafkaMetadataLogTest {
649
667
assertEquals(greaterSnapshotId, secondLog.latestSnapshotId().get)
650
668
assertEquals(3 * numberOfRecords, secondLog.startOffset)
651
669
assertEquals(epoch, secondLog.lastFetchedEpoch)
652
- mockTime.sleep(config.fileDeleteDelayMs )
670
+ mockTime.sleep(config.deleteDelayMillis )
653
671
654
672
// Assert that the log dir doesn't contain any older snapshots
655
673
Files
@@ -687,7 +705,18 @@ final class KafkaMetadataLogTest {
687
705
val leaderEpoch = 5
688
706
val maxBatchSizeInBytes = 16384
689
707
val recordSize = 64
690
- val log = buildMetadataLog(tempDir, mockTime, DefaultMetadataLogConfig .copy(maxBatchSizeInBytes = maxBatchSizeInBytes))
708
+ val config = new MetadataLogConfig (
709
+ DefaultMetadataLogConfig .logSegmentBytes,
710
+ DefaultMetadataLogConfig .logSegmentMinBytes,
711
+ DefaultMetadataLogConfig .logSegmentMillis,
712
+ DefaultMetadataLogConfig .retentionMaxBytes,
713
+ DefaultMetadataLogConfig .retentionMillis,
714
+ maxBatchSizeInBytes,
715
+ DefaultMetadataLogConfig .maxFetchSizeInBytes,
716
+ DefaultMetadataLogConfig .deleteDelayMillis,
717
+ DefaultMetadataLogConfig .nodeId
718
+ )
719
+ val log = buildMetadataLog(tempDir, mockTime, config)
691
720
692
721
val oversizeBatch = buildFullBatch(leaderEpoch, recordSize, maxBatchSizeInBytes + recordSize)
693
722
assertThrows(classOf [RecordTooLargeException ], () => {
@@ -897,18 +926,17 @@ final class KafkaMetadataLogTest {
897
926
898
927
@ Test
899
928
def testAdvanceLogStartOffsetAfterCleaning (): Unit = {
900
- val config = MetadataLogConfig (
901
- logSegmentBytes = 512 ,
902
- logSegmentMinBytes = 512 ,
903
- logSegmentMillis = 10 * 1000 ,
904
- retentionMaxBytes = 256 ,
905
- retentionMillis = 60 * 1000 ,
906
- maxBatchSizeInBytes = 512 ,
907
- maxFetchSizeInBytes = DefaultMetadataLogConfig .maxFetchSizeInBytes,
908
- fileDeleteDelayMs = ServerLogConfigs .LOG_DELETE_DELAY_MS_DEFAULT ,
909
- nodeId = 1
929
+ val config = new MetadataLogConfig (
930
+ 512 ,
931
+ 512 ,
932
+ 10 * 1000 ,
933
+ 256 ,
934
+ 60 * 1000 ,
935
+ 512 ,
936
+ DefaultMetadataLogConfig .maxFetchSizeInBytes,
937
+ ServerLogConfigs .LOG_DELETE_DELAY_MS_DEFAULT ,
938
+ 1
910
939
)
911
- config.copy()
912
940
val log = buildMetadataLog(tempDir, mockTime, config)
913
941
914
942
// Generate some segments
@@ -936,13 +964,16 @@ final class KafkaMetadataLogTest {
936
964
@ Test
937
965
def testDeleteSnapshots (): Unit = {
938
966
// Generate some logs and a few snapshots, set retention low and verify that cleaning occurs
939
- val config = DefaultMetadataLogConfig .copy(
940
- logSegmentBytes = 1024 ,
941
- logSegmentMinBytes = 1024 ,
942
- logSegmentMillis = 10 * 1000 ,
943
- retentionMaxBytes = 1024 ,
944
- retentionMillis = 60 * 1000 ,
945
- maxBatchSizeInBytes = 100
967
+ val config = new MetadataLogConfig (
968
+ 1024 ,
969
+ 1024 ,
970
+ 10 * 1000 ,
971
+ 1024 ,
972
+ 60 * 1000 ,
973
+ 100 ,
974
+ DefaultMetadataLogConfig .maxBatchSizeInBytes,
975
+ DefaultMetadataLogConfig .maxFetchSizeInBytes,
976
+ DefaultMetadataLogConfig .nodeId
946
977
)
947
978
val log = buildMetadataLog(tempDir, mockTime, config)
948
979
@@ -968,13 +999,16 @@ final class KafkaMetadataLogTest {
968
999
@ Test
969
1000
def testSoftRetentionLimit (): Unit = {
970
1001
// Set retention equal to the segment size and generate slightly more than one segment of logs
971
- val config = DefaultMetadataLogConfig .copy(
972
- logSegmentBytes = 10240 ,
973
- logSegmentMinBytes = 10240 ,
974
- logSegmentMillis = 10 * 1000 ,
975
- retentionMaxBytes = 10240 ,
976
- retentionMillis = 60 * 1000 ,
977
- maxBatchSizeInBytes = 100
1002
+ val config = new MetadataLogConfig (
1003
+ 10240 ,
1004
+ 10240 ,
1005
+ 10 * 1000 ,
1006
+ 10240 ,
1007
+ 60 * 1000 ,
1008
+ 100 ,
1009
+ DefaultMetadataLogConfig .maxFetchSizeInBytes,
1010
+ DefaultMetadataLogConfig .deleteDelayMillis,
1011
+ DefaultMetadataLogConfig .nodeId
978
1012
)
979
1013
val log = buildMetadataLog(tempDir, mockTime, config)
980
1014
@@ -1010,13 +1044,16 @@ final class KafkaMetadataLogTest {
1010
1044
1011
1045
@ Test
1012
1046
def testSegmentsLessThanLatestSnapshot (): Unit = {
1013
- val config = DefaultMetadataLogConfig .copy(
1014
- logSegmentBytes = 10240 ,
1015
- logSegmentMinBytes = 10240 ,
1016
- logSegmentMillis = 10 * 1000 ,
1017
- retentionMaxBytes = 10240 ,
1018
- retentionMillis = 60 * 1000 ,
1019
- maxBatchSizeInBytes = 200
1047
+ val config = new MetadataLogConfig (
1048
+ 10240 ,
1049
+ 10240 ,
1050
+ 10 * 1000 ,
1051
+ 10240 ,
1052
+ 60 * 1000 ,
1053
+ 200 ,
1054
+ DefaultMetadataLogConfig .maxFetchSizeInBytes,
1055
+ DefaultMetadataLogConfig .deleteDelayMillis,
1056
+ DefaultMetadataLogConfig .nodeId
1020
1057
)
1021
1058
val log = buildMetadataLog(tempDir, mockTime, config)
1022
1059
@@ -1067,16 +1104,16 @@ object KafkaMetadataLogTest {
1067
1104
override def read (input : protocol.Readable , size : Int ): Array [Byte ] = input.readArray(size)
1068
1105
}
1069
1106
1070
- val DefaultMetadataLogConfig = MetadataLogConfig (
1071
- logSegmentBytes = 100 * 1024 ,
1072
- logSegmentMinBytes = 100 * 1024 ,
1073
- logSegmentMillis = 10 * 1000 ,
1074
- retentionMaxBytes = 100 * 1024 ,
1075
- retentionMillis = 60 * 1000 ,
1076
- maxBatchSizeInBytes = KafkaRaftClient .MAX_BATCH_SIZE_BYTES ,
1077
- maxFetchSizeInBytes = KafkaRaftClient .MAX_FETCH_SIZE_BYTES ,
1078
- fileDeleteDelayMs = ServerLogConfigs .LOG_DELETE_DELAY_MS_DEFAULT ,
1079
- nodeId = 1
1107
+ val DefaultMetadataLogConfig = new MetadataLogConfig (
1108
+ 100 * 1024 ,
1109
+ 100 * 1024 ,
1110
+ 10 * 1000 ,
1111
+ 100 * 1024 ,
1112
+ 60 * 1000 ,
1113
+ KafkaRaftClient .MAX_BATCH_SIZE_BYTES ,
1114
+ KafkaRaftClient .MAX_FETCH_SIZE_BYTES ,
1115
+ ServerLogConfigs .LOG_DELETE_DELAY_MS_DEFAULT ,
1116
+ 1
1080
1117
)
1081
1118
1082
1119
def buildMetadataLogAndDir (
0 commit comments