-
Notifications
You must be signed in to change notification settings - Fork 14.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-17616: Remove KafkaServer #18384
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
This file was deleted.
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -69,6 +69,8 @@ object KafkaBroker { | |
* you do change it, be sure to make it match that regex or the system tests will fail. | ||
*/ | ||
val STARTED_MESSAGE = "Kafka Server started" | ||
|
||
val MIN_INCREMENTAL_FETCH_SESSION_EVICTION_MS: Long = 120000 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It's a bit odd that we're not following the Scala convention here. Since we're rewriting this stuff in Java anyway, maybe it's ok. |
||
} | ||
|
||
trait KafkaBroker extends Logging { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -66,18 +66,6 @@ object KafkaConfig { | |
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 = { | ||
|
@@ -192,11 +180,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) | |
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, _] = | ||
|
@@ -250,78 +233,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) | |
private val _quotaConfig = new QuotaConfig(this) | ||
def quotaConfig: QuotaConfig = _quotaConfig | ||
|
||
|
||
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) | ||
|
@@ -526,8 +437,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) | |
} | ||
|
||
/** ********* Controlled shutdown configuration ***********/ | ||
val controlledShutdownMaxRetries = getInt(ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG) | ||
val controlledShutdownRetryBackoffMs = getLong(ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Or is the KRaft code using There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it seems kraft broker relies on the event queue sending I have opened https://issues.apache.org/jira/browse/KAFKA-18417 to remove both configs There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We should also add a note to the zk to kraft migration guide regarding this. Said the same in the JIRA ticket. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Yes, I have added the note to KAFKA-18364 |
||
val controlledShutdownEnable = getBoolean(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG) | ||
|
||
/** ********* Feature configuration ***********/ | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we always throw this exception, we probably don't need to do
verifyListenerRegistrationAlterationSupported
before that and we may be able to delete that method if it's not used anywhere else.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also, we probably should include more context: it looks like we don't allow dynamic reconfiguration of listener registrations.
@cmccabe @jsancio Is this documented as part of the zk to kraft migration?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should remove
SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG
fromReconfigurableConfigs
as well.kafka/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
Line 912 in a628d9b
That can disallow users to configure advertised listeners dynamically.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I open a Jira to check all dynamic config which can't dynamic configure in Kraft, and only find
ADVERTISED_LISTENERS_CONFIG
can't, If we addressed it on this PR, I will close Jira and #18390There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There's a bunch of other ZooKeeper related logic in this class so I was planning to address them together in a follow up issue. Deleting
KafkaServer
enables us to start removing other classes that depend on this one.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm ok to merge this and then address comments in the follow-up, because there are many cleanup blocked by KafkaServer
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm ok with that, but let's file a JIRA so we don't lose track.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I care particularly about the error messages and such things since they are not easy to find afterwards.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I opened https://issues.apache.org/jira/browse/KAFKA-18405 with details about this issue.