@@ -85,6 +85,7 @@ class TestKafkaConfig {
8585 props.setProperty(" systems." + SYSTEM_NAME + " .samza.factory" , " org.apache.samza.system.kafka.KafkaSystemFactory" )
8686 props.setProperty(" stores.test1.changelog" , " kafka.mychangelog1" )
8787 props.setProperty(" stores.test2.changelog" , " kafka.mychangelog2" )
88+ props.setProperty(" stores.test2.changelog.max.message.bytes" , " 1024000" )
8889 props.setProperty(" job.changelog.system" , " kafka" )
8990 props.setProperty(" stores.test3.changelog" , " otherstream" )
9091 props.setProperty(" stores.test1.changelog.kafka.cleanup.policy" , " delete" )
@@ -98,6 +99,7 @@ class TestKafkaConfig {
9899 val kafkaConfig = new KafkaConfig (mapConfig)
99100 assertEquals(kafkaConfig.getChangelogKafkaProperties(" test1" ).getProperty(" cleanup.policy" ), " delete" )
100101 assertEquals(kafkaConfig.getChangelogKafkaProperties(" test2" ).getProperty(" cleanup.policy" ), " compact" )
102+ assertEquals(kafkaConfig.getChangelogKafkaProperties(" test2" ).getProperty(" max.message.bytes" ), " 1024000" )
101103 assertEquals(kafkaConfig.getChangelogKafkaProperties(" test3" ).getProperty(" cleanup.policy" ), " compact" )
102104 val storeToChangelog = kafkaConfig.getKafkaChangelogEnabledStores()
103105 assertEquals(" mychangelog1" , storeToChangelog.get(" test1" ).getOrElse(" " ))
@@ -114,6 +116,7 @@ class TestKafkaConfig {
114116
115117 assertEquals(kafkaConfig.getChangelogKafkaProperties(" test4" ).getProperty(" cleanup.policy" ), " delete" )
116118 assertEquals(kafkaConfig.getChangelogKafkaProperties(" test4" ).getProperty(" retention.ms" ), " 3600" )
119+ assertEquals(kafkaConfig.getChangelogKafkaProperties(" test4" ).getProperty(" max.message.bytes" ), null )
117120
118121 assertEquals(kafkaConfig.getChangelogKafkaProperties(" test5" ).getProperty(" cleanup.policy" ), " delete" )
119122 assertEquals(kafkaConfig.getChangelogKafkaProperties(" test5" ).getProperty(" retention.ms" ), " 1000" )
@@ -133,6 +136,8 @@ class TestKafkaConfig {
133136 assertEquals(batchKafkaConfig.getChangelogKafkaProperties(" test3" ).getProperty(" cleanup.policy" ), " compact" )
134137 assertEquals(batchKafkaConfig.getChangelogKafkaProperties(" test3" ).getProperty(" delete.retention.ms" ),
135138 String .valueOf(KafkaConfig .DEFAULT_RETENTION_MS_FOR_BATCH ))
139+ assertEquals(batchKafkaConfig.getChangelogKafkaProperties(" test3" ).getProperty(" max.message.bytes" ),
140+ KafkaConfig .DEFAULT_LOG_COMPACT_TOPIC_MAX_MESSAGE_BYTES )
136141 }
137142
138143 @ Test
@@ -247,97 +252,113 @@ class TestKafkaConfig {
247252 }
248253
249254 @ Test
250- def testCheckpointReplicationFactor () {
255+ def testCheckpointConfigs () {
251256 val emptyConfig = new KafkaConfig (new MapConfig ())
252257 assertEquals(" 3" , emptyConfig.getCheckpointReplicationFactor.orNull)
258+ assertEquals(1000012 , emptyConfig.getCheckpointMaxMessageBytes())
253259 assertNull(emptyConfig.getCheckpointSystem.orNull)
254260
255261 props.setProperty(KafkaConfig .CHECKPOINT_SYSTEM , " kafka-system" )
256262 props.setProperty(" task.checkpoint.replication.factor" , " 4" )
263+ props.setProperty(" task.checkpoint.max.message.bytes" , " 2048000" )
257264
258265 val mapConfig = new MapConfig (props.asScala.asJava)
259266 val kafkaConfig = new KafkaConfig (mapConfig)
260267 assertEquals(" kafka-system" , kafkaConfig.getCheckpointSystem.orNull)
261268 assertEquals(" 4" , kafkaConfig.getCheckpointReplicationFactor.orNull)
269+ assertEquals(2048000 , kafkaConfig.getCheckpointMaxMessageBytes())
262270 }
263271
264272 @ Test
265- def testCheckpointReplicationFactorWithSystemDefault () {
273+ def testCheckpointConfigsWithSystemDefault () {
266274 props.setProperty(JobConfig .JOB_DEFAULT_SYSTEM , " other-kafka-system" )
267275 props.setProperty(" systems.other-kafka-system.default.stream.replication.factor" , " 8" )
268276 props.setProperty(" systems.other-kafka-system.default.stream.segment.bytes" , " 8675309" )
277+ props.setProperty(" systems.other-kafka-system.default.stream.max.message.bytes" , " 4096000" )
269278
270279 val mapConfig = new MapConfig (props.asScala.asJava)
271280 val kafkaConfig = new KafkaConfig (mapConfig)
272281 assertEquals(" other-kafka-system" , kafkaConfig.getCheckpointSystem.orNull)
273282 assertEquals(" 8" , kafkaConfig.getCheckpointReplicationFactor.orNull)
274283 assertEquals(8675309 , kafkaConfig.getCheckpointSegmentBytes)
284+ assertEquals(4096000 , kafkaConfig.getCheckpointMaxMessageBytes())
275285 }
276286
277287 @ Test
278- def testCheckpointReplicationFactorWithSystemOverriddenDefault () {
288+ def testCheckpointConfigsWithSystemOverriddenDefault () {
279289 // Defaults
280290 props.setProperty(JobConfig .JOB_DEFAULT_SYSTEM , " other-kafka-system" )
281291 props.setProperty(" systems.other-kafka-system.default.stream.replication.factor" , " 8" )
292+ props.setProperty(" systems.other-kafka-system.default.stream.max.message.bytes" , " 4096000" )
282293 props.setProperty(" systems.kafka-system.default.stream.segment.bytes" , " 8675309" )
283294
284295 // Overrides
285296 props.setProperty(KafkaConfig .CHECKPOINT_SYSTEM , " kafka-system" )
286297 props.setProperty(KafkaConfig .CHECKPOINT_REPLICATION_FACTOR , " 4" )
298+ props.setProperty(KafkaConfig .CHECKPOINT_MAX_MESSAGE_BYTES , " 8192000" )
287299
288300 val mapConfig = new MapConfig (props.asScala.asJava)
289301 val kafkaConfig = new KafkaConfig (mapConfig)
290302 assertEquals(" kafka-system" , kafkaConfig.getCheckpointSystem.orNull)
291303 assertEquals(" 4" , kafkaConfig.getCheckpointReplicationFactor.orNull)
292304 assertEquals(8675309 , kafkaConfig.getCheckpointSegmentBytes)
305+ assertEquals(8192000 , kafkaConfig.getCheckpointMaxMessageBytes())
293306 }
294307
295308 @ Test
296- def testCoordinatorReplicationFactor () {
309+ def testCoordinatorConfigs () {
297310 val emptyConfig = new KafkaConfig (new MapConfig ())
298311 assertEquals(" 3" , emptyConfig.getCoordinatorReplicationFactor)
312+ assertEquals(" 1000012" , emptyConfig.getCoordinatorMaxMessageByte)
299313 assertNull(new JobConfig (new MapConfig ()).getCoordinatorSystemNameOrNull)
300314
301315 props.setProperty(JobConfig .JOB_COORDINATOR_SYSTEM , " kafka-system" )
302316 props.setProperty(KafkaConfig .JOB_COORDINATOR_REPLICATION_FACTOR , " 4" )
317+ props.setProperty(KafkaConfig .JOB_COORDINATOR_MAX_MESSAGE_BYTES , " 1024000" )
303318
304319 val mapConfig = new MapConfig (props.asScala.asJava)
305320 val kafkaConfig = new KafkaConfig (mapConfig)
306321 val jobConfig = new JobConfig (mapConfig)
307322 assertEquals(" kafka-system" , jobConfig.getCoordinatorSystemNameOrNull)
308323 assertEquals(" 4" , kafkaConfig.getCoordinatorReplicationFactor)
324+ assertEquals(" 1024000" , kafkaConfig.getCoordinatorMaxMessageByte)
309325 }
310326
311327 @ Test
312- def testCoordinatorReplicationFactorWithSystemDefault () {
328+ def testCoordinatorConfigsWithSystemDefault () {
313329 props.setProperty(JobConfig .JOB_DEFAULT_SYSTEM , " other-kafka-system" )
314330 props.setProperty(" systems.other-kafka-system.default.stream.replication.factor" , " 8" )
315331 props.setProperty(" systems.other-kafka-system.default.stream.segment.bytes" , " 8675309" )
332+ props.setProperty(" systems.other-kafka-system.default.stream.max.message.bytes" , " 2048000" )
316333
317334 val mapConfig = new MapConfig (props.asScala.asJava)
318335 val kafkaConfig = new KafkaConfig (mapConfig)
319336 val jobConfig = new JobConfig (mapConfig)
320337 assertEquals(" other-kafka-system" , jobConfig.getCoordinatorSystemNameOrNull)
321338 assertEquals(" 8" , kafkaConfig.getCoordinatorReplicationFactor)
322339 assertEquals(" 8675309" , kafkaConfig.getCoordinatorSegmentBytes)
340+ assertEquals(" 2048000" , kafkaConfig.getCoordinatorMaxMessageByte)
323341 }
324342
325343 @ Test
326- def testCoordinatorReplicationFactorWithSystemOverriddenDefault () {
344+ def testCoordinatorConfigsWithSystemOverriddenDefault () {
327345 // Defaults
328346 props.setProperty(JobConfig .JOB_DEFAULT_SYSTEM , " other-kafka-system" )
329347 props.setProperty(" systems.other-kafka-system.default.stream.replication.factor" , " 8" )
348+ props.setProperty(" systems.other-kafka-system.default.stream.max.message.byte" , " 2048000" )
330349 props.setProperty(" systems.kafka-system.default.stream.segment.bytes" , " 8675309" )
331350
332351 // Overrides
333352 props.setProperty(JobConfig .JOB_COORDINATOR_SYSTEM , " kafka-system" )
334353 props.setProperty(KafkaConfig .JOB_COORDINATOR_REPLICATION_FACTOR , " 4" )
354+ props.setProperty(KafkaConfig .JOB_COORDINATOR_MAX_MESSAGE_BYTES , " 4096000" )
335355
336356 val mapConfig = new MapConfig (props.asScala.asJava)
337357 val kafkaConfig = new KafkaConfig (mapConfig)
338358 val jobConfig = new JobConfig (mapConfig)
339359 assertEquals(" kafka-system" , jobConfig.getCoordinatorSystemNameOrNull)
340360 assertEquals(" 4" , kafkaConfig.getCoordinatorReplicationFactor)
361+ assertEquals(" 4096000" , kafkaConfig.getCoordinatorMaxMessageByte)
341362 assertEquals(" 8675309" , kafkaConfig.getCoordinatorSegmentBytes)
342363 }
343364}
0 commit comments