-
Notifications
You must be signed in to change notification settings - Fork 388
/
Copy pathKafkaConfig.scala
executable file
·1271 lines (1128 loc) · 74.6 KB
/
KafkaConfig.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* 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 com.automq.stream.s3.ByteBufAllocPolicy
import io.netty.util.internal.PlatformDependent
import kafka.autobalancer.config.{AutoBalancerControllerConfig, AutoBalancerMetricsReporterConfig}
import kafka.automq.AutoMQConfig
import java.util
import java.util.concurrent.TimeUnit
import java.util.Properties
import kafka.cluster.EndPoint
import kafka.utils.{CoreUtils, Logging}
import kafka.utils.Implicits._
import org.apache.kafka.common.Reconfigurable
import org.apache.kafka.common.config.{ConfigDef, ConfigException, ConfigResource, SaslConfigs, TopicConfig}
import org.apache.kafka.common.config.ConfigDef.ConfigKey
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
import org.apache.kafka.common.config.types.Password
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.record.{CompressionType, TimestampType}
import org.apache.kafka.common.security.auth.KafkaPrincipalSerde
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.coordinator.group.Group.GroupType
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.security.authorizer.AuthorizerUtils
import org.apache.kafka.security.PasswordEncoderConfigs
import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion._
import org.apache.kafka.server.config.{AbstractKafkaConfig, DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ShareGroupConfig, ZkConfigs}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.metrics.MetricConfigs
import org.apache.kafka.server.util.Csv
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
import org.apache.kafka.storage.internals.log.LogConfig.MessageFormatVersion
import org.apache.zookeeper.client.ZKClientConfig
import scala.annotation.nowarn
import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
import scala.collection.{Map, Seq}
object KafkaConfig {
def main(args: Array[String]): Unit = {
System.out.println(configDef.toHtml(4, (config: String) => "brokerconfigs_" + config,
DynamicBrokerConfig.dynamicConfigUpdateModes))
}
private[kafka] def zooKeeperClientProperty(clientConfig: ZKClientConfig, kafkaPropName: String): Option[String] = {
Option(clientConfig.getProperty(ZkConfigs.ZK_SSL_CONFIG_TO_SYSTEM_PROPERTY_MAP.get(kafkaPropName)))
}
private[kafka] def setZooKeeperClientProperty(clientConfig: ZKClientConfig, kafkaPropName: String, kafkaPropValue: Any): Unit = {
clientConfig.setProperty(ZkConfigs.ZK_SSL_CONFIG_TO_SYSTEM_PROPERTY_MAP.get(kafkaPropName),
kafkaPropName match {
case ZkConfigs.ZK_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG => (kafkaPropValue.toString.toUpperCase == "HTTPS").toString
case ZkConfigs.ZK_SSL_ENABLED_PROTOCOLS_CONFIG | ZkConfigs.ZK_SSL_CIPHER_SUITES_CONFIG => kafkaPropValue match {
case list: java.util.List[_] => list.asScala.mkString(",")
case _ => kafkaPropValue.toString
}
case _ => kafkaPropValue.toString
})
}
// For ZooKeeper TLS client authentication to be enabled the client must (at a minimum) configure itself as using TLS
// with both a client connection socket and a key store location explicitly set.
private[kafka] def zkTlsClientAuthEnabled(zkClientConfig: ZKClientConfig): Boolean = {
zooKeeperClientProperty(zkClientConfig, ZkConfigs.ZK_SSL_CLIENT_ENABLE_CONFIG).contains("true") &&
zooKeeperClientProperty(zkClientConfig, ZkConfigs.ZK_CLIENT_CNXN_SOCKET_CONFIG).isDefined &&
zooKeeperClientProperty(zkClientConfig, ZkConfigs.ZK_SSL_KEY_STORE_LOCATION_CONFIG).isDefined
}
val configDef = AbstractKafkaConfig.CONFIG_DEF
// AutoMQ inject start
AutoMQConfig.define(configDef)
AutoBalancerControllerConfig.CONFIG_DEF.configKeys().values().forEach(key => configDef.define(key))
AutoBalancerMetricsReporterConfig.CONFIG_DEF.configKeys().values().forEach(key => configDef.define(key))
// AutoMQ inject end
def configNames: Seq[String] = configDef.names.asScala.toBuffer.sorted
private[server] def defaultValues: Map[String, _] = configDef.defaultValues.asScala
private[server] def configKeys: Map[String, ConfigKey] = configDef.configKeys.asScala
def fromProps(props: Properties): KafkaConfig =
fromProps(props, true)
def fromProps(props: Properties, doLog: Boolean): KafkaConfig =
new KafkaConfig(props, doLog)
def fromProps(defaults: Properties, overrides: Properties): KafkaConfig =
fromProps(defaults, overrides, true)
def fromProps(defaults: Properties, overrides: Properties, doLog: Boolean): KafkaConfig = {
val props = new Properties()
props ++= defaults
props ++= overrides
fromProps(props, doLog)
}
def apply(props: java.util.Map[_, _], doLog: Boolean = true): KafkaConfig = new KafkaConfig(props, doLog)
private def typeOf(name: String): Option[ConfigDef.Type] = Option(configDef.configKeys.get(name)).map(_.`type`)
def configType(configName: String): Option[ConfigDef.Type] = {
val configType = configTypeExact(configName)
if (configType.isDefined) {
return configType
}
typeOf(configName) match {
case Some(t) => Some(t)
case None =>
DynamicBrokerConfig.brokerConfigSynonyms(configName, matchListenerOverride = true).flatMap(typeOf).headOption
}
}
private def configTypeExact(exactName: String): Option[ConfigDef.Type] = {
val configType = typeOf(exactName).orNull
if (configType != null) {
Some(configType)
} else {
val configKey = DynamicConfig.Broker.configKeys.get(exactName)
if (configKey != null) {
Some(configKey.`type`)
} else {
None
}
}
}
def maybeSensitive(configType: Option[ConfigDef.Type]): Boolean = {
// If we can't determine the config entry type, treat it as a sensitive config to be safe
configType.isEmpty || configType.contains(ConfigDef.Type.PASSWORD)
}
def loggableValue(resourceType: ConfigResource.Type, name: String, value: String): String = {
val maybeSensitive = resourceType match {
case ConfigResource.Type.BROKER => KafkaConfig.maybeSensitive(KafkaConfig.configType(name))
case ConfigResource.Type.TOPIC => KafkaConfig.maybeSensitive(LogConfig.configType(name).asScala)
case ConfigResource.Type.BROKER_LOGGER => false
case ConfigResource.Type.CLIENT_METRICS => false
case _ => true
}
if (maybeSensitive) Password.HIDDEN else value
}
/**
* Copy a configuration map, populating some keys that we want to treat as synonyms.
*/
def populateSynonyms(input: util.Map[_, _]): util.Map[Any, Any] = {
val output = new util.HashMap[Any, Any](input)
val brokerId = output.get(ServerConfigs.BROKER_ID_CONFIG)
val nodeId = output.get(KRaftConfigs.NODE_ID_CONFIG)
if (brokerId == null && nodeId != null) {
output.put(ServerConfigs.BROKER_ID_CONFIG, nodeId)
} else if (brokerId != null && nodeId == null) {
output.put(KRaftConfigs.NODE_ID_CONFIG, brokerId)
}
output
}
}
/**
* The class extend {@link AbstractKafkaConfig} which will be the future KafkaConfig.
* When add any new methods if it doesn't depend on anything in Core, then move it to org.apache.kafka.server.config.KafkaConfig instead of here.
* Any code depends on kafka.server.KafkaConfig will keep for using kafka.server.KafkaConfig for the time being until we move it out of core
* For more details check KAFKA-15853
*/
class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
extends AbstractKafkaConfig(KafkaConfig.configDef, props, Utils.castToStringObjectMap(props), doLog) with Logging {
def this(props: java.util.Map[_, _]) = this(true, KafkaConfig.populateSynonyms(props))
def this(props: java.util.Map[_, _], doLog: Boolean) = this(doLog, KafkaConfig.populateSynonyms(props))
// Cache the current config to avoid acquiring read lock to access from dynamicConfig
@volatile private var currentConfig = this
val processRoles: Set[ProcessRole] = parseProcessRoles()
private[server] val dynamicConfig = new DynamicBrokerConfig(this)
private[server] def updateCurrentConfig(newConfig: KafkaConfig): Unit = {
this.currentConfig = newConfig
}
// The following captures any system properties impacting ZooKeeper TLS configuration
// and defines the default values this instance will use if no explicit config is given.
// We make it part of each instance rather than the object to facilitate testing.
private val zkClientConfigViaSystemProperties = new ZKClientConfig()
override def originals: util.Map[String, AnyRef] =
if (this eq currentConfig) super.originals else currentConfig.originals
override def values: util.Map[String, _] =
if (this eq currentConfig) super.values else currentConfig.values
override def nonInternalValues: util.Map[String, _] =
if (this eq currentConfig) super.nonInternalValues else currentConfig.nonInternalValues
override def originalsStrings: util.Map[String, String] =
if (this eq currentConfig) super.originalsStrings else currentConfig.originalsStrings
override def originalsWithPrefix(prefix: String): util.Map[String, AnyRef] =
if (this eq currentConfig) super.originalsWithPrefix(prefix) else currentConfig.originalsWithPrefix(prefix)
override def valuesWithPrefixOverride(prefix: String): util.Map[String, AnyRef] =
if (this eq currentConfig) super.valuesWithPrefixOverride(prefix) else currentConfig.valuesWithPrefixOverride(prefix)
override def get(key: String): AnyRef =
if (this eq currentConfig) super.get(key) else currentConfig.get(key)
// During dynamic update, we use the values from this config, these are only used in DynamicBrokerConfig
private[server] def originalsFromThisConfig: util.Map[String, AnyRef] = super.originals
private[server] def valuesFromThisConfig: util.Map[String, _] = super.values
def valuesFromThisConfigWithPrefixOverride(prefix: String): util.Map[String, AnyRef] =
super.valuesWithPrefixOverride(prefix)
/** ********* Zookeeper Configuration ***********/
val zkConnect: String = getString(ZkConfigs.ZK_CONNECT_CONFIG)
val zkSessionTimeoutMs: Int = getInt(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG)
val zkConnectionTimeoutMs: Int =
Option(getInt(ZkConfigs.ZK_CONNECTION_TIMEOUT_MS_CONFIG)).map(_.toInt).getOrElse(getInt(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG))
val zkEnableSecureAcls: Boolean = getBoolean(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG)
val zkMaxInFlightRequests: Int = getInt(ZkConfigs.ZK_MAX_IN_FLIGHT_REQUESTS_CONFIG)
private val _remoteLogManagerConfig = new RemoteLogManagerConfig(this)
def remoteLogManagerConfig = _remoteLogManagerConfig
private val _groupCoordinatorConfig = new GroupCoordinatorConfig(this)
def groupCoordinatorConfig: GroupCoordinatorConfig = _groupCoordinatorConfig
private val _shareGroupConfig = new ShareGroupConfig(this)
def shareGroupConfig: ShareGroupConfig = _shareGroupConfig
private def zkBooleanConfigOrSystemPropertyWithDefaultValue(propKey: String): Boolean = {
// Use the system property if it exists and the Kafka config value was defaulted rather than actually provided
// Need to translate any system property value from true/false (String) to true/false (Boolean)
val actuallyProvided = originals.containsKey(propKey)
if (actuallyProvided) getBoolean(propKey) else {
val sysPropValue = KafkaConfig.zooKeeperClientProperty(zkClientConfigViaSystemProperties, propKey)
sysPropValue match {
case Some("true") => true
case Some(_) => false
case _ => getBoolean(propKey) // not specified so use the default value
}
}
}
private def zkStringConfigOrSystemPropertyWithDefaultValue(propKey: String): String = {
// Use the system property if it exists and the Kafka config value was defaulted rather than actually provided
val actuallyProvided = originals.containsKey(propKey)
if (actuallyProvided) getString(propKey) else {
KafkaConfig.zooKeeperClientProperty(zkClientConfigViaSystemProperties, propKey) match {
case Some(v) => v
case _ => getString(propKey) // not specified so use the default value
}
}
}
private def zkOptionalStringConfigOrSystemProperty(propKey: String): Option[String] = {
Option(getString(propKey)).orElse {
KafkaConfig.zooKeeperClientProperty(zkClientConfigViaSystemProperties, propKey)
}
}
private def zkPasswordConfigOrSystemProperty(propKey: String): Option[Password] = {
Option(getPassword(propKey)).orElse {
KafkaConfig.zooKeeperClientProperty(zkClientConfigViaSystemProperties, propKey).map(new Password(_))
}
}
private def zkListConfigOrSystemProperty(propKey: String): Option[util.List[String]] = {
Option(getList(propKey)).orElse {
KafkaConfig.zooKeeperClientProperty(zkClientConfigViaSystemProperties, propKey).map { sysProp =>
sysProp.split("\\s*,\\s*").toBuffer.asJava
}
}
}
val zkSslClientEnable = zkBooleanConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_CLIENT_ENABLE_CONFIG)
val zkClientCnxnSocketClassName = zkOptionalStringConfigOrSystemProperty(ZkConfigs.ZK_CLIENT_CNXN_SOCKET_CONFIG)
val zkSslKeyStoreLocation = zkOptionalStringConfigOrSystemProperty(ZkConfigs.ZK_SSL_KEY_STORE_LOCATION_CONFIG)
val zkSslKeyStorePassword = zkPasswordConfigOrSystemProperty(ZkConfigs.ZK_SSL_KEY_STORE_PASSWORD_CONFIG)
val zkSslKeyStoreType = zkOptionalStringConfigOrSystemProperty(ZkConfigs.ZK_SSL_KEY_STORE_TYPE_CONFIG)
val zkSslTrustStoreLocation = zkOptionalStringConfigOrSystemProperty(ZkConfigs.ZK_SSL_TRUST_STORE_LOCATION_CONFIG)
val zkSslTrustStorePassword = zkPasswordConfigOrSystemProperty(ZkConfigs.ZK_SSL_TRUST_STORE_PASSWORD_CONFIG)
val zkSslTrustStoreType = zkOptionalStringConfigOrSystemProperty(ZkConfigs.ZK_SSL_TRUST_STORE_TYPE_CONFIG)
val ZkSslProtocol = zkStringConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_PROTOCOL_CONFIG)
val ZkSslEnabledProtocols = zkListConfigOrSystemProperty(ZkConfigs.ZK_SSL_ENABLED_PROTOCOLS_CONFIG)
val ZkSslCipherSuites = zkListConfigOrSystemProperty(ZkConfigs.ZK_SSL_CIPHER_SUITES_CONFIG)
val ZkSslEndpointIdentificationAlgorithm = {
// Use the system property if it exists and the Kafka config value was defaulted rather than actually provided
// Need to translate any system property value from true/false to HTTPS/<blank>
val kafkaProp = ZkConfigs.ZK_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG
val actuallyProvided = originals.containsKey(kafkaProp)
if (actuallyProvided)
getString(kafkaProp)
else {
KafkaConfig.zooKeeperClientProperty(zkClientConfigViaSystemProperties, kafkaProp) match {
case Some("true") => "HTTPS"
case Some(_) => ""
case None => getString(kafkaProp) // not specified so use the default value
}
}
}
val ZkSslCrlEnable = zkBooleanConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_CRL_ENABLE_CONFIG)
val ZkSslOcspEnable = zkBooleanConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG)
/** ********* General Configuration ***********/
val brokerIdGenerationEnable: Boolean = getBoolean(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG)
val maxReservedBrokerId: Int = getInt(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG)
private[server] var _brokerId: Int = getInt(ServerConfigs.BROKER_ID_CONFIG)
def brokerId: Int = _brokerId
private[server] var _nodeId: Int = getInt(KRaftConfigs.NODE_ID_CONFIG)
def nodeId: Int = _nodeId
val initialRegistrationTimeoutMs: Int = getInt(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG)
val brokerHeartbeatIntervalMs: Int = getInt(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG)
val brokerSessionTimeoutMs: Int = getInt(KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_CONFIG)
def requiresZookeeper: Boolean = processRoles.isEmpty
def usesSelfManagedQuorum: Boolean = processRoles.nonEmpty
val migrationEnabled: Boolean = getBoolean(KRaftConfigs.MIGRATION_ENABLED_CONFIG)
val migrationMetadataMinBatchSize: Int = getInt(KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_CONFIG)
val elrEnabled: Boolean = getBoolean(KRaftConfigs.ELR_ENABLED_CONFIG)
private def parseProcessRoles(): Set[ProcessRole] = {
val roles = getList(KRaftConfigs.PROCESS_ROLES_CONFIG).asScala.map {
case "broker" => ProcessRole.BrokerRole
case "controller" => ProcessRole.ControllerRole
case role => throw new ConfigException(s"Unknown process role '$role'" +
" (only 'broker' and 'controller' are allowed roles)")
}
val distinctRoles: Set[ProcessRole] = roles.toSet
if (distinctRoles.size != roles.size) {
throw new ConfigException(s"Duplicate role names found in `${KRaftConfigs.PROCESS_ROLES_CONFIG}`: $roles")
}
distinctRoles
}
def isKRaftCombinedMode: Boolean = {
processRoles == Set(ProcessRole.BrokerRole, ProcessRole.ControllerRole)
}
def metadataLogDir: String = {
Option(getString(KRaftConfigs.METADATA_LOG_DIR_CONFIG)) match {
case Some(dir) => dir
case None => logDirs.head
}
}
def metadataLogSegmentBytes = getInt(KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG)
def metadataLogSegmentMillis = getLong(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG)
def metadataRetentionBytes = getLong(KRaftConfigs.METADATA_MAX_RETENTION_BYTES_CONFIG)
def metadataRetentionMillis = getLong(KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_CONFIG)
def metadataNodeIDConfig = getInt(KRaftConfigs.NODE_ID_CONFIG)
def metadataLogSegmentMinBytes = getInt(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)
val serverMaxStartupTimeMs = getLong(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG)
def backgroundThreads = getInt(ServerConfigs.BACKGROUND_THREADS_CONFIG)
def numIoThreads = getInt(ServerConfigs.NUM_IO_THREADS_CONFIG)
def messageMaxBytes = getInt(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG)
val requestTimeoutMs = getInt(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG)
val connectionSetupTimeoutMs = getLong(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG)
val connectionSetupTimeoutMaxMs = getLong(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG)
def getNumReplicaAlterLogDirsThreads: Int = {
val numThreads: Integer = Option(getInt(ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG)).getOrElse(logDirs.size)
numThreads
}
/************* Metadata Configuration ***********/
val metadataSnapshotMaxNewRecordBytes = getLong(KRaftConfigs.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG)
val metadataSnapshotMaxIntervalMs = getLong(KRaftConfigs.METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG)
val metadataMaxIdleIntervalNs: Option[Long] = {
val value = TimeUnit.NANOSECONDS.convert(getInt(KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG).toLong, TimeUnit.MILLISECONDS)
if (value > 0) Some(value) else None
}
/************* Authorizer Configuration ***********/
def createNewAuthorizer(): Option[Authorizer] = {
val className = getString(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG)
if (className == null || className.isEmpty)
None
else {
Some(AuthorizerUtils.createAuthorizer(className))
}
}
val earlyStartListeners: Set[ListenerName] = {
val listenersSet = listeners.map(_.listenerName).toSet
val controllerListenersSet = controllerListeners.map(_.listenerName).toSet
Option(getString(ServerConfigs.EARLY_START_LISTENERS_CONFIG)) match {
case None => controllerListenersSet
case Some(str) =>
str.split(",").map(_.trim()).filterNot(_.isEmpty).map { str =>
val listenerName = new ListenerName(str)
if (!listenersSet.contains(listenerName) && !controllerListenersSet.contains(listenerName))
throw new ConfigException(s"${ServerConfigs.EARLY_START_LISTENERS_CONFIG} contains " +
s"listener ${listenerName.value()}, but this is not contained in " +
s"${SocketServerConfigs.LISTENERS_CONFIG} or ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG}")
listenerName
}.toSet
}
}
/** ********* Socket Server Configuration ***********/
val socketSendBufferBytes = getInt(SocketServerConfigs.SOCKET_SEND_BUFFER_BYTES_CONFIG)
val socketReceiveBufferBytes = getInt(SocketServerConfigs.SOCKET_RECEIVE_BUFFER_BYTES_CONFIG)
val socketRequestMaxBytes = getInt(SocketServerConfigs.SOCKET_REQUEST_MAX_BYTES_CONFIG)
val socketListenBacklogSize = getInt(SocketServerConfigs.SOCKET_LISTEN_BACKLOG_SIZE_CONFIG)
val maxConnectionsPerIp = getInt(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_CONFIG)
val maxConnectionsPerIpOverrides: Map[String, Int] =
getMap(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG, getString(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG)).map { case (k, v) => (k, v.toInt)}
def maxConnections = getInt(SocketServerConfigs.MAX_CONNECTIONS_CONFIG)
def maxConnectionCreationRate = getInt(SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG)
val connectionsMaxIdleMs = getLong(SocketServerConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG)
val failedAuthenticationDelayMs = getInt(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG)
val queuedMaxRequests = getInt(SocketServerConfigs.QUEUED_MAX_REQUESTS_CONFIG)
// AutoMQ inject start
val queuedMaxRequestSize = getInt(SocketServerConfigs.QUEUED_MAX_REQUESTS_SIZE_BYTES_CONFIG)
// AutoMQ inject end
val queuedMaxBytes = getLong(SocketServerConfigs.QUEUED_MAX_BYTES_CONFIG)
def numNetworkThreads = getInt(SocketServerConfigs.NUM_NETWORK_THREADS_CONFIG)
/***************** rack configuration **************/
val rack = Option(getString(ServerConfigs.BROKER_RACK_CONFIG))
val replicaSelectorClassName = Option(getString(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG))
/** ********* Log Configuration ***********/
val autoCreateTopicsEnable = getBoolean(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG)
val numPartitions = getInt(ServerLogConfigs.NUM_PARTITIONS_CONFIG)
val logDirs: Seq[String] = Csv.parseCsvList(Option(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).getOrElse(getString(ServerLogConfigs.LOG_DIR_CONFIG))).asScala
def logSegmentBytes = getInt(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG)
def logFlushIntervalMessages = getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG)
val logCleanerThreads = getInt(CleanerConfig.LOG_CLEANER_THREADS_PROP)
def numRecoveryThreadsPerDataDir = getInt(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG)
val logFlushSchedulerIntervalMs = getLong(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG)
val logFlushOffsetCheckpointIntervalMs = getInt(ServerLogConfigs.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG).toLong
val logFlushStartOffsetCheckpointIntervalMs = getInt(ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG).toLong
val logCleanupIntervalMs = getLong(ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG)
def logCleanupPolicy = getList(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG)
def logRetentionBytes = getLong(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG)
val logCleanerDedupeBufferSize = getLong(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP)
val logCleanerDedupeBufferLoadFactor = getDouble(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP)
val logCleanerIoBufferSize = getInt(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP)
val logCleanerIoMaxBytesPerSecond = getDouble(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP)
def logCleanerDeleteRetentionMs = getLong(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP)
def logCleanerMinCompactionLagMs = getLong(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP)
def logCleanerMaxCompactionLagMs = getLong(CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP)
val logCleanerBackoffMs = getLong(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP)
def logCleanerMinCleanRatio = getDouble(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP)
val logCleanerEnable = getBoolean(CleanerConfig.LOG_CLEANER_ENABLE_PROP)
def logIndexSizeMaxBytes = getInt(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG)
def logIndexIntervalBytes = getInt(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG)
def logDeleteDelayMs = getLong(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG)
def logRollTimeMillis: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG)).getOrElse(60 * 60 * 1000L * getInt(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG))
def logRollTimeJitterMillis: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG)).getOrElse(60 * 60 * 1000L * getInt(ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG))
def logFlushIntervalMs: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG)).getOrElse(getLong(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG))
def minInSyncReplicas = getInt(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG)
def logPreAllocateEnable: java.lang.Boolean = getBoolean(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG)
def logInitialTaskDelayMs: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG)).getOrElse(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT)
// We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0`
// is passed, `0.10.0-IV0` may be picked)
@nowarn("cat=deprecation")
private val logMessageFormatVersionString = getString(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG)
/* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
@deprecated("3.0")
lazy val logMessageFormatVersion =
if (LogConfig.shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion))
MetadataVersion.fromVersionString(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DEFAULT)
else MetadataVersion.fromVersionString(logMessageFormatVersionString)
def logMessageTimestampType = TimestampType.forName(getString(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG))
/* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */
@deprecated("3.6")
def logMessageTimestampDifferenceMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG)
// In the transition period before logMessageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
// we are using its value if logMessageTimestampBeforeMaxMs default value hasn't changed.
// See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
@nowarn("cat=deprecation")
def logMessageTimestampBeforeMaxMs: Long = {
val messageTimestampBeforeMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG)
if (messageTimestampBeforeMaxMs != ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT) {
messageTimestampBeforeMaxMs
} else {
logMessageTimestampDifferenceMaxMs
}
}
// In the transition period before logMessageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
// we are using its value if logMessageTimestampAfterMaxMs default value hasn't changed.
// See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
@nowarn("cat=deprecation")
def logMessageTimestampAfterMaxMs: Long = {
val messageTimestampAfterMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG)
if (messageTimestampAfterMaxMs != Long.MaxValue) {
messageTimestampAfterMaxMs
} else {
logMessageTimestampDifferenceMaxMs
}
}
def logMessageDownConversionEnable: Boolean = getBoolean(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG)
def logDirFailureTimeoutMs: Long = getLong(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG)
/** ********* Replication configuration ***********/
val controllerSocketTimeoutMs: Int = getInt(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG)
val defaultReplicationFactor: Int = getInt(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG)
val replicaLagTimeMaxMs = getLong(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG)
val replicaSocketTimeoutMs = getInt(ReplicationConfigs.REPLICA_SOCKET_TIMEOUT_MS_CONFIG)
val replicaSocketReceiveBufferBytes = getInt(ReplicationConfigs.REPLICA_SOCKET_RECEIVE_BUFFER_BYTES_CONFIG)
val replicaFetchMaxBytes = getInt(ReplicationConfigs.REPLICA_FETCH_MAX_BYTES_CONFIG)
val replicaFetchWaitMaxMs = getInt(ReplicationConfigs.REPLICA_FETCH_WAIT_MAX_MS_CONFIG)
val replicaFetchMinBytes = getInt(ReplicationConfigs.REPLICA_FETCH_MIN_BYTES_CONFIG)
val replicaFetchResponseMaxBytes = getInt(ReplicationConfigs.REPLICA_FETCH_RESPONSE_MAX_BYTES_CONFIG)
val replicaFetchBackoffMs = getInt(ReplicationConfigs.REPLICA_FETCH_BACKOFF_MS_CONFIG)
def numReplicaFetchers = getInt(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG)
val replicaHighWatermarkCheckpointIntervalMs = getLong(ReplicationConfigs.REPLICA_HIGH_WATERMARK_CHECKPOINT_INTERVAL_MS_CONFIG)
val fetchPurgatoryPurgeIntervalRequests = getInt(ReplicationConfigs.FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG)
val producerPurgatoryPurgeIntervalRequests = getInt(ReplicationConfigs.PRODUCER_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG)
val deleteRecordsPurgatoryPurgeIntervalRequests = getInt(ReplicationConfigs.DELETE_RECORDS_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG)
val autoLeaderRebalanceEnable = getBoolean(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG)
val leaderImbalancePerBrokerPercentage = getInt(ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_CONFIG)
val leaderImbalanceCheckIntervalSeconds: Long = getLong(ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_CONFIG)
val uncleanLeaderElectionCheckIntervalMs: Long = getLong(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_INTERVAL_MS_CONFIG)
def uncleanLeaderElectionEnable: java.lang.Boolean = getBoolean(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG)
// We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0`
// is passed, `0.10.0-IV0` may be picked)
val interBrokerProtocolVersionString = getString(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)
val interBrokerProtocolVersion = if (processRoles.isEmpty) {
MetadataVersion.fromVersionString(interBrokerProtocolVersionString)
} else {
if (originals.containsKey(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)) {
// A user-supplied IBP was given
val configuredVersion = MetadataVersion.fromVersionString(interBrokerProtocolVersionString)
if (!configuredVersion.isKRaftSupported) {
throw new ConfigException(s"A non-KRaft version $interBrokerProtocolVersionString given for ${ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG}. " +
s"The minimum version is ${MetadataVersion.MINIMUM_KRAFT_VERSION}")
} else {
warn(s"${ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG} is deprecated in KRaft mode as of 3.3 and will only " +
s"be read when first upgrading from a KRaft prior to 3.3. See kafka-storage.sh help for details on setting " +
s"the metadata.version for a new KRaft cluster.")
}
}
// In KRaft mode, we pin this value to the minimum KRaft-supported version. This prevents inadvertent usage of
// the static IBP config in broker components running in KRaft mode
MetadataVersion.MINIMUM_KRAFT_VERSION
}
/** ********* Controlled shutdown configuration ***********/
val controlledShutdownMaxRetries = getInt(ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG)
val controlledShutdownRetryBackoffMs = getLong(ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG)
val controlledShutdownEnable = getBoolean(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG)
/** ********* Feature configuration ***********/
def isFeatureVersioningSupported = interBrokerProtocolVersion.isFeatureVersioningSupported
/** New group coordinator configs */
val groupCoordinatorRebalanceProtocols = {
val protocols = getList(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG)
.asScala.map(_.toUpperCase).map(GroupType.valueOf).toSet
if (!protocols.contains(GroupType.CLASSIC)) {
throw new ConfigException(s"Disabling the '${GroupType.CLASSIC}' protocol is not supported.")
}
if (protocols.contains(GroupType.CONSUMER)) {
if (processRoles.isEmpty) {
throw new ConfigException(s"The new '${GroupType.CONSUMER}' rebalance protocol is only supported in KRaft cluster.")
}
warn(s"The new '${GroupType.CONSUMER}' rebalance protocol is enabled along with the new group coordinator. " +
"This is part of the preview of KIP-848 and MUST NOT be used in production.")
}
protocols
}
// The new group coordinator is enabled in two cases: 1) The internal configuration to enable
// it is explicitly set; or 2) the consumer rebalance protocol is enabled.
val isNewGroupCoordinatorEnabled = getBoolean(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG) ||
groupCoordinatorRebalanceProtocols.contains(GroupType.CONSUMER)
/** ********* Transaction management configuration ***********/
val transactionalIdExpirationMs = getInt(TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_CONFIG)
val transactionMaxTimeoutMs = getInt(TransactionStateManagerConfigs.TRANSACTIONS_MAX_TIMEOUT_MS_CONFIG)
val transactionTopicMinISR = getInt(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG)
val transactionsLoadBufferSize = getInt(TransactionLogConfigs.TRANSACTIONS_LOAD_BUFFER_SIZE_CONFIG)
val transactionTopicReplicationFactor = getShort(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG)
val transactionTopicPartitions = getInt(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG)
val transactionTopicSegmentBytes = getInt(TransactionLogConfigs.TRANSACTIONS_TOPIC_SEGMENT_BYTES_CONFIG)
val transactionAbortTimedOutTransactionCleanupIntervalMs = getInt(TransactionStateManagerConfigs.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_CONFIG)
val transactionRemoveExpiredTransactionalIdCleanupIntervalMs = getInt(TransactionStateManagerConfigs.TRANSACTIONS_REMOVE_EXPIRED_TRANSACTIONAL_ID_CLEANUP_INTERVAL_MS_CONFIG)
def transactionPartitionVerificationEnable = getBoolean(TransactionLogConfigs.TRANSACTION_PARTITION_VERIFICATION_ENABLE_CONFIG)
def producerIdExpirationMs = getInt(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_MS_CONFIG)
val producerIdExpirationCheckIntervalMs = getInt(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_CONFIG)
/** ********* Metric Configuration **************/
val metricNumSamples = getInt(MetricConfigs.METRIC_NUM_SAMPLES_CONFIG)
val metricSampleWindowMs = getLong(MetricConfigs.METRIC_SAMPLE_WINDOW_MS_CONFIG)
val metricRecordingLevel = getString(MetricConfigs.METRIC_RECORDING_LEVEL_CONFIG)
/** ********* Kafka Client Telemetry Metrics Configuration ***********/
val clientTelemetryMaxBytes: Int = getInt(MetricConfigs.CLIENT_TELEMETRY_MAX_BYTES_CONFIG)
/** ********* SSL/SASL Configuration **************/
// Security configs may be overridden for listeners, so it is not safe to use the base values
// Hence the base SSL/SASL configs are not fields of KafkaConfig, listener configs should be
// retrieved using KafkaConfig#valuesWithPrefixOverride
private def saslEnabledMechanisms(listenerName: ListenerName): Set[String] = {
val value = valuesWithPrefixOverride(listenerName.configPrefix).get(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG)
if (value != null)
value.asInstanceOf[util.List[String]].asScala.toSet
else
Set.empty[String]
}
def interBrokerListenerName = getInterBrokerListenerNameAndSecurityProtocol._1
def interBrokerSecurityProtocol = getInterBrokerListenerNameAndSecurityProtocol._2
def controlPlaneListenerName = getControlPlaneListenerNameAndSecurityProtocol.map { case (listenerName, _) => listenerName }
def controlPlaneSecurityProtocol = getControlPlaneListenerNameAndSecurityProtocol.map { case (_, securityProtocol) => securityProtocol }
def saslMechanismInterBrokerProtocol = getString(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG)
val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion.isSaslInterBrokerHandshakeRequestEnabled
/** ********* DelegationToken Configuration **************/
val delegationTokenSecretKey = Option(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG))
.getOrElse(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG))
val tokenAuthEnabled = delegationTokenSecretKey != null && delegationTokenSecretKey.value.nonEmpty
val delegationTokenMaxLifeMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG)
val delegationTokenExpiryTimeMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG)
val delegationTokenExpiryCheckIntervalMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG)
/** ********* Password encryption configuration for dynamic configs *********/
def passwordEncoderSecret = Option(getPassword(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG))
def passwordEncoderOldSecret = Option(getPassword(PasswordEncoderConfigs.PASSWORD_ENCODER_OLD_SECRET_CONFIG))
def passwordEncoderCipherAlgorithm = getString(PasswordEncoderConfigs.PASSWORD_ENCODER_CIPHER_ALGORITHM_CONFIG)
def passwordEncoderKeyFactoryAlgorithm = getString(PasswordEncoderConfigs.PASSWORD_ENCODER_KEYFACTORY_ALGORITHM_CONFIG)
def passwordEncoderKeyLength = getInt(PasswordEncoderConfigs.PASSWORD_ENCODER_KEY_LENGTH_CONFIG)
def passwordEncoderIterations = getInt(PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_CONFIG)
/** ********* Quota Configuration **************/
val numQuotaSamples = getInt(QuotaConfigs.NUM_QUOTA_SAMPLES_CONFIG)
val quotaWindowSizeSeconds = getInt(QuotaConfigs.QUOTA_WINDOW_SIZE_SECONDS_CONFIG)
val numReplicationQuotaSamples = getInt(QuotaConfigs.NUM_REPLICATION_QUOTA_SAMPLES_CONFIG)
val replicationQuotaWindowSizeSeconds = getInt(QuotaConfigs.REPLICATION_QUOTA_WINDOW_SIZE_SECONDS_CONFIG)
val numAlterLogDirsReplicationQuotaSamples = getInt(QuotaConfigs.NUM_ALTER_LOG_DIRS_REPLICATION_QUOTA_SAMPLES_CONFIG)
val alterLogDirsReplicationQuotaWindowSizeSeconds = getInt(QuotaConfigs.ALTER_LOG_DIRS_REPLICATION_QUOTA_WINDOW_SIZE_SECONDS_CONFIG)
val numControllerQuotaSamples = getInt(QuotaConfigs.NUM_CONTROLLER_QUOTA_SAMPLES_CONFIG)
val controllerQuotaWindowSizeSeconds = getInt(QuotaConfigs.CONTROLLER_QUOTA_WINDOW_SIZE_SECONDS_CONFIG)
/** ********* Fetch Configuration **************/
val maxIncrementalFetchSessionCacheSlots = getInt(ServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_CONFIG)
val fetchMaxBytes = getInt(ServerConfigs.FETCH_MAX_BYTES_CONFIG)
/** ********* Request Limit Configuration ***********/
val maxRequestPartitionSizeLimit = getInt(ServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_CONFIG)
val deleteTopicEnable = getBoolean(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG)
def compressionType = getString(ServerConfigs.COMPRESSION_TYPE_CONFIG)
def gzipCompressionLevel = getInt(ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG)
def lz4CompressionLevel = getInt(ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG)
def zstdCompressionLevel = getInt(ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG)
/** ********* Raft Quorum Configuration *********/
val quorumVoters = getList(QuorumConfig.QUORUM_VOTERS_CONFIG)
val quorumBootstrapServers = getList(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG)
val quorumElectionTimeoutMs = getInt(QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG)
val quorumFetchTimeoutMs = getInt(QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG)
val quorumElectionBackoffMs = getInt(QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG)
val quorumLingerMs = getInt(QuorumConfig.QUORUM_LINGER_MS_CONFIG)
val quorumRequestTimeoutMs = getInt(QuorumConfig.QUORUM_REQUEST_TIMEOUT_MS_CONFIG)
val quorumRetryBackoffMs = getInt(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG)
// AutoMQ inject start
/**
* Adjust S3 configurations based on memory resource.
*
* @return (s3WALCacheSize, s3BlockCacheSize, s3WALUploadThreshold)
*/
private def adjustS3Configs(s3StreamAllocatorPolicy: ByteBufAllocPolicy): (Long, Long, Long) = {
val rawS3WALCacheSize = getLong(AutoMQConfig.S3_WAL_CACHE_SIZE_CONFIG)
val rawS3BlockCacheSize = getLong(AutoMQConfig.S3_BLOCK_CACHE_SIZE_CONFIG)
val rawS3WALUploadThreshold = getLong(AutoMQConfig.S3_WAL_UPLOAD_THRESHOLD_CONFIG)
val s3WALCacheSizeSet = rawS3WALCacheSize > 0
val s3BlockCacheSizeSet = rawS3BlockCacheSize > 0
val s3WALUploadThresholdSet = rawS3WALUploadThreshold > 0
if (s3WALCacheSizeSet != s3BlockCacheSizeSet) {
throw new ConfigException(s"${AutoMQConfig.S3_WAL_CACHE_SIZE_CONFIG} and ${AutoMQConfig.S3_BLOCK_CACHE_SIZE_CONFIG} must be set together")
}
val s3AvailableMemory = if (s3StreamAllocatorPolicy.isDirect) {
PlatformDependent.maxDirectMemory()
} else {
Runtime.getRuntime.maxMemory() / 2
}
val s3WALCacheSize: Long = {
if (s3WALCacheSizeSet) {
rawS3WALCacheSize
} else {
// for example:
// availableMemory = 3G, adjusted = max(3G / 3, (3G - 3G) / 3 * 2) = max(1G, 0) = 1G
// availableMemory = 6G, adjusted = max(6G / 3, (6G - 3G) / 3 * 2) = max(2G, 2G) = 2G
// availableMemory = 9G, adjusted = max(9G / 3, (9G - 3G) / 3 * 2) = max(3G, 4G) = 4G
// availableMemory = 12G, adjusted = max(12G / 3, (12G - 3G) / 3 * 2) = max(4G, 6G) = 6G
val adjusted = Math.max(s3AvailableMemory / 3, (s3AvailableMemory - 3L * 1024 * 1024 * 1024) / 3 * 2)
if (doLog) {
info(s"${AutoMQConfig.S3_WAL_CACHE_SIZE_CONFIG} is not set, using $adjusted as the default value")
}
adjusted
}
}
val s3BlockCacheSize: Long = {
if (s3BlockCacheSizeSet) {
rawS3BlockCacheSize
} else {
// it's just 1/2 of s3WALCacheSize
val adjusted = Math.max(s3AvailableMemory / 6, (s3AvailableMemory - 3L * 1024 * 1024 * 1024) / 3)
if (doLog) {
info(s"${AutoMQConfig.S3_BLOCK_CACHE_SIZE_CONFIG} is not set, using $adjusted as the default value")
}
adjusted
}
}
val s3WALUploadThreshold: Long = {
if (s3WALUploadThresholdSet) {
rawS3WALUploadThreshold
} else {
// it should not be greater than 1/3 of s3WALCacheSize and 500M
val adjusted = math.min(s3WALCacheSize / 3, 500L * 1024 * 1024)
if (doLog) {
info(s"${AutoMQConfig.S3_WAL_UPLOAD_THRESHOLD_CONFIG} is not set, using $adjusted as the default value")
}
adjusted
}
}
(s3WALCacheSize, s3BlockCacheSize, s3WALUploadThreshold)
}
val elasticStreamEnabled = getBoolean(AutoMQConfig.ELASTIC_STREAM_ENABLE_CONFIG)
val elasticStreamEndpoint = getString(AutoMQConfig.ELASTIC_STREAM_ENDPOINT_CONFIG)
val automq = new AutoMQConfig().setup(this)
val s3OpsTelemetryEnabled = getBoolean(AutoMQConfig.S3_TELEMETRY_OPS_ENABLED_CONFIG)
val s3StreamSplitSize = getInt(AutoMQConfig.S3_STREAM_SPLIT_SIZE_CONFIG)
val s3ObjectBlockSize = getInt(AutoMQConfig.S3_OBJECT_BLOCK_SIZE_CONFIG)
val s3ObjectPartSize = getInt(AutoMQConfig.S3_OBJECT_PART_SIZE_CONFIG)
val s3StreamAllocatorPolicy = Enum.valueOf(classOf[ByteBufAllocPolicy], getString(AutoMQConfig.S3_STREAM_ALLOCATOR_POLICY_CONFIG))
val (s3WALCacheSize, s3BlockCacheSize, s3WALUploadThreshold) = adjustS3Configs(s3StreamAllocatorPolicy)
val s3StreamObjectCompactionTaskIntervalMinutes = getInt(AutoMQConfig.S3_STREAM_OBJECT_COMPACTION_INTERVAL_MINUTES_CONFIG)
val s3StreamObjectCompactionMaxSizeBytes = getLong(AutoMQConfig.S3_STREAM_OBJECT_COMPACTION_MAX_SIZE_BYTES_CONFIG)
val s3ControllerRequestRetryMaxCount = getInt(AutoMQConfig.S3_CONTROLLER_REQUEST_RETRY_MAX_COUNT_CONFIG)
val s3ControllerRequestRetryBaseDelayMs = getLong(AutoMQConfig.S3_CONTROLLER_REQUEST_RETRY_BASE_DELAY_MS_CONFIG)
val s3StreamSetObjectCompactionInterval = getInt(AutoMQConfig.S3_STREAM_SET_OBJECT_COMPACTION_INTERVAL_CONFIG)
val s3StreamSetObjectCompactionCacheSize = getLong(AutoMQConfig.S3_STREAM_SET_OBJECT_COMPACTION_CACHE_SIZE_CONFIG)
val s3StreamSetObjectCompactionStreamSplitSize = getLong(AutoMQConfig.S3_STREAM_SET_OBJECT_COMPACTION_STREAM_SPLIT_SIZE_CONFIG)
val s3StreamSetObjectCompactionForceSplitMinutes = getInt(AutoMQConfig.S3_STREAM_SET_OBJECT_COMPACTION_FORCE_SPLIT_MINUTES_CONFIG)
val s3StreamSetObjectCompactionMaxObjectNum = getInt(AutoMQConfig.S3_STREAM_SET_OBJECT_COMPACTION_MAX_OBJECT_NUM_CONFIG)
val s3MaxStreamNumPerStreamSetObject = getInt(AutoMQConfig.S3_MAX_STREAM_NUM_PER_STREAM_SET_OBJECT_CONFIG)
val s3MaxStreamObjectNumPerCommit = getInt(AutoMQConfig.S3_MAX_STREAM_OBJECT_NUM_PER_COMMIT_CONFIG)
val s3MockEnable = getBoolean(AutoMQConfig.S3_MOCK_ENABLE_CONFIG)
val s3ObjectDeleteRetentionTimeInSecond = getLong(AutoMQConfig.S3_OBJECT_DELETION_MINUTES_CONFIG) * 60
val s3NetworkBaselineBandwidthProp = getLong(AutoMQConfig.S3_NETWORK_BASELINE_BANDWIDTH_CONFIG)
val s3RefillPeriodMsProp = getInt(AutoMQConfig.S3_NETWORK_REFILL_PERIOD_MS_CONFIG)
val s3MetricsLevel = getString(AutoMQConfig.S3_TELEMETRY_METRICS_LEVEL_CONFIG)
val s3ExporterReportIntervalMs = getInt(AutoMQConfig.S3_TELEMETRY_EXPORTER_REPORT_INTERVAL_MS_CONFIG)
val s3BackPressureEnabled = getBoolean(AutoMQConfig.S3_BACK_PRESSURE_ENABLED_CONFIG)
val s3BackPressureCooldownMs = getLong(AutoMQConfig.S3_BACK_PRESSURE_COOLDOWN_MS_CONFIG)
val tableTopicNamespace = getString(TopicConfig.TABLE_TOPIC_NAMESPACE_CONFIG)
val tableTopicSchemaRegistryUrl = getString(AutoMQConfig.TABLE_TOPIC_SCHEMA_REGISTRY_URL_CONFIG);
// AutoMQ inject end
/** Internal Configurations **/
val unstableApiVersionsEnabled = getBoolean(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG)
val unstableFeatureVersionsEnabled = getBoolean(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG)
def addReconfigurable(reconfigurable: Reconfigurable): Unit = {
dynamicConfig.addReconfigurable(reconfigurable)
}
def removeReconfigurable(reconfigurable: Reconfigurable): Unit = {
dynamicConfig.removeReconfigurable(reconfigurable)
}
def logRetentionTimeMillis: Long = {
val millisInMinute = 60L * 1000L
val millisInHour = 60L * millisInMinute
val millis: java.lang.Long =
Option(getLong(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG)).getOrElse(
Option(getInt(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG)) match {
case Some(mins) => millisInMinute * mins
case None => getInt(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG) * millisInHour
})
if (millis < 0) return -1
millis
}
private def getMap(propName: String, propValue: String): Map[String, String] = {
try {
Csv.parseCsvMap(propValue).asScala
} catch {
case e: Exception => throw new IllegalArgumentException("Error parsing configuration property '%s': %s".format(propName, e.getMessage))
}
}
def listeners: Seq[EndPoint] =
CoreUtils.listenerListToEndPoints(getString(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap)
def controllerListenerNames: Seq[String] = {
val value = Option(getString(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)).getOrElse("")
if (value.isEmpty) {
Seq.empty
} else {
value.split(",")
}
}
def controllerListeners: Seq[EndPoint] =
listeners.filter(l => controllerListenerNames.contains(l.listenerName.value()))
def saslMechanismControllerProtocol: String = getString(KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG)
def controlPlaneListener: Option[EndPoint] = {
controlPlaneListenerName.map { listenerName =>
listeners.filter(endpoint => endpoint.listenerName.value() == listenerName.value()).head
}
}
def dataPlaneListeners: Seq[EndPoint] = {
listeners.filterNot { listener =>
val name = listener.listenerName.value()
name.equals(getString(SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG)) ||
controllerListenerNames.contains(name)
}
}
def effectiveAdvertisedControllerListeners: Seq[EndPoint] = {
val controllerAdvertisedListeners = advertisedListeners.filter(l => controllerListenerNames.contains(l.listenerName.value()))
val controllerListenersValue = controllerListeners
controllerListenerNames.flatMap { name =>
controllerAdvertisedListeners
.find(endpoint => endpoint.listenerName.equals(ListenerName.normalised(name)))
.orElse(controllerListenersValue.find(endpoint => endpoint.listenerName.equals(ListenerName.normalised(name))))
}
}
def effectiveAdvertisedBrokerListeners: Seq[EndPoint] = {
advertisedListeners.filter(l => {
if (!controllerListenerNames.contains(l.listenerName.value())) {
true
} else if (migrationEnabled && Some(l.listenerName.value()).equals(controlPlaneListener.map(_.listenerName.value()))) {
// KAFKA-17788: during ZK migration, always include control.plane.listener.name
// in advertisedBrokerListeners.
true
} else {
false
}
})
}
// Use advertised listeners if defined, fallback to listeners otherwise
private def advertisedListeners: Seq[EndPoint] = {
val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG)
if (advertisedListenersProp != null) {
CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false)
} else {
listeners
}
}
private def getInterBrokerListenerNameAndSecurityProtocol: (ListenerName, SecurityProtocol) = {
Option(getString(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG)) match {
case Some(_) if originals.containsKey(ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG) =>
throw new ConfigException(s"Only one of ${ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG} and " +
s"${ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG} should be set.")
case Some(name) =>
val listenerName = ListenerName.normalised(name)
val securityProtocol = effectiveListenerSecurityProtocolMap.getOrElse(listenerName,
throw new ConfigException(s"Listener with name ${listenerName.value} defined in " +
s"${ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG} not found in ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG}."))
(listenerName, securityProtocol)
case None =>
val securityProtocol = getSecurityProtocol(getString(ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG),
ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG)
(ListenerName.forSecurityProtocol(securityProtocol), securityProtocol)
}
}
private def getControlPlaneListenerNameAndSecurityProtocol: Option[(ListenerName, SecurityProtocol)] = {
Option(getString(SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG)) match {
case Some(name) =>
val listenerName = ListenerName.normalised(name)
val securityProtocol = effectiveListenerSecurityProtocolMap.getOrElse(listenerName,
throw new ConfigException(s"Listener with ${listenerName.value} defined in " +
s"${SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG} not found in ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG}."))
Some(listenerName, securityProtocol)
case None => None
}
}
private def getSecurityProtocol(protocolName: String, configName: String): SecurityProtocol = {
try SecurityProtocol.forName(protocolName)
catch {
case _: IllegalArgumentException =>
throw new ConfigException(s"Invalid security protocol `$protocolName` defined in $configName")
}
}
def effectiveListenerSecurityProtocolMap: Map[ListenerName, SecurityProtocol] = {
val mapValue = getMap(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, getString(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG))
.map { case (listenerName, protocolName) =>
ListenerName.normalised(listenerName) -> getSecurityProtocol(protocolName, SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG)
}
if (usesSelfManagedQuorum && !originals.containsKey(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG)) {
// Nothing was specified explicitly for listener.security.protocol.map, so we are using the default value,
// and we are using KRaft.
// Add PLAINTEXT mappings for controller listeners as long as there is no SSL or SASL_{PLAINTEXT,SSL} in use
def isSslOrSasl(name: String): Boolean = name.equals(SecurityProtocol.SSL.name) || name.equals(SecurityProtocol.SASL_SSL.name) || name.equals(SecurityProtocol.SASL_PLAINTEXT.name)
// check controller listener names (they won't appear in listeners when process.roles=broker)
// as well as listeners for occurrences of SSL or SASL_*
if (controllerListenerNames.exists(isSslOrSasl) ||
Csv.parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).asScala.exists(listenerValue => isSslOrSasl(EndPoint.parseListenerName(listenerValue)))) {
mapValue // don't add default mappings since we found something that is SSL or SASL_*
} else {
// add the PLAINTEXT mappings for all controller listener names that are not explicitly PLAINTEXT
mapValue ++ controllerListenerNames.filterNot(SecurityProtocol.PLAINTEXT.name.equals(_)).map(
new ListenerName(_) -> SecurityProtocol.PLAINTEXT)
}
} else {
mapValue
}
}
// Topic IDs are used with all self-managed quorum clusters and ZK cluster with IBP greater than or equal to 2.8
def usesTopicId: Boolean =
usesSelfManagedQuorum || interBrokerProtocolVersion.isTopicIdsSupported
validateValues()
@nowarn("cat=deprecation")
private def validateValues(): Unit = {
if (nodeId != brokerId) {
throw new ConfigException(s"You must set `${KRaftConfigs.NODE_ID_CONFIG}` to the same value as `${ServerConfigs.BROKER_ID_CONFIG}`.")
}
if (requiresZookeeper) {
if (zkConnect == null) {
throw new ConfigException(s"Missing required configuration `${ZkConfigs.ZK_CONNECT_CONFIG}` which has no default value.")
}
if (brokerIdGenerationEnable) {
if (migrationEnabled) {
require(brokerId >= 0, "broker.id generation is incompatible with ZooKeeper migration. Please stop using it before enabling migration (set broker.id to a value greater or equal to 0).")
}
require(brokerId >= -1 && brokerId <= maxReservedBrokerId, "broker.id must be greater than or equal to -1 and not greater than reserved.broker.max.id")
} else {
require(brokerId >= 0, "broker.id must be greater than or equal to 0")
}
} else {
// KRaft-based metadata quorum
if (nodeId < 0) {
throw new ConfigException(s"Missing configuration `${KRaftConfigs.NODE_ID_CONFIG}` which is required " +
s"when `process.roles` is defined (i.e. when running in KRaft mode).")
}
if (migrationEnabled) {
if (zkConnect == null) {
throw new ConfigException(s"If using `${KRaftConfigs.MIGRATION_ENABLED_CONFIG}` in KRaft mode, `${ZkConfigs.ZK_CONNECT_CONFIG}` must also be set.")
}
}