diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index b5f9e408c9442..70bba904fb7e9 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -48,7 +48,7 @@ import org.apache.kafka.server.common.CheckpointFile; import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.common.StopPartition; -import org.apache.kafka.server.config.ServerConfigs; +import org.apache.kafka.server.config.KRaftConfigs; import org.apache.kafka.server.log.remote.metadata.storage.ClassLoaderAwareRemoteLogMetadataManager; import org.apache.kafka.server.log.remote.quota.RLMQuotaManager; import org.apache.kafka.server.log.remote.quota.RLMQuotaManagerConfig; @@ -163,7 +163,7 @@ public class RemoteLogManager implements Closeable { private static final Logger LOGGER = LoggerFactory.getLogger(RemoteLogManager.class); private static final String REMOTE_LOG_READER_THREAD_NAME_PATTERN = "remote-log-reader-%d"; private final RemoteLogManagerConfig rlmConfig; - private final int brokerId; + private final int nodeId; private final String logDir; private final Time time; private final Function> fetchLog; @@ -212,7 +212,7 @@ public class RemoteLogManager implements Closeable { * Creates RemoteLogManager instance with the given arguments. * * @param rlmConfig Configuration required for remote logging subsystem(tiered storage) at the broker level. - * @param brokerId id of the current broker. + * @param nodeId id of the current node. * @param logDir directory of Kafka log segments. * @param time Time instance. * @param clusterId The cluster id. @@ -223,7 +223,7 @@ public class RemoteLogManager implements Closeable { */ @SuppressWarnings({"this-escape"}) public RemoteLogManager(RemoteLogManagerConfig rlmConfig, - int brokerId, + int nodeId, String logDir, String clusterId, Time time, @@ -232,7 +232,7 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig, BrokerTopicStats brokerTopicStats, Metrics metrics) throws IOException { this.rlmConfig = rlmConfig; - this.brokerId = brokerId; + this.nodeId = nodeId; this.logDir = logDir; this.clusterId = clusterId; this.time = time; @@ -376,7 +376,7 @@ RemoteStorageManager createRemoteStorageManager() { private void configureRSM() { final Map rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps()); - rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); + rsmProps.put(KRaftConfigs.NODE_ID_CONFIG, nodeId); remoteLogStorageManager.configure(rsmProps); } @@ -406,7 +406,7 @@ private void configureRLMM() { // update the remoteLogMetadataProps here to override endpoint config if any rlmmProps.putAll(rlmConfig.remoteLogMetadataManagerProps()); - rlmmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId); + rlmmProps.put(KRaftConfigs.NODE_ID_CONFIG, nodeId); rlmmProps.put(LOG_DIR_CONFIG, logDir); rlmmProps.put("cluster.id", clusterId); @@ -570,7 +570,7 @@ private void deleteRemoteLogPartition(TopicIdPartition partition) throws RemoteS List deleteSegmentStartedEvents = metadataList.stream() .map(metadata -> new RemoteLogSegmentMetadataUpdate(metadata.remoteLogSegmentId(), time.milliseconds(), - metadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)) + metadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, nodeId)) .collect(Collectors.toList()); publishEvents(deleteSegmentStartedEvents).get(); @@ -585,7 +585,7 @@ private void deleteRemoteLogPartition(TopicIdPartition partition) throws RemoteS List deleteSegmentFinishedEvents = metadataList.stream() .map(metadata -> new RemoteLogSegmentMetadataUpdate(metadata.remoteLogSegmentId(), time.milliseconds(), - metadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)) + metadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, nodeId)) .collect(Collectors.toList()); publishEvents(deleteSegmentFinishedEvents).get(); } @@ -800,7 +800,7 @@ public RLMTask(TopicIdPartition topicIdPartition) { } protected LogContext getLogContext() { - return new LogContext("[RemoteLogManager=" + brokerId + " partition=" + topicIdPartition + "] "); + return new LogContext("[RemoteLogManager=" + nodeId + " partition=" + topicIdPartition + "] "); } public void run() { @@ -1014,7 +1014,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment boolean isTxnIdxEmpty = segment.txnIndex().isEmpty(); RemoteLogSegmentMetadata copySegmentStartedRlsm = new RemoteLogSegmentMetadata(segmentId, segment.baseOffset(), endOffset, - segment.largestTimestamp(), brokerId, time.milliseconds(), segment.log().sizeInBytes(), + segment.largestTimestamp(), nodeId, time.milliseconds(), segment.log().sizeInBytes(), segmentLeaderEpochs, isTxnIdxEmpty); remoteLogMetadataManager.addRemoteLogSegmentMetadata(copySegmentStartedRlsm).get(); @@ -1041,7 +1041,7 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment } RemoteLogSegmentMetadataUpdate copySegmentFinishedRlsm = new RemoteLogSegmentMetadataUpdate(segmentId, time.milliseconds(), - customMetadata, RemoteLogSegmentState.COPY_SEGMENT_FINISHED, brokerId); + customMetadata, RemoteLogSegmentState.COPY_SEGMENT_FINISHED, nodeId); if (customMetadata.isPresent()) { long customMetadataSize = customMetadata.get().value().length; @@ -1486,7 +1486,7 @@ private boolean deleteRemoteLogSegment( // Publish delete segment started event. remoteLogMetadataManager.updateRemoteLogSegmentMetadata( new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(), - segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get(); + segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, nodeId)).get(); brokerTopicStats.topicStats(topic).remoteDeleteRequestRate().mark(); brokerTopicStats.allTopicsStats().remoteDeleteRequestRate().mark(); @@ -1503,7 +1503,7 @@ private boolean deleteRemoteLogSegment( // Publish delete segment finished event. remoteLogMetadataManager.updateRemoteLogSegmentMetadata( new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(), - segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get(); + segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, nodeId)).get(); LOGGER.debug("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId()); return true; } diff --git a/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java b/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java index a62987be2036e..b73d24de90c9a 100644 --- a/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java +++ b/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java @@ -33,7 +33,7 @@ public MetadataVersionConfigValidator( KafkaConfig config, FaultHandler faultHandler ) { - int id = config.brokerId(); + int id = config.nodeId(); this.name = "MetadataVersionPublisher(id=" + id + ")"; this.config = config; this.faultHandler = faultHandler; diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 4315a981bac2f..02059bbc4a21e 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -153,7 +153,7 @@ object Partition { new Partition(topicPartition, _topicId = topicId, replicaLagTimeMaxMs = replicaManager.config.replicaLagTimeMaxMs, - localBrokerId = replicaManager.config.brokerId, + localBrokerId = replicaManager.config.nodeId, localBrokerEpochSupplier = replicaManager.brokerEpochSupplier, time = time, alterPartitionListener = isrChangeListener, @@ -1364,7 +1364,7 @@ class Partition(val topicPartition: TopicPartition, if (inSyncSize < minIsr && requiredAcks == -1) { throw new NotEnoughReplicasException(s"The size of the current ISR : $inSyncSize " + s"is insufficient to satisfy the min.isr requirement of $minIsr for partition $topicPartition, " + - s"live replica(s) broker.id are : $inSyncReplicaIds") + s"live replica(s) node.id are : $inSyncReplicaIds") } val info = leaderLog.appendAsLeader(records, leaderEpoch = this.leaderEpoch, origin, diff --git a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala index c1f071413e800..a377bc7d1956e 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala @@ -1773,8 +1773,8 @@ object GroupCoordinator { time: Time, metrics: Metrics ): GroupCoordinator = { - val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", config.brokerId) - val rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", config.brokerId) + val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", config.nodeId) + val rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", config.nodeId) GroupCoordinator(config, replicaManager, heartbeatPurgatory, rebalancePurgatory, time, metrics) } @@ -1804,8 +1804,8 @@ object GroupCoordinator { groupMaxSize = config.groupCoordinatorConfig.classicGroupMaxSize, groupInitialRebalanceDelayMs = config.groupCoordinatorConfig.classicGroupInitialRebalanceDelayMs) - val groupMetadataManager = new GroupMetadataManager(config.brokerId, offsetConfig, replicaManager, time, metrics) - new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupMetadataManager, heartbeatPurgatory, + val groupMetadataManager = new GroupMetadataManager(config.nodeId, offsetConfig, replicaManager, time, metrics) + new GroupCoordinator(config.nodeId, groupConfig, offsetConfig, groupMetadataManager, heartbeatPurgatory, rebalancePurgatory, time, metrics) } diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index d6ca11add8614..640ab22f6cbc9 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -56,10 +56,10 @@ object TransactionCoordinator { config.transactionStateManagerConfig.transactionRemoveExpiredTransactionalIdCleanupIntervalMs, config.requestTimeoutMs) - val txnStateManager = new TransactionStateManager(config.brokerId, scheduler, replicaManager, metadataCache, txnConfig, + val txnStateManager = new TransactionStateManager(config.nodeId, scheduler, replicaManager, metadataCache, txnConfig, time, metrics) - val logContext = new LogContext(s"[TransactionCoordinator id=${config.brokerId}] ") + val logContext = new LogContext(s"[TransactionCoordinator id=${config.nodeId}] ") val txnMarkerChannelManager = TransactionMarkerChannelManager(config, metrics, metadataCache, txnStateManager, time, logContext) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala index 2e71a72420957..a5edc7b09744a 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMarkerChannelManager.scala @@ -82,7 +82,7 @@ object TransactionMarkerChannelManager { val networkClient = new NetworkClient( selector, new ManualMetadataUpdater(), - s"broker-${config.brokerId}-txn-marker-sender", + s"broker-${config.nodeId}-txn-marker-sender", 1, 50, 50, @@ -161,12 +161,12 @@ class TransactionMarkerChannelManager( networkClient: NetworkClient, txnStateManager: TransactionStateManager, time: Time -) extends InterBrokerSendThread("TxnMarkerSenderThread-" + config.brokerId, networkClient, config.requestTimeoutMs, time) +) extends InterBrokerSendThread("TxnMarkerSenderThread-" + config.nodeId, networkClient, config.requestTimeoutMs, time) with Logging { private val metricsGroup = new KafkaMetricsGroup(this.getClass) - this.logIdent = "[Transaction Marker Channel Manager " + config.brokerId + "]: " + this.logIdent = "[Transaction Marker Channel Manager " + config.nodeId + "]: " private val interBrokerListenerName: ListenerName = config.interBrokerListenerName diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index d0a2c4e811d2f..db448c5945390 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -84,7 +84,7 @@ class SocketServer( private val maxQueuedRequests = config.queuedMaxRequests - protected val nodeId: Int = config.brokerId + protected val nodeId: Int = config.nodeId private val logContext = new LogContext(s"[SocketServer listenerType=${apiVersionManager.listenerType}, nodeId=$nodeId] ") diff --git a/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala b/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala index eab17214c64a8..bf2d913f36ef1 100644 --- a/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala +++ b/core/src/main/scala/kafka/server/AddPartitionsToTxnManager.scala @@ -89,7 +89,7 @@ class AddPartitionsToTxnManager( partitionFor: String => Int, time: Time ) extends InterBrokerSendThread( - "AddPartitionsToTxnSenderThread-" + config.brokerId, + "AddPartitionsToTxnSenderThread-" + config.nodeId, client, config.requestTimeoutMs, time diff --git a/core/src/main/scala/kafka/server/AlterPartitionManager.scala b/core/src/main/scala/kafka/server/AlterPartitionManager.scala index fa1c3602beedb..fdebb9f7a0668 100644 --- a/core/src/main/scala/kafka/server/AlterPartitionManager.scala +++ b/core/src/main/scala/kafka/server/AlterPartitionManager.scala @@ -91,7 +91,7 @@ object AlterPartitionManager { controllerChannelManager = channelManager, scheduler = scheduler, time = time, - brokerId = config.brokerId, + brokerId = config.nodeId, brokerEpochSupplier = brokerEpochSupplier, metadataVersionSupplier = () => metadataCache.metadataVersion() ) diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 47b89dd1de18f..0b186dbcf5ce6 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -291,7 +291,7 @@ class BrokerServer( ) alterPartitionManager.start() - val addPartitionsLogContext = new LogContext(s"[AddPartitionsToTxnManager broker=${config.brokerId}]") + val addPartitionsLogContext = new LogContext(s"[AddPartitionsToTxnManager broker=${config.nodeId}]") val addPartitionsToTxnNetworkClient = NetworkUtils.buildNetworkClient("AddPartitionsManager", config, metrics, time, addPartitionsLogContext) val addPartitionsToTxnManager = new AddPartitionsToTxnManager( config, @@ -315,7 +315,7 @@ class BrokerServer( assignmentsManager = new AssignmentsManager( time, assignmentsChannelManager, - config.brokerId, + config.nodeId, () => metadataCache.getImage(), (directoryId: Uuid) => logManager.directoryPath(directoryId). getOrElse("[unknown directory path]") @@ -370,7 +370,7 @@ class BrokerServer( groupCoordinator = createGroupCoordinator() val producerIdManagerSupplier = () => ProducerIdManager.rpc( - config.brokerId, + config.nodeId, time, () => lifecycleManager.brokerEpoch, clientToControllerChannelManager @@ -534,7 +534,7 @@ class BrokerServer( if (e != null) brokerMetadataPublisher.firstPublishFuture.completeExceptionally(e) }) metadataPublishers.add(brokerMetadataPublisher) - brokerRegistrationTracker = new BrokerRegistrationTracker(config.brokerId, + brokerRegistrationTracker = new BrokerRegistrationTracker(config.nodeId, () => lifecycleManager.resendBrokerRegistrationUnlessZkMode()) metadataPublishers.add(brokerRegistrationTracker) @@ -631,7 +631,7 @@ class BrokerServer( val writer = new CoordinatorPartitionWriter( replicaManager ) - new GroupCoordinatorService.Builder(config.brokerId, config.groupCoordinatorConfig) + new GroupCoordinatorService.Builder(config.nodeId, config.groupCoordinatorConfig) .withTime(time) .withTimer(timer) .withLoader(loader) @@ -669,7 +669,7 @@ class BrokerServer( val writer = new CoordinatorPartitionWriter( replicaManager ) - Some(new ShareCoordinatorService.Builder(config.brokerId, config.shareCoordinatorConfig) + Some(new ShareCoordinatorService.Builder(config.nodeId, config.shareCoordinatorConfig) .withTimer(timer) .withTime(time) .withLoader(loader) @@ -691,7 +691,7 @@ class BrokerServer( klass.getConstructor(classOf[PersisterStateManager]) .newInstance( new PersisterStateManager( - NetworkUtils.buildNetworkClient("Persister", config, metrics, Time.SYSTEM, new LogContext(s"[Persister broker=${config.brokerId}]")), + NetworkUtils.buildNetworkClient("Persister", config, metrics, Time.SYSTEM, new LogContext(s"[Persister broker=${config.nodeId}]")), new ShareCoordinatorMetadataCacheHelperImpl(metadataCache, key => shareCoordinator.get.partitionFor(key), config.interBrokerListenerName), Time.SYSTEM, new SystemTimerReaper( @@ -717,7 +717,7 @@ class BrokerServer( protected def createRemoteLogManager(): Option[RemoteLogManager] = { if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) { - Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time, + Some(new RemoteLogManager(config.remoteLogManagerConfig, config.nodeId, config.logDirs.head, clusterId, time, (tp: TopicPartition) => logManager.getLog(tp).toJava, (tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => { logManager.getLog(tp).foreach { log => diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index eabe12d0d942a..b9702a2df238e 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -115,12 +115,12 @@ class TopicConfigHandler(private val replicaManager: ReplicaManager, def updateThrottledList(prop: String, quotaManager: ReplicationQuotaManager): Unit = { if (topicConfig.containsKey(prop) && topicConfig.getProperty(prop).nonEmpty) { - val partitions = parseThrottledPartitions(topicConfig, kafkaConfig.brokerId, prop) + val partitions = parseThrottledPartitions(topicConfig, kafkaConfig.nodeId, prop) quotaManager.markThrottled(topic, partitions.map(Integer.valueOf).asJava) - debug(s"Setting $prop on broker ${kafkaConfig.brokerId} for topic: $topic and partitions $partitions") + debug(s"Setting $prop on broker ${kafkaConfig.nodeId} for topic: $topic and partitions $partitions") } else { quotaManager.removeThrottle(topic) - debug(s"Removing $prop from broker ${kafkaConfig.brokerId} for topic $topic") + debug(s"Removing $prop from broker ${kafkaConfig.nodeId} for topic $topic") } } updateThrottledList(QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, quotas.leader) @@ -206,8 +206,8 @@ class BrokerConfigHandler(private val brokerConfig: KafkaConfig, def processConfigChanges(brokerId: String, properties: Properties): Unit = { if (brokerId == ZooKeeperInternals.DEFAULT_STRING) brokerConfig.dynamicConfig.updateDefaultConfig(properties) - else if (brokerConfig.brokerId == brokerId.trim.toInt) { - brokerConfig.dynamicConfig.updateBrokerConfig(brokerConfig.brokerId, properties) + else if (brokerConfig.nodeId == brokerId.trim.toInt) { + brokerConfig.dynamicConfig.updateBrokerConfig(brokerConfig.nodeId, properties) } val updatedDynamicBrokerConfigs = brokerConfig.dynamicConfig.currentDynamicBrokerConfigs val updatedDynamicDefaultConfigs = brokerConfig.dynamicConfig.currentDynamicDefaultConfigs diff --git a/core/src/main/scala/kafka/server/ConfigHelper.scala b/core/src/main/scala/kafka/server/ConfigHelper.scala index 095a474441a13..cfeb895582449 100644 --- a/core/src/main/scala/kafka/server/ConfigHelper.scala +++ b/core/src/main/scala/kafka/server/ConfigHelper.scala @@ -118,17 +118,17 @@ class ConfigHelper(metadataCache: MetadataCache, config: KafkaConfig, configRepo if (resource.resourceName == null || resource.resourceName.isEmpty) createResponseConfig(config.dynamicConfig.currentDynamicDefaultConfigs, createBrokerConfigEntry(perBrokerConfig = false, includeSynonyms, includeDocumentation)) - else if (resourceNameToBrokerId(resource.resourceName) == config.brokerId) + else if (resourceNameToBrokerId(resource.resourceName) == config.nodeId) createResponseConfig(allConfigs(config), createBrokerConfigEntry(perBrokerConfig = true, includeSynonyms, includeDocumentation)) else - throw new InvalidRequestException(s"Unexpected broker id, expected ${config.brokerId} or empty string, but received ${resource.resourceName}") + throw new InvalidRequestException(s"Unexpected broker id, expected ${config.nodeId} or empty string, but received ${resource.resourceName}") case ConfigResource.Type.BROKER_LOGGER => if (resource.resourceName == null || resource.resourceName.isEmpty) throw new InvalidRequestException("Broker id must not be empty") - else if (resourceNameToBrokerId(resource.resourceName) != config.brokerId) - throw new InvalidRequestException(s"Unexpected broker id, expected ${config.brokerId} but received ${resource.resourceName}") + else if (resourceNameToBrokerId(resource.resourceName) != config.nodeId) + throw new InvalidRequestException(s"Unexpected broker id, expected ${config.nodeId} but received ${resource.resourceName}") else createResponseConfig(Log4jController.loggers, (name, value) => new DescribeConfigsResponseData.DescribeConfigsResourceResult().setName(name) diff --git a/core/src/main/scala/kafka/server/DelegationTokenManager.scala b/core/src/main/scala/kafka/server/DelegationTokenManager.scala index 4fa4cae209f11..9cac8fcaae72c 100644 --- a/core/src/main/scala/kafka/server/DelegationTokenManager.scala +++ b/core/src/main/scala/kafka/server/DelegationTokenManager.scala @@ -100,7 +100,7 @@ object DelegationTokenManager { class DelegationTokenManager(val config: KafkaConfig, val tokenCache: DelegationTokenCache, val time: Time) extends Logging { - this.logIdent = s"[Token Manager on Node ${config.brokerId}]: " + this.logIdent = s"[Token Manager on Node ${config.nodeId}]: " protected val lock = new Object() diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 32febffb546e8..7ced75e179ebc 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -38,7 +38,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.PasswordEncoder import org.apache.kafka.server.ProcessRole -import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms} +import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.{ClientMetricsReceiverPlugin, MetricConfigs} import org.apache.kafka.server.telemetry.ClientTelemetry @@ -246,7 +246,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging case _ => } addReconfigurable(kafkaServer.kafkaYammerMetrics) - addReconfigurable(new DynamicMetricsReporters(kafkaConfig.brokerId, kafkaServer.config, kafkaServer.metrics, kafkaServer.clusterId)) + addReconfigurable(new DynamicMetricsReporters(kafkaConfig.nodeId, kafkaServer.config, kafkaServer.metrics, kafkaServer.clusterId)) addReconfigurable(new DynamicClientQuotaCallback(kafkaServer.quotaManagers, kafkaServer.config)) addBrokerReconfigurable(new BrokerDynamicThreadPool(kafkaServer)) @@ -820,7 +820,7 @@ class DynamicMetricsReporters(brokerId: Int, config: KafkaConfig, metrics: Metri class DynamicMetricReporterState(brokerId: Int, config: KafkaConfig, metrics: Metrics, clusterId: String) { private[server] val dynamicConfig = config.dynamicConfig - private val propsOverride = Map[String, AnyRef](ServerConfigs.BROKER_ID_CONFIG -> brokerId.toString) + private val propsOverride = Map[String, AnyRef](KRaftConfigs.NODE_ID_CONFIG -> brokerId.toString) private[server] val currentReporters = mutable.Map[String, MetricsReporter]() createReporters(config, clusterId, metricsReporterClasses(dynamicConfig.currentKafkaConfig.values()).asJava, Collections.emptyMap[String, Object]) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index e243d40bbb2ff..ab88da49a27cd 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -129,21 +129,6 @@ object KafkaConfig { } 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 - } } /** @@ -155,8 +140,8 @@ object KafkaConfig { 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)) + def this(props: java.util.Map[_, _]) = this(true, props) + def this(props: java.util.Map[_, _], doLog: Boolean) = this(doLog, props) // Cache the current config to avoid acquiring read lock to access from dynamicConfig @volatile private var currentConfig = this @@ -221,9 +206,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) def quotaConfig: QuotaConfig = _quotaConfig /** ********* General Configuration ***********/ - val brokerIdGenerationEnable: Boolean = getBoolean(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG) - val maxReservedBrokerId: Int = getInt(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG) - var brokerId: Int = getInt(ServerConfigs.BROKER_ID_CONFIG) val nodeId: Int = getInt(KRaftConfigs.NODE_ID_CONFIG) val initialRegistrationTimeoutMs: Int = getInt(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG) val brokerHeartbeatIntervalMs: Int = getInt(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG) @@ -231,7 +213,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) val controllerPerformanceSamplePeriodMs: Long = getLong(KRaftConfigs.CONTROLLER_PERFORMANCE_SAMPLE_PERIOD_MS) val controllerPerformanceAlwaysLogThresholdMs: Long = getLong(KRaftConfigs.CONTROLLER_PERFORMANCE_ALWAYS_LOG_THRESHOLD_MS) - def requiresZookeeper: Boolean = processRoles.isEmpty def usesSelfManagedQuorum: Boolean = processRoles.nonEmpty private def parseProcessRoles(): Set[ProcessRole] = { @@ -633,9 +614,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) validateValues() 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}`.") - } require(logRollTimeMillis >= 1, "log.roll.ms must be greater than or equal to 1") require(logRollTimeJitterMillis >= 0, "log.roll.jitter.ms must be greater than or equal to 0") require(logRetentionTimeMillis >= 1 || logRetentionTimeMillis == -1, "log.retention.ms must be unlimited (-1) or, greater than or equal to 1") diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 5642c8aaa99ce..dc56c2ed0184d 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -94,7 +94,7 @@ class KafkaRaftServer( // the controller endpoints are passed to the KRaft manager controller.foreach(_.startup()) broker.foreach(_.startup()) - AppInfoParser.registerAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics, time.milliseconds()) + AppInfoParser.registerAppInfo(Server.MetricsPrefix, config.nodeId.toString, metrics, time.milliseconds()) info(KafkaBroker.STARTED_MESSAGE) } @@ -104,7 +104,7 @@ class KafkaRaftServer( // stops the raft client early on, which would disrupt broker shutdown. broker.foreach(_.shutdown()) controller.foreach(_.shutdown()) - CoreUtils.swallow(AppInfoParser.unregisterAppInfo(Server.MetricsPrefix, config.brokerId.toString, metrics), this) + CoreUtils.swallow(AppInfoParser.unregisterAppInfo(Server.MetricsPrefix, config.nodeId.toString, metrics), this) } override def awaitShutdown(): Unit = { diff --git a/core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala b/core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala index 1e2a6cd033e48..bcd2b48836269 100644 --- a/core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala +++ b/core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala @@ -50,7 +50,7 @@ class LocalLeaderEndPoint(sourceBroker: BrokerEndPoint, replicaManager: ReplicaManager, quota: ReplicaQuota) extends LeaderEndPoint with Logging { - private val replicaId = brokerConfig.brokerId + private val replicaId = brokerConfig.nodeId private val maxBytes = brokerConfig.replicaFetchResponseMaxBytes private val fetchSize = brokerConfig.replicaFetchMaxBytes private var inProgressPartition: Option[TopicPartition] = None diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala index c353a82550316..dffea380496d8 100644 --- a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala +++ b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala @@ -136,7 +136,7 @@ class NodeToControllerChannelManagerImpl( metrics, time, channelName, - Map("BrokerId" -> config.brokerId.toString).asJava, + Map("BrokerId" -> config.nodeId.toString).asJava, false, channelBuilder, logContext @@ -144,7 +144,7 @@ class NodeToControllerChannelManagerImpl( new NetworkClient( selector, manualMetadataUpdater, - config.brokerId.toString, + config.nodeId.toString, 1, 50, 50, diff --git a/core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala b/core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala index 94bb6f5140dca..ecf0e8f2968f4 100644 --- a/core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala +++ b/core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala @@ -113,7 +113,7 @@ class RemoteLeaderEndPoint(logPrefix: String, .setCurrentLeaderEpoch(currentLeaderEpoch) .setTimestamp(timestamp))) val metadataVersion = metadataVersionSupplier() - val requestBuilder = ListOffsetsRequest.Builder.forReplica(metadataVersion.listOffsetRequestVersion, brokerConfig.brokerId) + val requestBuilder = ListOffsetsRequest.Builder.forReplica(metadataVersion.listOffsetRequestVersion, brokerConfig.nodeId) .setTargetTimes(Collections.singletonList(topic)) val clientResponse = blockingSender.sendRequest(requestBuilder) @@ -143,7 +143,7 @@ class RemoteLeaderEndPoint(logPrefix: String, topic.partitions.add(epochData) } - val epochRequest = OffsetsForLeaderEpochRequest.Builder.forFollower(topics, brokerConfig.brokerId) + val epochRequest = OffsetsForLeaderEpochRequest.Builder.forFollower(topics, brokerConfig.nodeId) debug(s"Sending offset for leader epoch request $epochRequest") try { @@ -210,7 +210,7 @@ class RemoteLeaderEndPoint(logPrefix: String, metadataVersion.fetchRequestVersion } val requestBuilder = FetchRequest.Builder - .forReplica(version, brokerConfig.brokerId, brokerEpochSupplier(), maxWait, minBytes, fetchData.toSend) + .forReplica(version, brokerConfig.nodeId, brokerEpochSupplier(), maxWait, minBytes, fetchData.toSend) .setMaxBytes(maxBytes) .removed(fetchData.toForget) .replaced(fetchData.toReplace) diff --git a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsManager.scala b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsManager.scala index 7b2d7863077ea..5496783a23ba6 100644 --- a/core/src/main/scala/kafka/server/ReplicaAlterLogDirsManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaAlterLogDirsManager.scala @@ -29,7 +29,7 @@ class ReplicaAlterLogDirsManager(brokerConfig: KafkaConfig, directoryEventHandler: DirectoryEventHandler = DirectoryEventHandler.NOOP ) extends AbstractFetcherManager[ReplicaAlterLogDirsThread]( - name = s"ReplicaAlterLogDirsManager on broker ${brokerConfig.brokerId}", + name = s"ReplicaAlterLogDirsManager on broker ${brokerConfig.nodeId}", clientId = "ReplicaAlterLogDirs", numFetchers = brokerConfig.getNumReplicaAlterLogDirsThreads) { diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala index 621f50f9168cd..287d8fd1322a8 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherManager.scala @@ -32,17 +32,17 @@ class ReplicaFetcherManager(brokerConfig: KafkaConfig, metadataVersionSupplier: () => MetadataVersion, brokerEpochSupplier: () => Long) extends AbstractFetcherManager[ReplicaFetcherThread]( - name = "ReplicaFetcherManager on broker " + brokerConfig.brokerId, + name = "ReplicaFetcherManager on broker " + brokerConfig.nodeId, clientId = "Replica", numFetchers = brokerConfig.numReplicaFetchers) { override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): ReplicaFetcherThread = { val prefix = threadNamePrefix.map(tp => s"$tp:").getOrElse("") val threadName = s"${prefix}ReplicaFetcherThread-$fetcherId-${sourceBroker.id}" - val logContext = new LogContext(s"[ReplicaFetcher replicaId=${brokerConfig.brokerId}, leaderId=${sourceBroker.id}, " + + val logContext = new LogContext(s"[ReplicaFetcher replicaId=${brokerConfig.nodeId}, leaderId=${sourceBroker.id}, " + s"fetcherId=$fetcherId] ") val endpoint = new BrokerBlockingSender(sourceBroker, brokerConfig, metrics, time, fetcherId, - s"broker-${brokerConfig.brokerId}-fetcher-$fetcherId", logContext) + s"broker-${brokerConfig.nodeId}-fetcher-$fetcherId", logContext) val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id) val leader = new RemoteLeaderEndPoint(logContext.logPrefix, endpoint, fetchSessionHandler, brokerConfig, replicaManager, quotaManager, metadataVersionSupplier, brokerEpochSupplier) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 7f0c6d41dbdf6..eb9db609867b6 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -182,7 +182,7 @@ class ReplicaFetcherThread(name: String, // mark the future replica for truncation only when we do last truncation if (offsetTruncationState.truncationCompleted) - replicaMgr.replicaAlterLogDirsManager.markPartitionsForTruncation(brokerConfig.brokerId, tp, + replicaMgr.replicaAlterLogDirsManager.markPartitionsForTruncation(brokerConfig.nodeId, tp, offsetTruncationState.offset) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 9504731f6839d..8f14f5b7820f7 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -292,33 +292,33 @@ class ReplicaManager(val config: KafkaConfig, val delayedProducePurgatory = delayedProducePurgatoryParam.getOrElse( new DelayedOperationPurgatory[DelayedProduce]( - "Produce", config.brokerId, + "Produce", config.nodeId, config.producerPurgatoryPurgeIntervalRequests)) val delayedFetchPurgatory = delayedFetchPurgatoryParam.getOrElse( new DelayedOperationPurgatory[DelayedFetch]( - "Fetch", config.brokerId, + "Fetch", config.nodeId, config.fetchPurgatoryPurgeIntervalRequests)) val delayedDeleteRecordsPurgatory = delayedDeleteRecordsPurgatoryParam.getOrElse( new DelayedOperationPurgatory[DelayedDeleteRecords]( - "DeleteRecords", config.brokerId, + "DeleteRecords", config.nodeId, config.deleteRecordsPurgatoryPurgeIntervalRequests)) val delayedElectLeaderPurgatory = delayedElectLeaderPurgatoryParam.getOrElse( new DelayedOperationPurgatory[DelayedElectLeader]( - "ElectLeader", config.brokerId)) + "ElectLeader", config.nodeId)) val delayedRemoteFetchPurgatory = delayedRemoteFetchPurgatoryParam.getOrElse( new DelayedOperationPurgatory[DelayedRemoteFetch]( - "RemoteFetch", config.brokerId)) + "RemoteFetch", config.nodeId)) val delayedRemoteListOffsetsPurgatory = delayedRemoteListOffsetsPurgatoryParam.getOrElse( new DelayedOperationPurgatory[DelayedRemoteListOffsets]( - "RemoteListOffsets", config.brokerId)) + "RemoteListOffsets", config.nodeId)) val delayedShareFetchPurgatory = delayedShareFetchPurgatoryParam.getOrElse( new DelayedOperationPurgatory[DelayedShareFetch]( - "ShareFetch", config.brokerId, + "ShareFetch", config.nodeId, config.shareGroupConfig.shareFetchPurgatoryPurgeIntervalRequests)) /* epoch of the controller that last changed the leader */ @volatile private[server] var controllerEpoch: Int = 0 - protected val localBrokerId = config.brokerId + protected val localBrokerId = config.nodeId protected val allPartitions = new Pool[TopicPartition, HostedPartition]( valueFactory = Some(tp => HostedPartition.Online(Partition(tp, time, this))) ) @@ -1248,7 +1248,7 @@ class ReplicaManager(val config: KafkaConfig, val futureLog = futureLocalLogOrException(topicPartition) logManager.abortAndPauseCleaning(topicPartition) - val initialFetchState = InitialFetchState(topicId, new BrokerEndPoint(config.brokerId, "localhost", -1), + val initialFetchState = InitialFetchState(topicId, new BrokerEndPoint(config.nodeId, "localhost", -1), partition.getLeaderEpoch, futureLog.highWatermark) replicaAlterLogDirsManager.addFetcherForPartitions(Map(topicPartition -> initialFetchState)) } @@ -2270,7 +2270,7 @@ class ReplicaManager(val config: KafkaConfig, val topicPartition = partition.topicPartition logManager.getLog(topicPartition, isFuture = true).foreach { futureLog => partition.log.foreach { _ => - val leader = new BrokerEndPoint(config.brokerId, "localhost", -1) + val leader = new BrokerEndPoint(config.nodeId, "localhost", -1) // Add future replica log to partition's map if it's not existed if (partition.maybeCreateFutureReplica(futureLog.parentDir, offsetCheckpoints, topicIds(partition.topic))) { @@ -2915,7 +2915,7 @@ class ReplicaManager(val config: KafkaConfig, val isNewLeaderEpoch = partition.makeFollower(state, offsetCheckpoints, Some(info.topicId), partitionAssignedDirectoryId) if (isInControlledShutdown && (info.partition.leader == NO_LEADER || - !info.partition.isr.contains(config.brokerId))) { + !info.partition.isr.contains(config.nodeId))) { // During controlled shutdown, replica with no leaders and replica // where this broker is not in the ISR are stopped. partitionsToStopFetching.put(tp, false) diff --git a/core/src/main/scala/kafka/server/Server.scala b/core/src/main/scala/kafka/server/Server.scala index d85060cc72db2..dfb062e4d3e15 100644 --- a/core/src/main/scala/kafka/server/Server.scala +++ b/core/src/main/scala/kafka/server/Server.scala @@ -73,7 +73,7 @@ object Server { if (config.usesSelfManagedQuorum) { contextLabels.put(NodeIdLabel, config.nodeId.toString) } else { - contextLabels.put(BrokerIdLabel, config.brokerId.toString) + contextLabels.put(BrokerIdLabel, config.nodeId.toString) } contextLabels.putAll(config.originalsWithPrefix(CommonClientConfigs.METRICS_CONTEXT_PREFIX)) diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index 5f43f5294017f..bf828792c137f 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -124,7 +124,7 @@ class TestRaftServer( ) dataPlaneRequestHandlerPool = new KafkaRequestHandlerPool( - config.brokerId, + config.nodeId, socketServer.dataPlaneRequestChannel, requestHandler, time, diff --git a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java index 5731692f98e1d..5efb39e2e4800 100644 --- a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java +++ b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java @@ -32,6 +32,7 @@ import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.config.KRaftConfigs; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.extension.ExtendWith; @@ -258,7 +259,7 @@ public void testAlterReadOnlyConfigInKRaftThenShouldFail() { singletonMap(AUTO_LEADER_REBALANCE_ENABLE_CONFIG, "false"), alterOpts)); assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId), - singletonMap("broker.id", "1"), alterOpts)); + singletonMap(KRaftConfigs.NODE_ID_CONFIG, "1"), alterOpts)); } } @@ -313,7 +314,7 @@ public void testUpdatePerBrokerConfigInKRaftThenShouldFail() { @ClusterTest public void testUpdateInvalidBrokerConfigs() { updateAndCheckInvalidBrokerConfig(Optional.empty()); - updateAndCheckInvalidBrokerConfig(Optional.of(cluster.anyBrokerSocketServer().config().brokerId() + "")); + updateAndCheckInvalidBrokerConfig(Optional.of(cluster.anyBrokerSocketServer().config().nodeId() + "")); } private void updateAndCheckInvalidBrokerConfig(Optional brokerIdOrDefault) { diff --git a/core/src/test/java/kafka/admin/DeleteTopicTest.java b/core/src/test/java/kafka/admin/DeleteTopicTest.java index be87e086f7f5a..d1fe47b05b220 100644 --- a/core/src/test/java/kafka/admin/DeleteTopicTest.java +++ b/core/src/test/java/kafka/admin/DeleteTopicTest.java @@ -100,7 +100,7 @@ public void testResumeDeleteTopicWithRecoveredFollower(ClusterInstance cluster) TestUtils.waitForCondition(() -> cluster.brokers().values() .stream() - .filter(broker -> broker.config().brokerId() != follower.config().brokerId()) + .filter(broker -> broker.config().nodeId() != follower.config().nodeId()) .allMatch(b -> b.logManager().getLog(topicPartition, false).isEmpty()), "Online replicas have not deleted log."); @@ -173,7 +173,7 @@ public void testDeleteTopicDuringAddPartition(ClusterInstance cluster) throws Ex follower.shutdown(); // wait until the broker is in shutting down state - int followerBrokerId = follower.config().brokerId(); + int followerBrokerId = follower.config().nodeId(); TestUtils.waitForCondition(() -> follower.brokerState().equals(BrokerState.SHUTTING_DOWN), "Follower " + followerBrokerId + " was not shutdown"); Map newPartitionSet = Map.of(DEFAULT_TOPIC, NewPartitions.increaseTo(3)); @@ -329,13 +329,13 @@ private Supplier> isLeaderKnown(Map idTo .stream() .filter(broker -> OptionConverters.toJava(broker.replicaManager().onlinePartition(topicPartition)) .stream().anyMatch(tp -> tp.leaderIdIfLocal().isDefined())) - .map(broker -> broker.config().brokerId()) + .map(broker -> broker.config().nodeId()) .findFirst(); } private KafkaBroker findFollower(Collection idToBroker, int leaderId) { return idToBroker.stream() - .filter(broker -> broker.config().brokerId() != leaderId) + .filter(broker -> broker.config().nodeId() != leaderId) .findFirst() .orElseGet(() -> fail("Can't find any follower")); } @@ -354,7 +354,7 @@ private void waitUtilTopicGone(Admin admin) throws Exception { private Map findPartitionHostingBrokers(Map brokers) { return brokers.entrySet() .stream() - .filter(broker -> expectedReplicaAssignment.get(0).contains(broker.getValue().config().brokerId())) + .filter(broker -> expectedReplicaAssignment.get(0).contains(broker.getValue().config().nodeId())) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index b956c12380694..1af4f78393b10 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -40,7 +40,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.common.OffsetAndEpoch; import org.apache.kafka.server.common.StopPartition; -import org.apache.kafka.server.config.ServerConfigs; +import org.apache.kafka.server.config.KRaftConfigs; import org.apache.kafka.server.log.remote.quota.RLMQuotaManager; import org.apache.kafka.server.log.remote.quota.RLMQuotaManagerConfig; import org.apache.kafka.server.log.remote.storage.ClassLoaderAwareRemoteStorageManager; @@ -187,7 +187,7 @@ public class RemoteLogManagerTest { private final Time time = new MockTime(); - private final int brokerId = 0; + private final int nodeId = 0; private final String logDir = TestUtils.tempDirectory("kafka-").toString(); private final String clusterId = "dummyId"; private final String remoteLogStorageTestProp = "remote.log.storage.test"; @@ -242,7 +242,7 @@ void setUp() throws Exception { config = KafkaConfig.fromProps(props); brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig().isRemoteStorageSystemEnabled()); - remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, + remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), nodeId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { @@ -394,7 +394,7 @@ void testRemoteLogMetadataManagerWithEndpointConfig() { assertEquals(host + ":" + port, capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers")); assertEquals(securityProtocol, capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol")); assertEquals(clusterId, capture.getValue().get("cluster.id")); - assertEquals(brokerId, capture.getValue().get(ServerConfigs.BROKER_ID_CONFIG)); + assertEquals(nodeId, capture.getValue().get(KRaftConfigs.NODE_ID_CONFIG)); } @Test @@ -407,7 +407,7 @@ void testRemoteLogMetadataManagerWithEndpointConfigOverridden() throws IOExcepti KafkaConfig config = KafkaConfig.fromProps(props); try (RemoteLogManager remoteLogManager = new RemoteLogManager( config.remoteLogManagerConfig(), - brokerId, + nodeId, logDir, clusterId, time, @@ -436,7 +436,7 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { // should be overridden as SSL assertEquals("SSL", capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol")); assertEquals(clusterId, capture.getValue().get("cluster.id")); - assertEquals(brokerId, capture.getValue().get(ServerConfigs.BROKER_ID_CONFIG)); + assertEquals(nodeId, capture.getValue().get(KRaftConfigs.NODE_ID_CONFIG)); } } @@ -445,11 +445,11 @@ void testStartup() { remoteLogManager.startup(); ArgumentCaptor> capture = ArgumentCaptor.forClass(Map.class); verify(remoteStorageManager, times(1)).configure(capture.capture()); - assertEquals(brokerId, capture.getValue().get("broker.id")); + assertEquals(nodeId, capture.getValue().get(KRaftConfigs.NODE_ID_CONFIG)); assertEquals(remoteLogStorageTestVal, capture.getValue().get(remoteLogStorageTestProp)); verify(remoteLogMetadataManager, times(1)).configure(capture.capture()); - assertEquals(brokerId, capture.getValue().get("broker.id")); + assertEquals(nodeId, capture.getValue().get(KRaftConfigs.NODE_ID_CONFIG)); assertEquals(logDir, capture.getValue().get("log.dir")); // verify the configs starting with "remote.log.metadata", "remote.log.metadata.common.client." @@ -675,7 +675,7 @@ void testCustomMetadataSizeExceedsLimit() throws Exception { // Check we attempt to delete the segment data providing the custom metadata back. RemoteLogSegmentMetadataUpdate expectedMetadataUpdate = new RemoteLogSegmentMetadataUpdate( remoteLogSegmentMetadataArg.getValue().remoteLogSegmentId(), time.milliseconds(), - Optional.of(customMetadata), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, brokerId); + Optional.of(customMetadata), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, nodeId); RemoteLogSegmentMetadata expectedDeleteMetadata = remoteLogSegmentMetadataArg.getValue().createWithUpdates(expectedMetadataUpdate); verify(remoteStorageManager, times(1)).deleteLogSegmentData(eq(expectedDeleteMetadata)); @@ -1326,13 +1326,13 @@ private void verifyRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSe assertEquals(leaderEpochEntries.next(), leaderEpochs.firstEntry()); assertEquals(leaderEpochEntries.next(), leaderEpochs.lastEntry()); - assertEquals(brokerId, remoteLogSegmentMetadata.brokerId()); + assertEquals(nodeId, remoteLogSegmentMetadata.brokerId()); assertEquals(RemoteLogSegmentState.COPY_SEGMENT_STARTED, remoteLogSegmentMetadata.state()); } private void verifyRemoteLogSegmentMetadataUpdate(RemoteLogSegmentMetadataUpdate remoteLogSegmentMetadataUpdate) { assertEquals(leaderTopicIdPartition, remoteLogSegmentMetadataUpdate.remoteLogSegmentId().topicIdPartition()); - assertEquals(brokerId, remoteLogSegmentMetadataUpdate.brokerId()); + assertEquals(nodeId, remoteLogSegmentMetadataUpdate.brokerId()); assertEquals(RemoteLogSegmentState.COPY_SEGMENT_FINISHED, remoteLogSegmentMetadataUpdate.state()); } @@ -1357,7 +1357,7 @@ private void verifyLogSegmentData(LogSegmentData logSegmentData, void testGetClassLoaderAwareRemoteStorageManager() throws Exception { ClassLoaderAwareRemoteStorageManager rsmManager = mock(ClassLoaderAwareRemoteStorageManager.class); try (RemoteLogManager remoteLogManager = - new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, + new RemoteLogManager(config.remoteLogManagerConfig(), nodeId, logDir, clusterId, time, t -> Optional.empty(), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { @@ -1700,12 +1700,12 @@ void testFetchOffsetByTimestampWithTieredStorageDoesNotFetchIndexWhenExistsLocal long timestamp = time.milliseconds(); RemoteLogSegmentMetadata metadata0 = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(tpId, Uuid.randomUuid()), - 0, 99, timestamp, brokerId, timestamp, 1024, Optional.empty(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, truncateAndGetLeaderEpochs(epochEntries, 0L, 99L)); + 0, 99, timestamp, nodeId, timestamp, 1024, Optional.empty(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, truncateAndGetLeaderEpochs(epochEntries, 0L, 99L)); RemoteLogSegmentMetadata metadata1 = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(tpId, Uuid.randomUuid()), - 100, 199, timestamp + 1, brokerId, timestamp + 1, 1024, Optional.empty(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, truncateAndGetLeaderEpochs(epochEntries, 100L, 199L)); + 100, 199, timestamp + 1, nodeId, timestamp + 1, 1024, Optional.empty(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, truncateAndGetLeaderEpochs(epochEntries, 100L, 199L)); // Note that the metadata2 is in COPY_SEGMENT_STARTED state RemoteLogSegmentMetadata metadata2 = new RemoteLogSegmentMetadata(new RemoteLogSegmentId(tpId, Uuid.randomUuid()), - 100, 299, timestamp + 2, brokerId, timestamp + 2, 1024, Optional.empty(), RemoteLogSegmentState.COPY_SEGMENT_STARTED, truncateAndGetLeaderEpochs(epochEntries, 200L, 299L)); + 100, 299, timestamp + 2, nodeId, timestamp + 2, 1024, Optional.empty(), RemoteLogSegmentState.COPY_SEGMENT_STARTED, truncateAndGetLeaderEpochs(epochEntries, 200L, 299L)); when(remoteLogMetadataManager.listRemoteLogSegments(eq(tpId), anyInt())) .thenAnswer(ans -> { @@ -1748,7 +1748,7 @@ void testFetchOffsetByTimestampWithTieredStorageDoesNotFetchIndexWhenExistsLocal }); when(mockLog.logEndOffset()).thenReturn(300L); - remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, + remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), nodeId, logDir, clusterId, time, partition -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { @@ -1806,7 +1806,7 @@ void testIdempotentClose() throws IOException { @Test public void testRemoveMetricsOnClose() throws IOException { try (MockedConstruction mockMetricsGroupCtor = mockConstruction(KafkaMetricsGroup.class)) { - RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, + RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), nodeId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { public RemoteStorageManager createRemoteStorageManager() { @@ -2193,9 +2193,9 @@ public void testFindLogStartOffset() throws RemoteStorageException, IOException int segmentSize = 1024; List segmentMetadataList = Arrays.asList( new RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), - 500, 539, timestamp, brokerId, timestamp, segmentSize, truncateAndGetLeaderEpochs(epochEntries, 500L, 539L)), + 500, 539, timestamp, nodeId, timestamp, segmentSize, truncateAndGetLeaderEpochs(epochEntries, 500L, 539L)), new RemoteLogSegmentMetadata(new RemoteLogSegmentId(leaderTopicIdPartition, Uuid.randomUuid()), - 540, 700, timestamp, brokerId, timestamp, segmentSize, truncateAndGetLeaderEpochs(epochEntries, 540L, 700L)) + 540, 700, timestamp, nodeId, timestamp, segmentSize, truncateAndGetLeaderEpochs(epochEntries, 540L, 700L)) ); when(remoteLogMetadataManager.listRemoteLogSegments(eq(leaderTopicIdPartition), anyInt())) .thenAnswer(invocation -> { @@ -2205,7 +2205,7 @@ public void testFindLogStartOffset() throws RemoteStorageException, IOException else return Collections.emptyIterator(); }); - try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, + try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), nodeId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { @@ -2230,7 +2230,7 @@ public void testFindLogStartOffsetFallbackToLocalLogStartOffsetWhenRemoteIsEmpty when(remoteLogMetadataManager.listRemoteLogSegments(eq(leaderTopicIdPartition), anyInt())) .thenReturn(Collections.emptyIterator()); - try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, + try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), nodeId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) { @@ -2264,7 +2264,7 @@ public void testLogStartOffsetUpdatedOnStartup() throws RemoteStorageException, }); AtomicLong logStartOffset = new AtomicLong(0); - try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, + try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), nodeId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> logStartOffset.set(offset), brokerTopicStats, metrics) { @@ -2318,7 +2318,7 @@ public synchronized Iterator listRemoteLogSegments(Top } }; - remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, + remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), nodeId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { @@ -2972,7 +2972,7 @@ private void verifyDeleteLogSegment(List segmentMetada @Test public void testDeleteRetentionMsOnExpiredSegment() throws RemoteStorageException, IOException { AtomicLong logStartOffset = new AtomicLong(0); - try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, + try (RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), nodeId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> logStartOffset.set(offset), brokerTopicStats, metrics) { @@ -3061,7 +3061,7 @@ private List listRemoteLogSegmentMetadataByTime(TopicI startOffset, endOffset, timestamp, - brokerId, + nodeId, timestamp, segmentSize, Optional.empty(), @@ -3084,7 +3084,7 @@ private RemoteLogSegmentMetadata createRemoteLogSegmentMetadata(RemoteLogSegment startOffset, endOffset, time.milliseconds(), - brokerId, + nodeId, time.milliseconds(), segmentSize, Optional.empty(), @@ -3134,7 +3134,7 @@ public void testReadForMissingFirstBatchInRemote() throws RemoteStorageException try (RemoteLogManager remoteLogManager = new RemoteLogManager( config.remoteLogManagerConfig(), - brokerId, + nodeId, logDir, clusterId, time, @@ -3211,7 +3211,7 @@ public void testReadForFirstBatchMoreThanMaxFetchBytes(boolean minOneMessage) th try (RemoteLogManager remoteLogManager = new RemoteLogManager( config.remoteLogManagerConfig(), - brokerId, + nodeId, logDir, clusterId, time, @@ -3296,7 +3296,7 @@ public void testReadForFirstBatchInLogCompaction() throws RemoteStorageException try (RemoteLogManager remoteLogManager = new RemoteLogManager( config.remoteLogManagerConfig(), - brokerId, + nodeId, logDir, clusterId, time, @@ -3651,7 +3651,7 @@ public void testRemoteReadFetchDataInfo() throws RemoteStorageException, IOExcep when(remoteStorageManager.fetchLogSegment(any(RemoteLogSegmentMetadata.class), anyInt())) .thenReturn(fileInputStream); - RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, + RemoteLogManager remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), nodeId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { @@ -3694,7 +3694,7 @@ public void testRLMOpsWhenMetadataIsNotReady() throws InterruptedException, IOEx appendRLMConfig(props); config = KafkaConfig.fromProps(props); - remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), brokerId, logDir, clusterId, time, + remoteLogManager = new RemoteLogManager(config.remoteLogManagerConfig(), nodeId, logDir, clusterId, time, tp -> Optional.of(mockLog), (topicPartition, offset) -> currentLogStartOffset.set(offset), brokerTopicStats, metrics) { diff --git a/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java b/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java index daa0aacca7de4..c2e4c1c979f80 100644 --- a/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java +++ b/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java @@ -71,7 +71,7 @@ void testValidatesConfigOnMetadataChange() throws Exception { KafkaConfig config = mock(KafkaConfig.class); FaultHandler faultHandler = mock(FaultHandler.class); - when(config.brokerId()).thenReturn(8); + when(config.nodeId()).thenReturn(8); testWith(metadataVersion, config, faultHandler); @@ -88,7 +88,7 @@ void testInvokesFaultHandlerOnException() throws Exception { FaultHandler faultHandler = mock(FaultHandler.class); when(faultHandler.handleFault(any(), any())).thenReturn(new RuntimeException("returned exception")); - when(config.brokerId()).thenReturn(8); + when(config.nodeId()).thenReturn(8); willAnswer(invocation -> { throw exception; }).given(config).validateWithMetadataVersion(eq(metadataVersion)); diff --git a/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala b/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala index 37348c0657862..91d2bb9440a6f 100644 --- a/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala @@ -181,7 +181,7 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness { .allTopicNames().get().get(topic).partitions().get(0).leader().id() val previousLeader = leader() - val newLeader = brokers.map(_.config.brokerId).find(_ != previousLeader).get + val newLeader = brokers.map(_.config.nodeId).find(_ != previousLeader).get // change the leader to new one adminClient.alterPartitionReassignments(java.util.Collections.singletonMap(new TopicPartition(topic, 0), @@ -200,7 +200,7 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness { // case 2: test the offsets from recovery path. // server will rebuild offset index according to log files if the index files are nonexistent val indexFiles = brokers.flatMap(_.config.logDirs).toSet - brokers.foreach(b => killBroker(b.config.brokerId)) + brokers.foreach(b => killBroker(b.config.nodeId)) indexFiles.foreach { root => val files = new File(s"$root/$topic-0").listFiles() if (files != null) files.foreach { f => diff --git a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala index dc836352787b2..40092a5751f1e 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuild import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.authorizer.StandardAuthorizer -import org.apache.kafka.server.config.ServerConfigs +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs} import org.junit.jupiter.api.{BeforeEach, TestInfo} import java.util.Properties @@ -93,7 +93,7 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group) override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(ServerConfigs.BROKER_ID_CONFIG, brokerId.toString) + properties.put(KRaftConfigs.NODE_ID_CONFIG, brokerId.toString) addNodeProperties(properties) } diff --git a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala index b11cb96ef8ce0..a8c0f8fb5619f 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala @@ -126,7 +126,7 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with createTopic(topic3) // Set a mutable broker config - val brokerResource = new ConfigResource(ConfigResource.Type.BROKER, brokers.head.config.brokerId.toString) + val brokerResource = new ConfigResource(ConfigResource.Type.BROKER, brokers.head.config.nodeId.toString) var alterResult = client.incrementalAlterConfigs(Collections.singletonMap(brokerResource, util.Arrays.asList(new AlterConfigOp(new ConfigEntry(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "50000"), OpType.SET)))) alterResult.all.get diff --git a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala index 3f5fa5c0457fc..b84f8d6e26e88 100644 --- a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala @@ -84,8 +84,8 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { val numPartitions = 1 val leaders = createTopic(topic1, numPartitions, brokerCount, adminClientConfig = adminClientConfig) - leaderNode = if (leaders(0) == brokers.head.config.brokerId) brokers.head else brokers(1) - followerNode = if (leaders(0) != brokers.head.config.brokerId) brokers.head else brokers(1) + leaderNode = if (leaders(0) == brokers.head.config.nodeId) brokers.head else brokers(1) + followerNode = if (leaders(0) != brokers.head.config.nodeId) brokers.head else brokers(1) quotaTestClients = createQuotaTestClients(topic1, leaderNode) } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index a8dbe0ecdaa5d..fe35600b8f3f4 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -223,7 +223,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { receiveExactRecords(poller, numRecords, 10000) poller.shutdown() - brokerServers.foreach(server => killBroker(server.config.brokerId)) + brokerServers.foreach(server => killBroker(server.config.nodeId)) Thread.sleep(500) restartDeadBrokers() @@ -306,7 +306,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { this.consumerConfig.setProperty(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, requestTimeout.toString) val consumer2 = createConsumerAndReceive(group2, manualAssign = true, numRecords) - brokerServers.foreach(server => killBroker(server.config.brokerId)) + brokerServers.foreach(server => killBroker(server.config.nodeId)) val closeTimeout = 2000 val future1 = submitCloseAndValidate(consumer1, closeTimeout, None, Some(closeTimeout)) val future2 = submitCloseAndValidate(consumer2, Long.MaxValue, None, Some(requestTimeout)) @@ -341,7 +341,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { for (serverIdx <- brokerServers.indices) { killBroker(serverIdx) val config = newConfigs(serverIdx) - servers(serverIdx) = createBroker(config, time = brokerTime(config.brokerId)) + servers(serverIdx) = createBroker(config, time = brokerTime(config.nodeId)) restartDeadBrokers() } @@ -452,7 +452,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { // Trigger another rebalance and shutdown all brokers // This consumer poll() doesn't complete and `tearDown` shuts down the executor and closes the consumer createConsumerToRebalance() - brokerServers.foreach(server => killBroker(server.config.brokerId)) + brokerServers.foreach(server => killBroker(server.config.nodeId)) // consumer2 should close immediately without LeaveGroup request since there are no brokers available val closeFuture2 = submitCloseAndValidate(consumer2, Long.MaxValue, None, Some(0)) diff --git a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala index 4d443a7f9b7af..95df872418287 100644 --- a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala +++ b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth._ import org.apache.kafka.common.{Cluster, Reconfigurable} -import org.apache.kafka.server.config.{QuotaConfig, ServerConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, QuotaConfig, ServerConfigs} import org.apache.kafka.server.quota._ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo} @@ -371,7 +371,7 @@ class GroupedUserQuotaCallback extends ClientQuotaCallback with Reconfigurable w val partitionRatio = new ConcurrentHashMap[String, Double]() override def configure(configs: util.Map[String, _]): Unit = { - brokerId = configs.get(ServerConfigs.BROKER_ID_CONFIG).toString.toInt + brokerId = configs.get(KRaftConfigs.NODE_ID_CONFIG).toString.toInt callbackInstances.incrementAndGet } diff --git a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala index e9a0644a26c63..d558332766439 100644 --- a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST -import org.apache.kafka.server.config.ServerConfigs +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.{BeforeEach, TestInfo} @@ -77,7 +77,7 @@ class GroupAuthorizerIntegrationTest extends BaseRequestTest { } override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(ServerConfigs.BROKER_ID_CONFIG, brokerId.toString) + properties.put(KRaftConfigs.NODE_ID_CONFIG, brokerId.toString) addNodeProperties(properties) } diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 0f4174e1250d2..c6440aa1d3aa5 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -53,7 +53,7 @@ import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEX import org.apache.kafka.coordinator.group.{GroupConfig, GroupCoordinatorConfig} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.{QuotaConfig, ServerConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, QuotaConfig, ServerConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogFileUtils} import org.apache.kafka.test.TestUtils.{DEFAULT_MAX_WAIT_MS, assertFutureThrows} import org.apache.logging.log4j.core.config.Configurator @@ -90,7 +90,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { super.setUp(testInfo) Configurator.reconfigure(); brokerLoggerConfigResource = new ConfigResource( - ConfigResource.Type.BROKER_LOGGER, brokers.head.config.brokerId.toString) + ConfigResource.Type.BROKER_LOGGER, brokers.head.config.nodeId.toString) } @ParameterizedTest @@ -103,8 +103,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { try { // Describe and broker - val brokerResource1 = new ConfigResource(ConfigResource.Type.BROKER, brokers(1).config.brokerId.toString) - val brokerResource2 = new ConfigResource(ConfigResource.Type.BROKER, brokers(2).config.brokerId.toString) + val brokerResource1 = new ConfigResource(ConfigResource.Type.BROKER, brokers(1).config.nodeId.toString) + val brokerResource2 = new ConfigResource(ConfigResource.Type.BROKER, brokers(2).config.nodeId.toString) val configResources = Seq(brokerResource1, brokerResource2) val exception = assertThrows(classOf[ExecutionException], () => { @@ -591,7 +591,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("kraft")) def testListNodesWithFencedBroker(quorum: String): Unit = { client = createAdminClient - val fencedBrokerId = brokers.last.config.brokerId + val fencedBrokerId = brokers.last.config.nodeId killBroker(fencedBrokerId, JDuration.ofMillis(0)) // It takes a few seconds for a broker to get fenced after being killed // So we retry until only 2 of 3 brokers returned in the result or the max wait is reached @@ -790,7 +790,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val controller = result.controller().get() // In KRaft, we return a random brokerId as the current controller. - val brokerIds = brokers.map(_.config.brokerId).toSet + val brokerIds = brokers.map(_.config.nodeId).toSet assertTrue(brokerIds.contains(controller.id)) val brokerEndpoints = bootstrapServers().split(",") @@ -815,7 +815,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val logDirInfosByBroker = client.describeLogDirs(brokerIds.asJava).allDescriptions.get (0 until brokerCount).foreach { brokerId => - val server = brokers.find(_.config.brokerId == brokerId).get + val server = brokers.find(_.config.nodeId == brokerId).get val expectedPartitions = partitionsByBroker(brokerId) val logDirInfos = logDirInfosByBroker.get(brokerId) val replicaInfos = logDirInfos.asScala.flatMap { case (_, logDirInfo) => @@ -846,7 +846,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val replicaDirInfos = client.describeReplicaLogDirs(replicas.asJavaCollection).all.get replicaDirInfos.forEach { (topicPartitionReplica, replicaDirInfo) => - val server = brokers.find(_.config.brokerId == topicPartitionReplica.brokerId()).get + val server = brokers.find(_.config.nodeId == topicPartitionReplica.brokerId()).get val tp = new TopicPartition(topicPartitionReplica.topic(), topicPartitionReplica.partition()) assertEquals(server.logManager.getLog(tp).get.dir.getParent, replicaDirInfo.getCurrentReplicaLogDir) } @@ -863,11 +863,11 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { // Generate two mutually exclusive replicaAssignment val firstReplicaAssignment = brokers.map { server => val logDir = new File(server.config.logDirs(randomNums(server))).getAbsolutePath - new TopicPartitionReplica(topic, 0, server.config.brokerId) -> logDir + new TopicPartitionReplica(topic, 0, server.config.nodeId) -> logDir }.toMap val secondReplicaAssignment = brokers.map { server => val logDir = new File(server.config.logDirs(1 - randomNums(server))).getAbsolutePath - new TopicPartitionReplica(topic, 0, server.config.brokerId) -> logDir + new TopicPartitionReplica(topic, 0, server.config.nodeId) -> logDir }.toMap // Verify that replica can be created in the specified log directory @@ -882,7 +882,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { ensureConsistentKRaftMetadata() brokers.foreach { server => val logDir = server.logManager.getLog(tp).get.dir.getParent - assertEquals(firstReplicaAssignment(new TopicPartitionReplica(topic, 0, server.config.brokerId)), logDir) + assertEquals(firstReplicaAssignment(new TopicPartitionReplica(topic, 0, server.config.nodeId)), logDir) } // Verify that replica can be moved to the specified log directory after the topic has been created @@ -890,7 +890,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { brokers.foreach { server => TestUtils.waitUntilTrue(() => { val logDir = server.logManager.getLog(tp).get.dir.getParent - secondReplicaAssignment(new TopicPartitionReplica(topic, 0, server.config.brokerId)) == logDir + secondReplicaAssignment(new TopicPartitionReplica(topic, 0, server.config.nodeId)) == logDir }, "timed out waiting for replica movement") } @@ -923,7 +923,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { brokers.foreach { server => TestUtils.waitUntilTrue(() => { val logDir = server.logManager.getLog(tp).get.dir.getParent - firstReplicaAssignment(new TopicPartitionReplica(topic, 0, server.config.brokerId)) == logDir + firstReplicaAssignment(new TopicPartitionReplica(topic, 0, server.config.nodeId)) == logDir }, s"timed out waiting for replica movement. Producer future ${producerFuture.value}") } @@ -997,8 +997,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { createTopic(topic2) // Describe topics and broker - val brokerResource1 = new ConfigResource(ConfigResource.Type.BROKER, brokers(1).config.brokerId.toString) - val brokerResource2 = new ConfigResource(ConfigResource.Type.BROKER, brokers(2).config.brokerId.toString) + val brokerResource1 = new ConfigResource(ConfigResource.Type.BROKER, brokers(1).config.nodeId.toString) + val brokerResource2 = new ConfigResource(ConfigResource.Type.BROKER, brokers(2).config.nodeId.toString) val configResources = Seq(topicResource1, topicResource2, brokerResource1, brokerResource2) val describeResult = client.describeConfigs(configResources.asJava) val configs = describeResult.all.get @@ -1033,7 +1033,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { }) assertEquals(brokers(1).config.nonInternalValues.size + numInternalConfigsSet, configs.get(brokerResource1).entries.size) - assertEquals(brokers(1).config.brokerId.toString, configs.get(brokerResource1).get(ServerConfigs.BROKER_ID_CONFIG).value) + assertEquals(brokers(1).config.nodeId.toString, configs.get(brokerResource1).get(KRaftConfigs.NODE_ID_CONFIG).value) val listenerSecurityProtocolMap = configs.get(brokerResource1).get(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG) assertEquals(brokers(1).config.getString(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG), listenerSecurityProtocolMap.value) assertEquals(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, listenerSecurityProtocolMap.name) @@ -1055,7 +1055,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { assertEquals(brokers(2).config.nonInternalValues.size + numInternalConfigsSet, configs.get(brokerResource2).entries.size) - assertEquals(brokers(2).config.brokerId.toString, configs.get(brokerResource2).get(ServerConfigs.BROKER_ID_CONFIG).value) + assertEquals(brokers(2).config.nodeId.toString, configs.get(brokerResource2).get(KRaftConfigs.NODE_ID_CONFIG).value) assertEquals(brokers(2).config.logCleanerThreads.toString, configs.get(brokerResource2).get(CleanerConfig.LOG_CLEANER_THREADS_PROP).value) @@ -1434,7 +1434,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { @ValueSource(strings = Array("kraft")) def testReplicaCanFetchFromLogStartOffsetAfterDeleteRecords(quorum: String): Unit = { val leaders = createTopic(topic, replicationFactor = brokerCount) - val followerIndex = if (leaders(0) != brokers.head.config.brokerId) 0 else 1 + val followerIndex = if (leaders(0) != brokers.head.config.nodeId) 0 else 1 def waitForFollowerLog(expectedStartOffset: Long, expectedEndOffset: Long): Unit = { TestUtils.waitUntilTrue(() => brokers(followerIndex).replicaManager.localLog(topicPartition).isDefined, @@ -1499,7 +1499,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { // we will create another dir just for one server val futureLogDir = brokers(0).config.logDirs(1) - val futureReplica = new TopicPartitionReplica(topic, 0, brokers(0).config.brokerId) + val futureReplica = new TopicPartitionReplica(topic, 0, brokers(0).config.nodeId) // Verify that replica can be moved to the specified log directory client.alterReplicaLogDirs(Map(futureReplica -> futureLogDir).asJava).all.get @@ -3359,8 +3359,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { val topic = "incremental-alter-configs-topic" val topicResource = new ConfigResource(ConfigResource.Type.TOPIC, topic) - val appendValues = s"0:${brokers.head.config.brokerId}" - val subtractValues = brokers.tail.map(broker => s"0:${broker.config.brokerId}").mkString(",") + val appendValues = s"0:${brokers.head.config.nodeId}" + val subtractValues = brokers.tail.map(broker => s"0:${broker.config.nodeId}").mkString(",") assertNotEquals("", subtractValues) val topicCreateConfigs = new Properties @@ -4073,7 +4073,7 @@ object PlaintextAdminIntegrationTest { val topicResource2 = new ConfigResource(ConfigResource.Type.TOPIC, topic2) createTopicWithAdmin(admin, topic2, test.brokers, test.controllerServers, numPartitions = 1, replicationFactor = 1) - val brokerResource = new ConfigResource(ConfigResource.Type.BROKER, test.brokers.head.config.brokerId.toString) + val brokerResource = new ConfigResource(ConfigResource.Type.BROKER, test.brokers.head.config.nodeId.toString) // Alter configs: first and third are invalid, second is valid val alterConfigs = new util.HashMap[ConfigResource, util.Collection[AlterConfigOp]]() diff --git a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala index a6cd0d905decd..f7d2cf0826694 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala @@ -186,13 +186,13 @@ class TransactionsBounceTest extends IntegrationTestHarness { private class BounceScheduler extends ShutdownableThread("daemon-broker-bouncer", false) { override def doWork(): Unit = { for (server <- brokers) { - trace("Shutting down server : %s".format(server.config.brokerId)) + trace("Shutting down server : %s".format(server.config.nodeId)) server.shutdown() server.awaitShutdown() Thread.sleep(500) - trace("Server %s shut down. Starting it up again.".format(server.config.brokerId)) + trace("Server %s shut down. Starting it up again.".format(server.config.nodeId)) server.startup() - trace("Restarted server: %s".format(server.config.brokerId)) + trace("Restarted server: %s".format(server.config.nodeId)) Thread.sleep(500) } diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index 66ff64f2cdc03..5d74d38458ba2 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -999,7 +999,7 @@ class TransactionsTest extends IntegrationTestHarness { partitionStartOffsets.forall { case (partition, offset) => { val lso = broker.replicaManager.localLog(partition).get.logStartOffset - offsets.put(broker.config.brokerId, lso) + offsets.put(broker.config.nodeId, lso) offset == lso } } diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 49a0ebc21f4bf..dddb364a9e96f 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -57,7 +57,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.network.SocketServerConfigs -import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms} +import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms} import org.apache.kafka.server.metrics.{KafkaYammerMetrics, MetricConfigs} import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.ShutdownableThread @@ -717,8 +717,8 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup val partitionInfo0 = partitionInfo assertEquals(partitionInfo0.replicas.get(0), partitionInfo0.leader) - val leaderBroker = servers.find(_.config.brokerId == partitionInfo0.replicas.get(0).id).get - val followerBroker = servers.find(_.config.brokerId == partitionInfo0.replicas.get(1).id).get + val leaderBroker = servers.find(_.config.nodeId == partitionInfo0.replicas.get(0).id).get + val followerBroker = servers.find(_.config.nodeId == partitionInfo0.replicas.get(1).id).get // Stop follower followerBroker.shutdown() @@ -745,7 +745,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup // Verify that the old follower with missing records is elected as the new leader val (newLeader, elected) = TestUtils.computeUntilTrue(partitionInfo.leader)(leader => leader != null) assertTrue(elected, "Unclean leader not elected") - assertEquals(followerBroker.config.brokerId, newLeader.id) + assertEquals(followerBroker.config.nodeId, newLeader.id) // New leader doesn't have the last 10 records committed on the old leader that have already been consumed. // With unclean leader election enabled, we should be able to produce to the new leader. The first 10 records @@ -928,7 +928,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup reporter.verifyMetricValue("request-total", "socket-server-metrics") } } - assertEquals(Set(controllerServer.config.nodeId) ++ servers.map(_.config.brokerId), + assertEquals(Set(controllerServer.config.nodeId) ++ servers.map(_.config.nodeId), TestMetricsReporter.configuredBrokers.toSet) // non-default value to trigger a new metric @@ -1009,7 +1009,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup val client = adminClients.head val broker = servers.head assertEquals(2, broker.config.dynamicConfig.reconfigurables.asScala.count(r => r.isInstanceOf[DataPlaneAcceptor])) - val broker0Resource = new ConfigResource(ConfigResource.Type.BROKER, broker.config.brokerId.toString) + val broker0Resource = new ConfigResource(ConfigResource.Type.BROKER, broker.config.nodeId.toString) def acceptors: Seq[DataPlaneAcceptor] = broker.config.dynamicConfig.reconfigurables.asScala.filter(_.isInstanceOf[DataPlaneAcceptor]) .map(_.asInstanceOf[DataPlaneAcceptor]).toSeq @@ -1115,7 +1115,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup private def describeConfig(adminClient: Admin, servers: Seq[KafkaBroker] = this.servers): Config = { val configResources = servers.map { server => - new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString) + new ConfigResource(ConfigResource.Type.BROKER, server.config.nodeId.toString) } val describeOptions = new DescribeConfigsOptions().includeSynonyms(true) val describeResult = adminClient.describeConfigs(configResources.asJava, describeOptions).all.get @@ -1178,7 +1178,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup private def alterConfigsOnServer(server: KafkaBroker, props: Properties): Unit = { val configEntries = props.asScala.map { case (k, v) => new AlterConfigOp(new ConfigEntry(k, v), OpType.SET) }.toList.asJava val alterConfigs = new java.util.HashMap[ConfigResource, java.util.Collection[AlterConfigOp]]() - alterConfigs.put(new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString), configEntries) + alterConfigs.put(new ConfigResource(ConfigResource.Type.BROKER, server.config.nodeId.toString), configEntries) adminClients.head.incrementalAlterConfigs(alterConfigs) props.asScala.foreach { case (k, v) => waitForConfigOnServer(server, k, v) } } @@ -1188,7 +1188,7 @@ val configEntries = props.asScala.map { case (k, v) => new AlterConfigOp(new Con val configEntries = props.asScala.map { case (k, v) => new AlterConfigOp(new ConfigEntry(k, v), OpType.SET) }.toList.asJava val configs = if (perBrokerConfig) { val alterConfigs = new java.util.HashMap[ConfigResource, java.util.Collection[AlterConfigOp]]() - servers.foreach(server => alterConfigs.put(new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString), configEntries)) + servers.foreach(server => alterConfigs.put(new ConfigResource(ConfigResource.Type.BROKER, server.config.nodeId.toString), configEntries)) alterConfigs } else { val alterConfigs = new java.util.HashMap[ConfigResource, java.util.Collection[AlterConfigOp]]() @@ -1203,7 +1203,7 @@ val configEntries = props.asScala.map { case (k, v) => new AlterConfigOp(new Con if (expectFailure) { val oldProps = servers.head.config.values.asScala.filter { case (k, _) => newProps.containsKey(k) } val brokerResources = if (perBrokerConfig) - servers.map(server => new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString)) + servers.map(server => new ConfigResource(ConfigResource.Type.BROKER, server.config.nodeId.toString)) else { Seq(new ConfigResource(ConfigResource.Type.BROKER, "")) } @@ -1310,7 +1310,7 @@ val configEntries = props.asScala.map { case (k, v) => new AlterConfigOp(new Con "--command-config", propsFile.getAbsolutePath, "--alter", "--add-config", props.asScala.map { case (k, v) => s"$k=$v" }.mkString(","), "--entity-type", "brokers", - "--entity-name", server.config.brokerId.toString) + "--entity-name", server.config.nodeId.toString) ConfigCommand.main(args) } } @@ -1517,7 +1517,7 @@ class TestMetricsReporter extends MetricsReporter with Reconfigurable with Close } override def configure(configs: util.Map[String, _]): Unit = { - configuredBrokers += configs.get(ServerConfigs.BROKER_ID_CONFIG).toString.toInt + configuredBrokers += configs.get(KRaftConfigs.NODE_ID_CONFIG).toString.toInt configureCount += 1 pollingInterval = configs.get(PollingIntervalProp).toString.toInt } diff --git a/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala b/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala index a9961c7c48225..1b2371d748c5f 100644 --- a/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala @@ -199,7 +199,7 @@ class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest { val consumers = brokers.map { server => consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") consumerConfig.setProperty(ConsumerConfig.CLIENT_RACK_CONFIG, server.config.rack.orNull) - consumerConfig.setProperty(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, s"instance-${server.config.brokerId}") + consumerConfig.setProperty(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, s"instance-${server.config.nodeId}") consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000") createConsumer() } diff --git a/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala b/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala index db2d570b761b7..dc926d4e23859 100644 --- a/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala +++ b/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala @@ -110,10 +110,10 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends QuorumT } servers.map(_.config).foreach { config => - assertEquals(4, config.listeners.size, s"Unexpected listener count for broker ${config.brokerId}") + assertEquals(4, config.listeners.size, s"Unexpected listener count for broker ${config.nodeId}") // KAFKA-5184 seems to show that this value can sometimes be PLAINTEXT, so verify it here assertEquals(Internal, config.interBrokerListenerName.value, - s"Unexpected ${ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG} for broker ${config.brokerId}") + s"Unexpected ${ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG} for broker ${config.nodeId}") } val adminClientConfig = new java.util.HashMap[String, Object]() diff --git a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala index 25e32ca4d79fb..14e39a111f034 100644 --- a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala +++ b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala @@ -69,7 +69,7 @@ class LocalLeaderEndPointTest extends Logging { scheduler = new MockScheduler(time), logManager = mockLogMgr, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterPartitionManager) val partition = replicaManager.createPartition(topicPartition) diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala index 3eecdfe65e190..f0f9cf0154a4f 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorConcurrencyTest.scala @@ -76,8 +76,8 @@ class GroupCoordinatorConcurrencyTest extends AbstractCoordinatorConcurrencyTest val config = KafkaConfig.fromProps(serverProps) - heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.brokerId, 1000, false, true) - rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", timer, config.brokerId, 1000, false, true) + heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.nodeId, 1000, false, true) + rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", timer, config.nodeId, 1000, false, true) metrics = new Metrics groupCoordinator = GroupCoordinator(config, replicaManager, heartbeatPurgatory, rebalancePurgatory, timer.time, metrics) diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala index 2373d09816cb4..5265cb67be94e 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -113,8 +113,8 @@ class GroupCoordinatorTest { val config = KafkaConfig.fromProps(props) - val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, 1000, config.brokerId, false, true) - val rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", timer, 1000, config.brokerId, false, true) + val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, 1000, config.nodeId, false, true) + val rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", timer, 1000, config.nodeId, false, true) groupCoordinator = GroupCoordinator(config, replicaManager, heartbeatPurgatory, rebalancePurgatory, timer.time, new Metrics()) // make two partitions of the group topic to make sure some partitions are not owned by the coordinator diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 0fbf014374839..8a7bc9a7aac6f 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -92,7 +92,7 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness { instanceConfigs } - def serverForId(id: Int): Option[KafkaBroker] = brokers.find(s => s.config.brokerId == id) + def serverForId(id: Int): Option[KafkaBroker] = brokers.find(s => s.config.nodeId == id) def boundPort(server: KafkaBroker): Int = server.boundPort(listenerName) @@ -347,11 +347,11 @@ abstract class KafkaServerTestHarness extends QuorumTestHarness { } private def createBrokerFromConfig(config: KafkaConfig): KafkaBroker = { - createBroker(config, brokerTime(config.brokerId), startup = false) + createBroker(config, brokerTime(config.nodeId), startup = false) } def aliveBrokers: Seq[KafkaBroker] = { - _brokers.filter(broker => alive(broker.config.brokerId)).toSeq + _brokers.filter(broker => alive(broker.config.nodeId)).toSeq } def ensureConsistentKRaftMetadata(): Unit = { diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index b4d3bee35d171..6fdace1e41070 100755 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -206,7 +206,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { assertEquals(List("first"), consumeAllMessages(topic, 1)) // shutdown follower server - brokers.filter(broker => broker.config.brokerId == followerId).map(broker => shutdownBroker(broker)) + brokers.filter(broker => broker.config.nodeId == followerId).map(broker => shutdownBroker(broker)) produceMessage(brokers, topic, "second") assertEquals(List("first", "second"), consumeAllMessages(topic, 2)) @@ -217,8 +217,8 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { assertEquals(0, uncleanLeaderElectionsPerSec) // shutdown leader and then restart follower - brokers.filter(_.config.brokerId == leaderId).map(shutdownBroker) - val followerBroker = brokers.find(_.config.brokerId == followerId).get + brokers.filter(_.config.nodeId == leaderId).map(shutdownBroker) + val followerBroker = brokers.find(_.config.nodeId == followerId).get followerBroker.startup() // wait until new leader is (uncleanly) elected @@ -248,7 +248,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { assertEquals(List("first"), consumeAllMessages(topic, 1)) // shutdown follower server - brokers.filter(broker => broker.config.brokerId == followerId).map(broker => shutdownBroker(broker)) + brokers.filter(broker => broker.config.nodeId == followerId).map(broker => shutdownBroker(broker)) produceMessage(brokers, topic, "second") assertEquals(List("first", "second"), consumeAllMessages(topic, 2)) @@ -259,8 +259,8 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { assertEquals(0, uncleanLeaderElectionsPerSec) // shutdown leader and then restart follower - brokers.filter(_.config.brokerId == leaderId).map(shutdownBroker) - val followerServer = brokers.find(_.config.brokerId == followerId).get + brokers.filter(_.config.nodeId == leaderId).map(shutdownBroker) + val followerServer = brokers.find(_.config.nodeId == followerId).get followerServer.startup() // verify that unclean election to non-ISR follower does not occur. @@ -276,7 +276,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { assertEquals(List.empty[String], consumeAllMessages(topic, 0)) // restart leader temporarily to send a successfully replicated message - brokers.find(_.config.brokerId == leaderId).get.startup() + brokers.find(_.config.nodeId == leaderId).get.startup() awaitLeaderChange(brokers, topicPartition, expectedLeaderOpt = Some(leaderId)) produceMessage(brokers, topic, "third") @@ -286,7 +286,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { partitionInfoOpt.isDefined && partitionInfoOpt.get.isr.contains(followerId) }, "Inconsistent metadata after first server startup") - brokers.filter(_.config.brokerId == leaderId).map(shutdownBroker) + brokers.filter(_.config.nodeId == leaderId).map(shutdownBroker) // verify clean leader transition to ISR follower awaitLeaderChange(brokers, topicPartition, expectedLeaderOpt = Some(followerId)) @@ -360,7 +360,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { } // shutdown follower server - brokers.filter(broker => broker.config.brokerId == followerId).map(broker => shutdownBroker(broker)) + brokers.filter(broker => broker.config.nodeId == followerId).map(broker => shutdownBroker(broker)) produceMessage(brokers, topic, "second") assertEquals(List("first", "second"), consumeAllMessages(topic, 2)) @@ -371,8 +371,8 @@ class UncleanLeaderElectionTest extends QuorumTestHarness { assertEquals(0, uncleanLeaderElectionsPerSec) // shutdown leader and then restart follower - brokers.filter(_.config.brokerId == leaderId).map(shutdownBroker) - val followerBroker = brokers.find(_.config.brokerId == followerId).get + brokers.filter(_.config.nodeId == leaderId).map(shutdownBroker) + val followerBroker = brokers.find(_.config.nodeId == followerId).get followerBroker.startup() // verify that unclean election to non-ISR follower does not occur. diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index edbc8db0fb2e0..b45237c555ca7 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -2074,7 +2074,7 @@ class UnifiedLogTest { @Test def testFetchOffsetByTimestampFromRemoteStorage(): Unit = { val config: KafkaConfig = createKafkaConfigWithRLM - val purgatory = new DelayedOperationPurgatory[DelayedRemoteListOffsets]("RemoteListOffsets", config.brokerId) + val purgatory = new DelayedOperationPurgatory[DelayedRemoteListOffsets]("RemoteListOffsets", config.nodeId) val remoteLogManager = spy(new RemoteLogManager(config.remoteLogManagerConfig, 0, logDir.getAbsolutePath, @@ -2171,7 +2171,7 @@ class UnifiedLogTest { @Test def testFetchLatestTieredTimestampWithRemoteStorage(): Unit = { val config: KafkaConfig = createKafkaConfigWithRLM - val purgatory = new DelayedOperationPurgatory[DelayedRemoteListOffsets]("RemoteListOffsets", config.brokerId) + val purgatory = new DelayedOperationPurgatory[DelayedRemoteListOffsets]("RemoteListOffsets", config.nodeId) val remoteLogManager = spy(new RemoteLogManager(config.remoteLogManagerConfig, 0, logDir.getAbsolutePath, diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index b1bc03b6ff479..8683774a5ccf1 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -186,7 +186,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { val topicPartition = new TopicPartition(topic, 0) brokers.foreach { broker => val log = broker.logManager.getLog(new TopicPartition(topic, 0)) - val brokerId = broker.config.brokerId + val brokerId = broker.config.nodeId val logSize = log.map(_.size) assertTrue(logSize.exists(_ > 0), s"Expected broker $brokerId to have a Log for $topicPartition with positive size, actual: $logSize") } diff --git a/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala index b0f0f74e88dc8..4c2b07b840bbf 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala @@ -24,7 +24,7 @@ import org.apache.kafka.common.message.MetadataRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.assertEquals abstract class AbstractMetadataRequestTest extends BaseRequestTest { @@ -32,7 +32,7 @@ abstract class AbstractMetadataRequestTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") - properties.setProperty(ServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(ServerConfigs.BROKER_ID_CONFIG)}") + properties.setProperty(ServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(KRaftConfigs.NODE_ID_CONFIG)}") } protected def requestData(topics: List[String], allowAutoTopicCreation: Boolean): MetadataRequestData = { diff --git a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala index 1b35f93961946..2b4743f3ee834 100644 --- a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala @@ -92,7 +92,7 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest { AddPartitionsToTxnRequest.Builder.forBroker(transactions).build(version) } - val leaderId = brokers.head.config.brokerId + val leaderId = brokers.head.config.nodeId val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(leaderId)) val errors = @@ -172,10 +172,10 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest { private def setUpTransactions(transactionalId: String, verifyOnly: Boolean, partitions: Set[TopicPartition]): (Int, AddPartitionsToTxnTransaction) = { val findCoordinatorRequest = new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData().setKey(transactionalId).setKeyType(CoordinatorType.TRANSACTION.id)).build() // First find coordinator request creates the state topic, then wait for transactional topics to be created. - connectAndReceive[FindCoordinatorResponse](findCoordinatorRequest, brokerSocketServer(brokers.head.config.brokerId)) + connectAndReceive[FindCoordinatorResponse](findCoordinatorRequest, brokerSocketServer(brokers.head.config.nodeId)) TestUtils.waitForAllPartitionsMetadata(brokers, "__transaction_state", 50) TestUtils.ensureConsistentKRaftMetadata(brokers, controllerServer) - val findCoordinatorResponse = connectAndReceive[FindCoordinatorResponse](findCoordinatorRequest, brokerSocketServer(brokers.head.config.brokerId)) + val findCoordinatorResponse = connectAndReceive[FindCoordinatorResponse](findCoordinatorRequest, brokerSocketServer(brokers.head.config.nodeId)) val coordinatorId = findCoordinatorResponse.data().coordinators().get(0).nodeId() var initPidResponse: InitProducerIdResponse = null diff --git a/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala index 6c882d3877d83..ca36f8c3707db 100644 --- a/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala @@ -68,7 +68,7 @@ class AllocateProducerIdsRequestTest(cluster: ClusterInstance) { ): AllocateProducerIdsResponse = { val allocateRequest = new AllocateProducerIdsRequest.Builder( new AllocateProducerIdsRequestData() - .setBrokerId(sourceBroker.config.brokerId) + .setBrokerId(sourceBroker.config.nodeId) .setBrokerEpoch(sourceBroker.lifecycleManager.brokerEpoch) ).build() diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index 3a0ffe1b4779f..099c9ece3ae61 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -62,7 +62,7 @@ abstract class BaseRequestTest extends IntegrationTestHarness { def brokerSocketServer(brokerId: Int): SocketServer = { brokers.find { broker => - broker.config.brokerId == brokerId + broker.config.nodeId == brokerId }.map(_.socketServer).getOrElse(throw new IllegalStateException(s"Could not find broker with id $brokerId")) } diff --git a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala index 6e43f904c11c7..6f67b8ce23504 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.resource.ResourceType import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -40,7 +40,7 @@ class DescribeClusterRequestTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") - properties.setProperty(ServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(ServerConfigs.BROKER_ID_CONFIG)}") + properties.setProperty(ServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(KRaftConfigs.NODE_ID_CONFIG)}") } @BeforeEach @@ -63,7 +63,7 @@ class DescribeClusterRequestTest extends BaseRequestTest { def testDescribeClusterRequest(includeClusterAuthorizedOperations: Boolean): Unit = { val expectedBrokers = brokers.map { server => new DescribeClusterResponseData.DescribeClusterBroker() - .setBrokerId(server.config.brokerId) + .setBrokerId(server.config.nodeId) .setHost("localhost") .setPort(server.socketServer.boundPort(listenerName)) .setRack(server.config.rack.orNull) diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 10b42f96b4e54..bc91c9911a0b3 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -299,7 +299,7 @@ class DynamicBrokerConfigTest { def testReconfigurableValidation(): Unit = { val origProps = TestUtils.createBrokerConfig(0, port = 8181) val config = KafkaConfig(origProps) - val invalidReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, ServerConfigs.BROKER_ID_CONFIG, "some.prop") + val invalidReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, KRaftConfigs.NODE_ID_CONFIG, "some.prop") val validReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "some.prop") def createReconfigurable(configs: Set[String]) = new Reconfigurable { diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 27f746a6b6d61..39e2842a23c31 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -349,10 +349,10 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { @ValueSource(strings = Array("kraft")) def testBrokerIdConfigChangeAndDelete(quorum: String): Unit = { val newValue: Long = 100000L - val brokerId: String = this.brokers.head.config.brokerId.toString + val brokerId: String = this.brokers.head.config.nodeId.toString setBrokerConfigs(brokerId, newValue) for (b <- this.brokers) { - val value = if (b.config.brokerId.toString == brokerId) newValue else QuotaConfig.QUOTA_BYTES_PER_SECOND_DEFAULT + val value = if (b.config.nodeId.toString == brokerId) newValue else QuotaConfig.QUOTA_BYTES_PER_SECOND_DEFAULT TestUtils.retry(10000) { assertEquals(value, b.quotaManagers.leader.upperBound) assertEquals(value, b.quotaManagers.follower.upperBound) @@ -396,12 +396,12 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { @ValueSource(strings = Array("kraft")) def testDefaultAndBrokerIdConfigChange(quorum: String): Unit = { val newValue: Long = 100000L - val brokerId: String = this.brokers.head.config.brokerId.toString + val brokerId: String = this.brokers.head.config.nodeId.toString setBrokerConfigs(brokerId, newValue) val newDefaultValue: Long = 200000L setBrokerConfigs("", newDefaultValue) for (b <- this.brokers) { - val value = if (b.config.brokerId.toString == brokerId) newValue else newDefaultValue + val value = if (b.config.nodeId.toString == brokerId) newValue else newDefaultValue TestUtils.retry(10000) { assertEquals(value, b.quotaManagers.leader.upperBound) assertEquals(value, b.quotaManagers.follower.upperBound) diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala index f96b2ceca3159..bf57360e29541 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -61,7 +61,7 @@ class FetchRequestTest extends BaseFetchRequestTest { val topicNames = topicIds.asScala.map(_.swap).asJava produceData(topicPartitions, messagesPerPartition) - val leaderId = brokers.head.config.brokerId + val leaderId = brokers.head.config.nodeId val partitionsForLeader = topicPartitionToLeader.toVector.collect { case (tp, partitionLeaderId) if partitionLeaderId == leaderId => tp } @@ -175,9 +175,9 @@ class FetchRequestTest extends BaseFetchRequestTest { val topicIds = getTopicIds().asJava val topicNames = topicIds.asScala.map(_.swap).asJava val leader = partitionToLeader(partition) - val nonReplicaOpt = brokers.find(_.config.brokerId != leader) + val nonReplicaOpt = brokers.find(_.config.nodeId != leader) assertTrue(nonReplicaOpt.isDefined) - val nonReplicaId = nonReplicaOpt.get.config.brokerId + val nonReplicaId = nonReplicaOpt.get.config.nodeId // Send the fetch request to the non-replica and verify the error code val fetchRequest = FetchRequest.Builder.forConsumer(ApiKeys.FETCH.latestVersion, Int.MaxValue, 0, createPartitionMap(1024, diff --git a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala index 916c244c2954d..bdba6b86c6000 100644 --- a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala @@ -891,7 +891,7 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { private def brokerSocketServer(brokerId: Int): SocketServer = { getBrokers.find { broker => - broker.config.brokerId == brokerId + broker.config.nodeId == brokerId }.map(_.socketServer).getOrElse(throw new IllegalStateException(s"Could not find broker with id $brokerId")) } } diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index f625afa1fa719..ce5b48b65f1dd 100755 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -70,7 +70,7 @@ class HighwatermarkPersistenceTest { scheduler = scheduler, logManager = logManagers.head, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(configs.head.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(configs.head.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = logDirFailureChannels.head, alterPartitionManager = alterIsrManager) replicaManager.startup() @@ -85,9 +85,9 @@ class HighwatermarkPersistenceTest { partition0.setLog(log0, isFutureLog = false) partition0.updateAssignmentAndIsr( - replicas = Seq(configs.head.brokerId, configs.last.brokerId), + replicas = Seq(configs.head.nodeId, configs.last.nodeId), isLeader = true, - isr = Set(configs.head.brokerId), + isr = Set(configs.head.nodeId), addingReplicas = Seq.empty, removingReplicas = Seq.empty, leaderRecoveryState = LeaderRecoveryState.RECOVERED @@ -128,7 +128,7 @@ class HighwatermarkPersistenceTest { scheduler = scheduler, logManager = logManagers.head, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(configs.head.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(configs.head.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = logDirFailureChannels.head, alterPartitionManager = alterIsrManager) replicaManager.startup() diff --git a/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala b/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala index 2f11690bacdc4..fcb3ac0092683 100644 --- a/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala @@ -73,7 +73,7 @@ class IsrExpirationTest { scheduler = null, logManager = logManager, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(configs.head.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(configs.head.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(configs.head.logDirs.size), alterPartitionManager = alterIsrManager) } @@ -94,7 +94,7 @@ class IsrExpirationTest { // create one partition and all replicas val partition0 = getPartitionWithAllReplicasInIsr(topic, 0, time, configs.head, log) - assertEquals(configs.map(_.brokerId).toSet, partition0.inSyncReplicaIds, "All replicas should be in ISR") + assertEquals(configs.map(_.nodeId).toSet, partition0.inSyncReplicaIds, "All replicas should be in ISR") // let the follower catch up to the Leader logEndOffset - 1 for (replica <- partition0.remoteReplicas) @@ -112,7 +112,7 @@ class IsrExpirationTest { // now follower hasn't pulled any data for > replicaMaxLagTimeMs ms. So it is stuck partition0OSR = partition0.getOutOfSyncReplicas(configs.head.replicaLagTimeMaxMs) - assertEquals(Set(configs.last.brokerId), partition0OSR, "Replica 1 should be out of sync") + assertEquals(Set(configs.last.nodeId), partition0OSR, "Replica 1 should be out of sync") verify(log, atLeastOnce()).logEndOffset } @@ -125,13 +125,13 @@ class IsrExpirationTest { // create one partition and all replicas val partition0 = getPartitionWithAllReplicasInIsr(topic, 0, time, configs.head, log) - assertEquals(configs.map(_.brokerId).toSet, partition0.inSyncReplicaIds, "All replicas should be in ISR") + assertEquals(configs.map(_.nodeId).toSet, partition0.inSyncReplicaIds, "All replicas should be in ISR") // Let enough time pass for the replica to be considered stuck time.sleep(150) val partition0OSR = partition0.getOutOfSyncReplicas(configs.head.replicaLagTimeMaxMs) - assertEquals(Set(configs.last.brokerId), partition0OSR, "Replica 1 should be out of sync") + assertEquals(Set(configs.last.nodeId), partition0OSR, "Replica 1 should be out of sync") verify(log, atLeastOnce()).logEndOffset } @@ -145,7 +145,7 @@ class IsrExpirationTest { val log = logMock // add one partition val partition0 = getPartitionWithAllReplicasInIsr(topic, 0, time, configs.head, log) - assertEquals(configs.map(_.brokerId).toSet, partition0.inSyncReplicaIds, "All replicas should be in ISR") + assertEquals(configs.map(_.nodeId).toSet, partition0.inSyncReplicaIds, "All replicas should be in ISR") // Make the remote replica not read to the end of log. It should be not be out of sync for at least 100 ms for (replica <- partition0.remoteReplicas) replica.updateFetchStateOrThrow( @@ -177,7 +177,7 @@ class IsrExpirationTest { // The replicas will no longer be in ISR partition0OSR = partition0.getOutOfSyncReplicas(configs.head.replicaLagTimeMaxMs) - assertEquals(Set(configs.last.brokerId), partition0OSR, "Replica 1 should be out of sync") + assertEquals(Set(configs.last.nodeId), partition0OSR, "Replica 1 should be out of sync") // Now actually make a fetch to the end of the log. The replicas should be back in ISR partition0.remoteReplicas.foreach { r => @@ -202,7 +202,7 @@ class IsrExpirationTest { // create one partition and all replicas val partition0 = getPartitionWithAllReplicasInIsr(topic, 0, time, configs.head, log) - assertEquals(configs.map(_.brokerId).toSet, partition0.inSyncReplicaIds, "All replicas should be in ISR") + assertEquals(configs.map(_.nodeId).toSet, partition0.inSyncReplicaIds, "All replicas should be in ISR") // let the follower catch up to the Leader logEndOffset for (replica <- partition0.remoteReplicas) @@ -227,15 +227,15 @@ class IsrExpirationTest { private def getPartitionWithAllReplicasInIsr(topic: String, partitionId: Int, time: Time, config: KafkaConfig, localLog: UnifiedLog): Partition = { - val leaderId = config.brokerId + val leaderId = config.nodeId val tp = new TopicPartition(topic, partitionId) val partition = replicaManager.createPartition(tp) partition.setLog(localLog, isFutureLog = false) partition.updateAssignmentAndIsr( - replicas = configs.map(_.brokerId), + replicas = configs.map(_.nodeId), isLeader = true, - isr = configs.map(_.brokerId).toSet, + isr = configs.map(_.nodeId).toSet, addingReplicas = Seq.empty, removingReplicas = Seq.empty, leaderRecoveryState = LeaderRecoveryState.RECOVERED diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 31c192ff9ef30..47e4deacd7b84 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -158,7 +158,7 @@ class KafkaConfigTest { val hostName = "fake-host" val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, s"$hostName:$controllerPort") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"PLAINTEXT://$hostName:$brokerProt") @@ -191,7 +191,7 @@ class KafkaConfigTest { def testDuplicateListeners(): Unit = { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9095") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") @@ -219,7 +219,7 @@ class KafkaConfigTest { def testIPv4AndIPv6SamePortListeners(): Unit = { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.put(ServerConfigs.BROKER_ID_CONFIG, "1") + props.put(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "CONTROLLER://localhost:9091") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_DEFAULT + ",CONTROLLER:PLAINTEXT") @@ -476,7 +476,7 @@ class KafkaConfigTest { def testBadListenerProtocol(): Unit = { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "CONTROLLER://localhost:9092") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "BAD://localhost:9091") @@ -488,7 +488,7 @@ class KafkaConfigTest { def testListenerNamesWithAdvertisedListenerUnset(): Unit = { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "CONTROLLER://localhost:9092") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") @@ -515,7 +515,7 @@ class KafkaConfigTest { def testListenerAndAdvertisedListenerNames(): Unit = { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "CONTROLLER://localhost:9092") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") @@ -548,7 +548,7 @@ class KafkaConfigTest { @Test def testListenerNameMissingFromListenerSecurityProtocolMap(): Unit = { val props = new Properties() - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091,REPLICATION://localhost:9092") @@ -559,7 +559,7 @@ class KafkaConfigTest { @Test def testInterBrokerListenerNameMissingFromListenerSecurityProtocolMap(): Unit = { val props = new Properties() - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091") @@ -570,7 +570,7 @@ class KafkaConfigTest { @Test def testInterBrokerListenerNameAndSecurityProtocolSet(): Unit = { val props = new Properties() - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091") @@ -583,7 +583,7 @@ class KafkaConfigTest { def testCaseInsensitiveListenerProtocol(): Unit = { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "CONTROLLER://localhost:9093") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "plaintext://localhost:9091,SsL://localhost:9092") @@ -601,7 +601,7 @@ class KafkaConfigTest { def testListenerDefaults(): Unit = { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9093") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") @@ -616,7 +616,7 @@ class KafkaConfigTest { def testVersionConfiguration(): Unit = { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") val conf = KafkaConfig.fromProps(props) @@ -784,7 +784,7 @@ class KafkaConfigTest { def baseProperties: Properties = { val validRequiredProperties = new Properties() validRequiredProperties.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - validRequiredProperties.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + validRequiredProperties.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") validRequiredProperties.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") validRequiredProperties.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") validRequiredProperties @@ -814,7 +814,6 @@ class KafkaConfigTest { case ZkConfigs.ZK_SSL_CRL_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean") case ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean") - case ServerConfigs.BROKER_ID_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ServerConfigs.NUM_IO_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.BACKGROUND_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") @@ -1084,7 +1083,7 @@ class KafkaConfigTest { def baseProperties: Properties = { val validRequiredProperties = new Properties() validRequiredProperties.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - validRequiredProperties.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + validRequiredProperties.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") validRequiredProperties.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9093") validRequiredProperties.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") validRequiredProperties @@ -1182,9 +1181,7 @@ class KafkaConfigTest { defaults.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") // For ZkConnectionTimeoutMs defaults.setProperty(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG, "1234") - defaults.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") - defaults.setProperty(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG, "1") - defaults.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + defaults.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") defaults.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:1122") defaults.setProperty(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG, "127.0.0.1:2, 127.0.0.2:3") defaults.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, "/tmp1,/tmp2") @@ -1199,9 +1196,7 @@ class KafkaConfigTest { val config = KafkaConfig.fromProps(defaults) assertEquals(1234, config.zkConnectionTimeoutMs) - assertEquals(false, config.brokerIdGenerationEnable) - assertEquals(1, config.maxReservedBrokerId) - assertEquals(1, config.brokerId) + assertEquals(1, config.nodeId) assertEquals(Seq("PLAINTEXT://127.0.0.1:1122"), config.effectiveAdvertisedBrokerListeners.map(_.connectionString)) assertEquals(Map("127.0.0.1" -> 2, "127.0.0.2" -> 3), config.maxConnectionsPerIpOverrides) assertEquals(List("/tmp1", "/tmp2"), config.logDirs) @@ -1225,7 +1220,7 @@ class KafkaConfigTest { def testNonroutableAdvertisedListeners(): Unit = { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "CONTROLLER://localhost:9092") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://0.0.0.0:9092") @@ -1458,16 +1453,6 @@ class KafkaConfigTest { assertFalse(isValidKafkaConfig(props)) } - @Test - def testRejectsNegativeNodeIdForRaftBasedCaseWithAutoGenDisabled(): Unit = { - // -1 is the default for both node.id and broker.id - val props = new Properties() - props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") - props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") - assertFalse(isValidKafkaConfig(props)) - } - @Test def testCustomMetadataLogDir(): Unit = { val metadataDir = "/path/to/metadata/dir" @@ -1505,41 +1490,6 @@ class KafkaConfigTest { assertEquals(Seq(dataDir1, dataDir2), config.logDirs) } - @Test - def testPopulateSynonymsOnEmptyMap(): Unit = { - assertEquals(Collections.emptyMap(), KafkaConfig.populateSynonyms(Collections.emptyMap())) - } - - @Test - def testPopulateSynonymsOnMapWithoutNodeId(): Unit = { - val input = new util.HashMap[String, String]() - input.put(ServerConfigs.BROKER_ID_CONFIG, "4") - val expectedOutput = new util.HashMap[String, String]() - expectedOutput.put(ServerConfigs.BROKER_ID_CONFIG, "4") - expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4") - assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input)) - } - - @Test - def testPopulateSynonymsOnMapWithoutBrokerId(): Unit = { - val input = new util.HashMap[String, String]() - input.put(KRaftConfigs.NODE_ID_CONFIG, "4") - val expectedOutput = new util.HashMap[String, String]() - expectedOutput.put(ServerConfigs.BROKER_ID_CONFIG, "4") - expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4") - assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input)) - } - - @Test - def testNodeIdMustNotBeDifferentThanBrokerId(): Unit = { - val props = new Properties() - props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") - props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") - assertEquals("You must set `node.id` to the same value as `broker.id`.", - assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage()) - } - @Test def testNodeIdOrBrokerIdMustBeSetWithKraft(): Unit = { val props = new Properties() @@ -1554,13 +1504,13 @@ class KafkaConfigTest { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "3") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "3") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") val config = KafkaConfig.fromProps(props) - assertEquals(3, config.brokerId) + assertEquals(3, config.nodeId) assertEquals(3, config.nodeId) val originals = config.originals() - assertEquals("3", originals.get(ServerConfigs.BROKER_ID_CONFIG)) + assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG)) assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG)) } @@ -1578,10 +1528,10 @@ class KafkaConfigTest { val props = new Properties() props.putAll(kraftProps()) val config = KafkaConfig.fromProps(props) - assertEquals(3, config.brokerId) + assertEquals(3, config.nodeId) assertEquals(3, config.nodeId) val originals = config.originals() - assertEquals("3", originals.get(ServerConfigs.BROKER_ID_CONFIG)) + assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG)) assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG)) } @@ -1589,7 +1539,7 @@ class KafkaConfigTest { def testSaslJwksEndpointRetryDefaults(): Unit = { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") props.setProperty(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "CONTROLLER://localhost:9092") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") val config = KafkaConfig.fromProps(props) diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala index e07ae3032ca6b..4bbc65c503080 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala @@ -20,7 +20,7 @@ import java.util import java.util.concurrent.atomic.AtomicReference import kafka.utils.{CoreUtils, TestUtils} import org.apache.kafka.common.metrics.{KafkaMetric, MetricsContext, MetricsReporter} -import org.apache.kafka.server.config.ServerConfigs +import org.apache.kafka.server.config.KRaftConfigs import org.apache.kafka.server.metrics.MetricConfigs import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.Assertions._ @@ -73,8 +73,7 @@ class KafkaMetricsReporterTest extends QuorumTestHarness { super.setUp(testInfo) val props = TestUtils.createBrokerConfig(1) props.setProperty(MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG, "kafka.server.KafkaMetricsReporterTest$MockMetricsReporter") - props.setProperty(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "true") - props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") + props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1") config = KafkaConfig.fromProps(props) broker = createBroker(config, threadNamePrefix = Option(this.getClass.getName)) broker.startup() diff --git a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala index 5ba6ef34603a3..54fc5f36e2458 100644 --- a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala @@ -59,7 +59,7 @@ class ListOffsetsRequestTest extends BaseRequestTest { .build() val replicaRequest = ListOffsetsRequest.Builder - .forReplica(ApiKeys.LIST_OFFSETS.latestVersion, brokers.head.config.brokerId) + .forReplica(ApiKeys.LIST_OFFSETS.latestVersion, brokers.head.config.nodeId) .setTargetTimes(targetTimes) .build() @@ -69,7 +69,7 @@ class ListOffsetsRequestTest extends BaseRequestTest { .build() // Unknown topic - val randomBrokerId = brokers.head.config.brokerId + val randomBrokerId = brokers.head.config.nodeId assertResponseError(Errors.UNKNOWN_TOPIC_OR_PARTITION, randomBrokerId, consumerRequest) assertResponseError(Errors.UNKNOWN_TOPIC_OR_PARTITION, randomBrokerId, replicaRequest) assertResponseError(Errors.UNKNOWN_TOPIC_OR_PARTITION, randomBrokerId, debugReplicaRequest) @@ -79,7 +79,7 @@ class ListOffsetsRequestTest extends BaseRequestTest { val replicas = topicDescription.get(partition.topic).partitions.get(partition.partition).replicas.asScala.map(_.id).toSet val leader = partitionToLeader(partition.partition) val follower = replicas.find(_ != leader).get - val nonReplica = brokers.map(_.config.brokerId).find(!replicas.contains(_)).get + val nonReplica = brokers.map(_.config.nodeId).find(!replicas.contains(_)).get // Follower assertResponseError(Errors.NOT_LEADER_OR_FOLLOWER, follower, consumerRequest) diff --git a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala index aa2e634e9bfaf..bff79da8eb00f 100644 --- a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala @@ -78,7 +78,7 @@ class LogDirFailureTest extends IntegrationTestHarness { val partition = new TopicPartition(topic, 0) val leaderServerId = producer.partitionsFor(topic).asScala.find(_.partition() == 0).get.leader().id() - val leaderServer = brokers.find(_.config.brokerId == leaderServerId).get + val leaderServer = brokers.find(_.config.nodeId == leaderServerId).get // shut down the controller to simulate the case where the broker is not able to send the log dir notification controllerServer.shutdown() @@ -120,9 +120,9 @@ class LogDirFailureTest extends IntegrationTestHarness { val partitionInfo = producer.partitionsFor(topic).asScala.find(_.partition() == 0).get val leaderServerId = partitionInfo.leader().id() - val leaderServer = brokers.find(_.config.brokerId == leaderServerId).get + val leaderServer = brokers.find(_.config.nodeId == leaderServerId).get val followerServerId = partitionInfo.replicas().map(_.id()).find(_ != leaderServerId).get - val followerServer = brokers.find(_.config.brokerId == followerServerId).get + val followerServer = brokers.find(_.config.nodeId == followerServerId).get followerServer.replicaManager.markPartitionOffline(partition) // Send a message to another partition whose leader is the same as partition 0 @@ -153,7 +153,7 @@ class LogDirFailureTest extends IntegrationTestHarness { val record = new ProducerRecord(topic, 0, s"key".getBytes, s"value".getBytes) val leaderServerId = producer.partitionsFor(topic).asScala.find(_.partition() == 0).get.leader().id() - val leaderServer = brokers.find(_.config.brokerId == leaderServerId).get + val leaderServer = brokers.find(_.config.nodeId == leaderServerId).get causeLogDirFailure(failureType, leaderServer, partition) @@ -174,7 +174,7 @@ class LogDirFailureTest extends IntegrationTestHarness { val record = new ProducerRecord(topic, 0, s"key".getBytes, s"value".getBytes) val originalLeaderServerId = producer.partitionsFor(topic).asScala.find(_.partition() == 0).get.leader().id() - val originalLeaderServer = brokers.find(_.config.brokerId == originalLeaderServerId).get + val originalLeaderServer = brokers.find(_.config.nodeId == originalLeaderServerId).get // The first send() should succeed producer.send(record).get() diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index f9970d2967afa..fff225df3500a 100755 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -260,7 +260,7 @@ class LogRecoveryTest extends QuorumTestHarness { def leaderExists: Option[Int] = { brokers.find { broker => broker.replicaManager.onlinePartition(tp).exists(_.leaderLogIfLocal.isDefined) - }.map(_.config.brokerId) + }.map(_.config.nodeId) } waitUntilTrue(() => leaderExists.isDefined, diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala index ddfa9b42d4c0e..e0cd80a501b48 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala @@ -221,7 +221,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { val v4MetadataResponse = sendMetadataRequest(new MetadataRequest.Builder(List(replicaDownTopic).asJava, true).build(4)) val v4BrokerIds = v4MetadataResponse.brokers().asScala.map(_.id).toSeq assertTrue(v4MetadataResponse.errors.isEmpty, "Response should have no errors") - assertFalse(v4BrokerIds.contains(downNode.config.brokerId), s"The downed broker should not be in the brokers list") + assertFalse(v4BrokerIds.contains(downNode.config.nodeId), s"The downed broker should not be in the brokers list") assertEquals(1, v4MetadataResponse.topicMetadata.size, "Response should have one topic") val v4PartitionMetadata = v4MetadataResponse.topicMetadata.asScala.head.partitionMetadata.asScala.head assertEquals(Errors.NONE, v4PartitionMetadata.error, "PartitionMetadata should have no errors") @@ -236,14 +236,14 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { topic: String ): Unit = { val activeBrokers = brokers.filter(_.brokerState != BrokerState.NOT_RUNNING) - val expectedIsr = activeBrokers.map(_.config.brokerId).toSet + val expectedIsr = activeBrokers.map(_.config.nodeId).toSet // Assert that topic metadata at new brokers is updated correctly activeBrokers.foreach { broker => var actualIsr = Set.empty[Int] TestUtils.waitUntilTrue(() => { val metadataResponse = sendMetadataRequest(new MetadataRequest.Builder(Seq(topic).asJava, false).build, - Some(brokerSocketServer(broker.config.brokerId))) + Some(brokerSocketServer(broker.config.nodeId))) val firstPartitionMetadata = metadataResponse.topicMetadata.asScala.headOption.flatMap(_.partitionMetadata.asScala.headOption) actualIsr = firstPartitionMetadata.map { partitionMetadata => partitionMetadata.inSyncReplicaIds.asScala.map(Int.unbox).toSet @@ -287,7 +287,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { brokers.filter(_.brokerState == BrokerState.RUNNING).foreach { broker => TestUtils.waitUntilTrue(() => { val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build, - Some(brokerSocketServer(broker.config.brokerId))) + Some(brokerSocketServer(broker.config.nodeId))) val brokers = metadataResponse.brokers.asScala.toSeq.sortBy(_.id) val topicMetadata = metadataResponse.topicMetadata.asScala.toSeq.sortBy(_.topic) brokersSorted == brokers && metadataResponse.topicMetadata.asScala.toSeq.sortBy(_.topic) == topicMetadata diff --git a/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala index fc06a9eeeb759..b4718e6b6529d 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala @@ -43,7 +43,7 @@ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest { val request = OffsetsForLeaderEpochRequest.Builder.forFollower(epochs, 1).build() // Unknown topic - val randomBrokerId = brokers.head.config.brokerId + val randomBrokerId = brokers.head.config.nodeId assertResponseError(Errors.UNKNOWN_TOPIC_OR_PARTITION, randomBrokerId, request) val partitionToLeader = createTopic(topic, replicationFactor = 2) @@ -51,7 +51,7 @@ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest { val replicas = topicDescription.get(partition.topic()).partitions().get(partition.partition()).replicas().asScala.map(_.id()).toSet val leader = partitionToLeader(partition.partition) val follower = replicas.find(_ != leader).get - val nonReplica = brokers.map(_.config.brokerId).find(!replicas.contains(_)).get + val nonReplica = brokers.map(_.config.nodeId).find(!replicas.contains(_)).get assertResponseError(Errors.NOT_LEADER_OR_FOLLOWER, follower, request) assertResponseError(Errors.NOT_LEADER_OR_FOLLOWER, nonReplica, request) diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index 64111f1487513..5bcddbe949ee3 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -176,9 +176,9 @@ class ProduceRequestTest extends BaseRequestTest { ) val partitionToLeader = getPartitionToLeader(admin, topic) val leader = partitionToLeader(partition) - val nonReplicaOpt = brokers.find(_.config.brokerId != leader) + val nonReplicaOpt = brokers.find(_.config.nodeId != leader) assertTrue(nonReplicaOpt.isDefined) - val nonReplicaId = nonReplicaOpt.get.config.brokerId + val nonReplicaId = nonReplicaOpt.get.config.nodeId // Send the produce request to the non-replica val records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("key".getBytes, "value".getBytes)) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala index c7c43e85965bc..2c28f332cdcec 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala @@ -85,7 +85,7 @@ class ReplicaFetcherThreadTest { replicaMgr: ReplicaManager, quota: ReplicaQuota, leaderEndpointBlockingSend: BlockingSend): ReplicaFetcherThread = { - val logContext = new LogContext(s"[ReplicaFetcher replicaId=${brokerConfig.brokerId}, leaderId=${leaderEndpointBlockingSend.brokerEndPoint().id}, fetcherId=$fetcherId] ") + val logContext = new LogContext(s"[ReplicaFetcher replicaId=${brokerConfig.nodeId}, leaderId=${leaderEndpointBlockingSend.brokerEndPoint().id}, fetcherId=$fetcherId] ") val fetchSessionHandler = new FetchSessionHandler(logContext, leaderEndpointBlockingSend.brokerEndPoint().id) val leader = new RemoteLeaderEndPoint(logContext.logPrefix, leaderEndpointBlockingSend, fetchSessionHandler, brokerConfig, replicaMgr, quota, () => brokerConfig.interBrokerProtocolVersion, () => 1) @@ -275,7 +275,7 @@ class ReplicaFetcherThreadTest { // Create the fetcher thread val mockNetwork = new MockBlockingSender(Collections.emptyMap(), brokerEndPoint, Time.SYSTEM) - val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") + val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.nodeId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") val fetchSessionHandler = new FetchSessionHandler(logContext, brokerEndPoint.id) val leader = new RemoteLeaderEndPoint(logContext.logPrefix, mockNetwork, fetchSessionHandler, config, replicaManager, quota, () => config.interBrokerProtocolVersion, () => 1) @@ -379,7 +379,7 @@ class ReplicaFetcherThreadTest { when(partition.localLogOrException).thenReturn(log) when(partition.appendRecordsToFollowerOrFutureReplica(any(), any())).thenReturn(None) - val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") + val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.nodeId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") val mockNetwork = new MockBlockingSender( Collections.emptyMap(), @@ -471,7 +471,7 @@ class ReplicaFetcherThreadTest { -1L ))) - val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") + val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.nodeId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") val mockNetwork = new MockBlockingSender( Collections.emptyMap(), @@ -592,7 +592,7 @@ class ReplicaFetcherThreadTest { when(replicaQuota.isThrottled(any[TopicPartition])).thenReturn(false) when(log.logStartOffset).thenReturn(0) - val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.brokerId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") + val logContext = new LogContext(s"[ReplicaFetcher replicaId=${config.nodeId}, leaderId=${brokerEndPoint.id}, fetcherId=0] ") val fetchSessionHandler = new FetchSessionHandler(logContext, brokerEndPoint.id) val leader = new RemoteLeaderEndPoint(logContext.logPrefix, mockBlockingSend, fetchSessionHandler, config, replicaManager, replicaQuota, () => config.interBrokerProtocolVersion, () => 1) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala index f0a4be811bb3f..230bb1dc4a5e6 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala @@ -62,7 +62,7 @@ class ReplicaManagerQuotasTest { @Test def shouldExcludeSubsequentThrottledPartitions(): Unit = { setUpMocks(fetchInfo) - val followerReplicaId = configs.last.brokerId + val followerReplicaId = configs.last.nodeId val quota = mockQuota() when(quota.isQuotaExceeded) @@ -80,7 +80,7 @@ class ReplicaManagerQuotasTest { @Test def shouldGetNoMessagesIfQuotasExceededOnSubsequentPartitions(): Unit = { setUpMocks(fetchInfo) - val followerReplicaId = configs.last.brokerId + val followerReplicaId = configs.last.nodeId val quota = mockQuota() when(quota.isQuotaExceeded) @@ -98,7 +98,7 @@ class ReplicaManagerQuotasTest { @Test def shouldGetBothMessagesIfQuotasAllow(): Unit = { setUpMocks(fetchInfo) - val followerReplicaId = configs.last.brokerId + val followerReplicaId = configs.last.nodeId val quota = mockQuota() when(quota.isQuotaExceeded) @@ -116,7 +116,7 @@ class ReplicaManagerQuotasTest { @Test def shouldIncludeInSyncThrottledReplicas(): Unit = { setUpMocks(fetchInfo, bothReplicasInSync = true) - val followerReplicaId = configs.last.brokerId + val followerReplicaId = configs.last.nodeId val quota = mockQuota() when(quota.isQuotaExceeded) @@ -299,7 +299,7 @@ class ReplicaManagerQuotasTest { val alterIsrManager: AlterPartitionManager = mock(classOf[AlterPartitionManager]) - val leaderBrokerId = configs.head.brokerId + val leaderBrokerId = configs.head.nodeId quotaManager = QuotaFactory.instantiate(configs.head, metrics, time, "") replicaManager = new ReplicaManager( metrics = metrics, @@ -320,9 +320,9 @@ class ReplicaManagerQuotasTest { partition.setLog(log, isFutureLog = false) partition.updateAssignmentAndIsr( - replicas = Seq(leaderBrokerId, configs.last.brokerId), + replicas = Seq(leaderBrokerId, configs.last.nodeId), isLeader = true, - isr = if (bothReplicasInSync) Set(leaderBrokerId, configs.last.brokerId) else Set(leaderBrokerId), + isr = if (bothReplicasInSync) Set(leaderBrokerId, configs.last.nodeId) else Set(leaderBrokerId), addingReplicas = Seq.empty, removingReplicas = Seq.empty, leaderRecoveryState = LeaderRecoveryState.RECOVERED diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 5365de394e52a..1252efdf96d3f 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -174,7 +174,7 @@ class ReplicaManagerTest { scheduler = new MockScheduler(time), logManager = mockLogMgr, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterPartitionManager) try { @@ -203,7 +203,7 @@ class ReplicaManagerTest { scheduler = new MockScheduler(time), logManager = mockLogMgr, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterPartitionManager) try { @@ -229,7 +229,7 @@ class ReplicaManagerTest { scheduler = new MockScheduler(time), logManager = mockLogMgr, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterPartitionManager, threadNamePrefix = Option(this.getClass.getName)) @@ -488,7 +488,7 @@ class ReplicaManagerTest { scheduler = new MockScheduler(time), logManager = mockLogMgr, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterPartitionManager, threadNamePrefix = Option(this.getClass.getName)) @@ -2542,7 +2542,7 @@ class ReplicaManagerTest { config.dynamicConfig.initialize(None) val props = new Properties() props.put(TransactionLogConfig.TRANSACTION_PARTITION_VERIFICATION_ENABLE_CONFIG, "true") - config.dynamicConfig.updateBrokerConfig(config.brokerId, props) + config.dynamicConfig.updateBrokerConfig(config.nodeId, props) TestUtils.waitUntilTrue(() => config.transactionLogConfig.transactionPartitionVerificationEnable == true, "Config did not dynamically update.") // Try to append more records. We don't need to send a request since the transaction is already ongoing. @@ -2594,7 +2594,7 @@ class ReplicaManagerTest { config.dynamicConfig.initialize(None) val props = new Properties() props.put(TransactionLogConfig.TRANSACTION_PARTITION_VERIFICATION_ENABLE_CONFIG, "false") - config.dynamicConfig.updateBrokerConfig(config.brokerId, props) + config.dynamicConfig.updateBrokerConfig(config.nodeId, props) TestUtils.waitUntilTrue(() => config.transactionLogConfig.transactionPartitionVerificationEnable == false, "Config did not dynamically update.") // Confirm we did not write to the log and instead returned error. @@ -2700,7 +2700,7 @@ class ReplicaManagerTest { scheduler = time.scheduler, logManager = logManager, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterPartitionManager, threadNamePrefix = Option(this.getClass.getName)) @@ -2780,7 +2780,7 @@ class ReplicaManagerTest { scheduler = time.scheduler, logManager = logManager, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterPartitionManager, threadNamePrefix = Option(this.getClass.getName)) @@ -3019,7 +3019,7 @@ class ReplicaManagerTest { new ReplicaFetcherManager(this.config, rm, metrics, time, threadNamePrefix, replicationQuotaManager, () => this.metadataCache.metadataVersion(), () => 1) { override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): ReplicaFetcherThread = { - val logContext = new LogContext(s"[ReplicaFetcher replicaId=${rm.config.brokerId}, leaderId=${sourceBroker.id}, " + + val logContext = new LogContext(s"[ReplicaFetcher replicaId=${rm.config.nodeId}, leaderId=${sourceBroker.id}, " + s"fetcherId=$fetcherId] ") val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id) val leader = new RemoteLeaderEndPoint(logContext.logPrefix, blockingSend, fetchSessionHandler, rm.config, @@ -4625,7 +4625,7 @@ class ReplicaManagerTest { scheduler = new MockScheduler(time), logManager = mockLogMgr, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterPartitionManager) { override def getPartitionOrException(topicPartition: TopicPartition): Partition = { @@ -6422,7 +6422,7 @@ class ReplicaManagerTest { scheduler = new MockScheduler(time), logManager = mockLogMgr, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterPartitionManager)) diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index ae1e0a1f5e871..ed3ba7f08d9b6 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -257,7 +257,7 @@ class ReplicationQuotasTest extends QuorumTestHarness { }, s"Offsets did not match for partition $partitionId on broker $brokerId", 60000) } - private def brokerFor(id: Int): KafkaBroker = brokers.filter(_.config.brokerId == id).head + private def brokerFor(id: Int): KafkaBroker = brokers.filter(_.config.nodeId == id).head def createBrokers(brokerIds: Seq[Int]): Unit = { brokerIds.foreach { id => diff --git a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala index f60fc12f47b4b..02fe21dfc201f 100644 --- a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala @@ -121,9 +121,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo val topicIdPartition = new TopicIdPartition(topicId, new TopicPartition(topic, partition)) val topicNames = topicIds.asScala.map(_.swap).asJava val leader = partitionToLeader(topicIdPartition) - val nonReplicaOpt = getBrokers.find(_.config.brokerId != leader) + val nonReplicaOpt = getBrokers.find(_.config.nodeId != leader) assertTrue(nonReplicaOpt.isDefined) - val nonReplicaId = nonReplicaOpt.get.config.brokerId + val nonReplicaId = nonReplicaOpt.get.config.nodeId val send: Seq[TopicIdPartition] = Seq(topicIdPartition) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala index 71addf733b04a..4b3c582d13225 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala @@ -846,8 +846,8 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { // Restart the only running broker. val broker = cluster.brokers().values().iterator().next() - cluster.shutdownBroker(broker.config.brokerId) - cluster.startBroker(broker.config.brokerId) + cluster.shutdownBroker(broker.config.nodeId) + cluster.startBroker(broker.config.nodeId) // Prepare the next heartbeat for member with no updates. shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( diff --git a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala index d3bed2905f6ba..690a7471ba940 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala @@ -237,7 +237,7 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging { } private def sender(from: KafkaBroker, to: KafkaBroker): BlockingSend = { - val node = from.metadataCache.getAliveBrokerNode(to.config.brokerId, + val node = from.metadataCache.getAliveBrokerNode(to.config.nodeId, from.config.interBrokerListenerName).get val endPoint = new BrokerEndPoint(node.id(), node.host(), node.port()) new BrokerBlockingSender(endPoint, from.config, new Metrics(), Time.SYSTEM, 42, "TestFetcher", new LogContext()) diff --git a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala index 79f4be41b8f62..9ec5032518525 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala @@ -72,12 +72,12 @@ class OffsetsForLeaderEpochTest { scheduler = null, logManager = logManager, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterIsrManager) val partition = replicaManager.createPartition(tp) partition.setLog(mockLog, isFutureLog = false) - partition.leaderReplicaIdOpt = Some(config.brokerId) + partition.leaderReplicaIdOpt = Some(config.nodeId) //When val response = replicaManager.lastOffsetForLeaderEpoch(request) @@ -101,7 +101,7 @@ class OffsetsForLeaderEpochTest { scheduler = null, logManager = logManager, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterIsrManager) replicaManager.createPartition(tp) @@ -132,7 +132,7 @@ class OffsetsForLeaderEpochTest { scheduler = null, logManager = logManager, quotaManagers = quotaManager, - metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), + metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), alterPartitionManager = alterIsrManager) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 202600e0f3c42..c115449ff886b 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -253,7 +253,6 @@ object TestUtils extends Logging { props.put(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") props.setProperty(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, TimeUnit.MINUTES.toMillis(10).toString) props.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString) - props.put(ServerConfigs.BROKER_ID_CONFIG, nodeId.toString) props.put(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") @@ -778,7 +777,7 @@ object TestUtils extends Logging { def findLeaderEpoch(brokerId: Int, topicPartition: TopicPartition, brokers: Iterable[KafkaBroker]): Int = { - val leaderBroker = brokers.find(_.config.brokerId == brokerId) + val leaderBroker = brokers.find(_.config.nodeId == brokerId) val leaderPartition = leaderBroker.flatMap(_.replicaManager.onlinePartition(topicPartition)) .getOrElse(throw new AssertionError(s"Failed to find expected replica on broker $brokerId")) leaderPartition.getLeaderEpoch @@ -788,12 +787,12 @@ object TestUtils extends Logging { brokers: Iterable[KafkaBroker]): Int = { val followerOpt = brokers.find { server => server.replicaManager.onlinePartition(topicPartition) match { - case Some(partition) => !partition.leaderReplicaIdOpt.contains(server.config.brokerId) + case Some(partition) => !partition.leaderReplicaIdOpt.contains(server.config.nodeId) case None => false } } followerOpt - .map(_.config.brokerId) + .map(_.config.nodeId) .getOrElse(throw new AssertionError(s"Unable to locate follower for $topicPartition")) } @@ -806,7 +805,7 @@ object TestUtils extends Logging { def waitUntilBrokerMetadataIsPropagated[B <: KafkaBroker]( brokers: Seq[B], timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Unit = { - val expectedBrokerIds = brokers.map(_.config.brokerId).toSet + val expectedBrokerIds = brokers.map(_.config.nodeId).toSet waitUntilTrue(() => brokers.forall(server => expectedBrokerIds == server.dataPlaneRequestProcessor.metadataCache.getAliveBrokers().map(_.id).toSet ), "Timed out waiting for broker metadata to propagate to all servers", timeout) @@ -897,23 +896,23 @@ object TestUtils extends Logging { if (expectedLeaderOpt.isDefined) { debug(s"Checking leader that has changed to ${expectedLeaderOpt.get}") brokers.find { broker => - broker.config.brokerId == expectedLeaderOpt.get && + broker.config.nodeId == expectedLeaderOpt.get && broker.replicaManager.onlinePartition(tp).exists(_.leaderLogIfLocal.isDefined) - }.map(_.config.brokerId) + }.map(_.config.nodeId) } else if (oldLeaderOpt.isDefined) { debug(s"Checking leader that has changed from ${oldLeaderOpt}") brokers.find { broker => broker.replicaManager.onlinePartition(tp).exists(_.leaderLogIfLocal.isDefined) - broker.config.brokerId != oldLeaderOpt.get && + broker.config.nodeId != oldLeaderOpt.get && broker.replicaManager.onlinePartition(tp).exists(_.leaderLogIfLocal.isDefined) - }.map(_.config.brokerId) + }.map(_.config.nodeId) } else { debug(s"Checking the elected leader") brokers.find { broker => broker.replicaManager.onlinePartition(tp).exists(_.leaderLogIfLocal.isDefined) - }.map(_.config.brokerId) + }.map(_.config.nodeId) } } @@ -930,7 +929,7 @@ object TestUtils extends Logging { def leaderIfExists: Option[Int] = { brokers.find { broker => broker.replicaManager.onlinePartition(tp).exists(_.leaderLogIfLocal.isDefined) - }.map(_.config.brokerId) + }.map(_.config.nodeId) } waitUntilTrue(() => leaderIfExists.isDefined, @@ -1299,7 +1298,7 @@ object TestUtils extends Logging { val configEntries = props.asScala.map { case (k, v) => new AlterConfigOp(new ConfigEntry(k, v), opType) }.toList.asJavaCollection val configs = if (perBrokerConfig) { servers.map { server => - val resource = new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString) + val resource = new ConfigResource(ConfigResource.Type.BROKER, server.config.nodeId.toString) (resource, configEntries) }.toMap.asJava } else { diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java index 16e54582a290a..bb4aa4c642536 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/fetcher/ReplicaFetcherThreadBenchmark.java @@ -256,7 +256,7 @@ static class ReplicaFetcherBenchThread extends ReplicaFetcherThread { Partition> partitions) { super("name", new RemoteLeaderEndPoint( - String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.brokerId(), 3, 3), + String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.nodeId(), 3, 3), new BrokerBlockingSender( new BrokerEndPoint(3, "host", 3000), config, @@ -264,10 +264,10 @@ static class ReplicaFetcherBenchThread extends ReplicaFetcherThread { Time.SYSTEM, 3, String.format("broker-%d-fetcher-%d", 3, 3), - new LogContext(String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.brokerId(), 3, 3)) + new LogContext(String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.nodeId(), 3, 3)) ), new FetchSessionHandler( - new LogContext(String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.brokerId(), 3, 3)), 3), + new LogContext(String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.nodeId(), 3, 3)), 3), config, replicaManager, replicaQuota, @@ -303,7 +303,7 @@ public Map fetch(FetchRequest.B new FailedPartitions(), replicaManager, replicaQuota, - String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.brokerId(), 3, 3), + String.format("[ReplicaFetcher replicaId=%d, leaderId=%d, fetcherId=%d", config.nodeId(), 3, 3), config::interBrokerProtocolVersion ); diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java index ddea968215ebd..e8590acb6a5a2 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java @@ -116,7 +116,7 @@ public void setup() { scheduler.startup(); final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(false); final MetadataCache metadataCache = - MetadataCache.kRaftMetadataCache(this.brokerProperties.brokerId(), () -> KRAFT_VERSION_1); + MetadataCache.kRaftMetadataCache(this.brokerProperties.nodeId(), () -> KRAFT_VERSION_1); this.quotaManagers = QuotaFactory.instantiate(this.brokerProperties, this.metrics, diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java index 179bcafdfa718..88ea8266ae515 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java @@ -153,7 +153,7 @@ public void setup() { setLogManager(logManager). setQuotaManagers(quotaManagers). setBrokerTopicStats(brokerTopicStats). - setMetadataCache(MetadataCache.kRaftMetadataCache(this.brokerProperties.brokerId(), () -> KRAFT_VERSION_1)). + setMetadataCache(MetadataCache.kRaftMetadataCache(this.brokerProperties.nodeId(), () -> KRAFT_VERSION_1)). setLogDirFailureChannel(failureChannel). setAlterPartitionManager(alterPartitionManager). build(); diff --git a/server/src/main/java/org/apache/kafka/server/config/ServerConfigs.java b/server/src/main/java/org/apache/kafka/server/config/ServerConfigs.java index cd5da5ae9fd00..c97df71ea4b02 100644 --- a/server/src/main/java/org/apache/kafka/server/config/ServerConfigs.java +++ b/server/src/main/java/org/apache/kafka/server/config/ServerConfigs.java @@ -35,21 +35,6 @@ public class ServerConfigs { /** ********* General Configuration ***********/ - public static final String RESERVED_BROKER_MAX_ID_CONFIG = "reserved.broker.max.id"; - public static final int RESERVED_BROKER_MAX_ID_DEFAULT = 1000; - public static final String RESERVED_BROKER_MAX_ID_DOC = "Max number that can be used for a broker.id"; - - public static final String BROKER_ID_GENERATION_ENABLE_CONFIG = "broker.id.generation.enable"; - public static final boolean BROKER_ID_GENERATION_ENABLE_DEFAULT = true; - public static final String BROKER_ID_GENERATION_ENABLE_DOC = "Enable automatic broker id generation on the server. When enabled the value configured for " + RESERVED_BROKER_MAX_ID_CONFIG + " should be reviewed."; - - - public static final String BROKER_ID_CONFIG = "broker.id"; - public static final int BROKER_ID_DEFAULT = -1; - public static final String BROKER_ID_DOC = "The broker id for this server. If unset, a unique broker id will be generated." + - "To avoid conflicts between ZooKeeper generated broker id's and user configured broker id's, generated broker ids " + - "start from " + RESERVED_BROKER_MAX_ID_CONFIG + " + 1."; - public static final String MESSAGE_MAX_BYTES_CONFIG = "message.max.bytes"; public static final String MESSAGE_MAX_BYTES_DOC = TopicConfig.MAX_MESSAGE_BYTES_DOC + "This can be set per topic with the topic level " + TopicConfig.MAX_MESSAGE_BYTES_CONFIG + " config."; @@ -131,9 +116,6 @@ public class ServerConfigs { "the StandardAuthorizer (which stores ACLs in the metadata log.) By default, all listeners included in controller.listener.names " + "will also be early start listeners. A listener should not appear in this list if it accepts external traffic."; public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(BROKER_ID_GENERATION_ENABLE_CONFIG, BOOLEAN, BROKER_ID_GENERATION_ENABLE_DEFAULT, MEDIUM, BROKER_ID_GENERATION_ENABLE_DOC) - .define(RESERVED_BROKER_MAX_ID_CONFIG, INT, RESERVED_BROKER_MAX_ID_DEFAULT, atLeast(0), MEDIUM, RESERVED_BROKER_MAX_ID_DOC) - .define(BROKER_ID_CONFIG, INT, BROKER_ID_DEFAULT, HIGH, BROKER_ID_DOC) .define(MESSAGE_MAX_BYTES_CONFIG, INT, LogConfig.DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), HIGH, MESSAGE_MAX_BYTES_DOC) .define(NUM_IO_THREADS_CONFIG, INT, NUM_IO_THREADS_DEFAULT, atLeast(1), HIGH, NUM_IO_THREADS_DOC) .define(NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG, INT, null, HIGH, NUM_REPLICA_ALTER_LOG_DIRS_THREADS_DOC) diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java index 2e0adda5858d7..1494fa74cd58d 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java @@ -136,7 +136,7 @@ public final class LocalTieredStorage implements RemoteStorageManager { /** * The ID of the broker which owns this instance of {@link LocalTieredStorage}. */ - public static final String BROKER_ID = "broker.id"; + public static final String NODE_ID = "node.id"; private static final String ROOT_STORAGE_DIR_NAME = "kafka-tiered-storage"; @@ -235,14 +235,14 @@ public void configure(Map configs) { final String shouldDeleteOnClose = (String) configs.get(DELETE_ON_CLOSE_CONFIG); final String transfererClass = (String) configs.get(TRANSFERER_CLASS_CONFIG); final String isDeleteEnabled = (String) configs.get(ENABLE_DELETE_API_CONFIG); - final Integer brokerIdInt = (Integer) configs.get(BROKER_ID); + final Integer nodeIdInt = (Integer) configs.get(NODE_ID); - if (brokerIdInt == null) { + if (nodeIdInt == null) { throw new InvalidConfigurationException( - "Broker ID is required to configure the LocalTieredStorage manager."); + "Node ID is required to configure the LocalTieredStorage manager."); } - brokerId = brokerIdInt; + brokerId = nodeIdInt; logger = new LogContext(format("[LocalTieredStorage Id=%d] ", brokerId)).logger(this.getClass()); if (shouldDeleteOnClose != null) { diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorageTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorageTest.java index f1358b838cbc8..3fe1f1aaa4218 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorageTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorageTest.java @@ -95,7 +95,7 @@ private void init(Map extraConfig, String testName) { Map config = new HashMap<>(); config.put(LocalTieredStorage.STORAGE_DIR_CONFIG, storageDir); config.put(LocalTieredStorage.DELETE_ON_CLOSE_CONFIG, "true"); - config.put(LocalTieredStorage.BROKER_ID, 1); + config.put(LocalTieredStorage.NODE_ID, 1); config.putAll(extraConfig); tieredStorage.configure(config); diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java index 2fdb9483fe628..ebfdb640db20d 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestContext.java @@ -285,7 +285,7 @@ public TopicSpec topicSpec(String topicName) { } public LocalTieredStorageSnapshot takeTieredStorageSnapshot() { - int aliveBrokerId = harness.aliveBrokers().head().config().brokerId(); + int aliveBrokerId = harness.aliveBrokers().head().config().nodeId(); return LocalTieredStorageSnapshot.takeSnapshot(remoteStorageManager(aliveBrokerId)); } @@ -322,7 +322,7 @@ public Admin admin() { } public boolean isActive(Integer brokerId) { - return harness.aliveBrokers().exists(b -> b.config().brokerId() == brokerId); + return harness.aliveBrokers().exists(b -> b.config().nodeId() == brokerId); } public boolean isAssignedReplica(TopicPartition topicPartition, Integer replicaId) diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java index ebf4a3f1269c9..539924293d436 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/TieredStorageTestHarness.java @@ -146,7 +146,7 @@ public static List remoteStorageManagers(Seq br storages.add((LocalTieredStorage) storageManager); } } else { - throw new AssertionError("Broker " + broker.config().brokerId() + throw new AssertionError("Broker " + broker.config().nodeId() + " does not have a remote log manager."); } }); @@ -155,7 +155,7 @@ public static List remoteStorageManagers(Seq br public static List localStorages(Seq brokers) { return CollectionConverters.asJava(brokers).stream() - .map(b -> new BrokerLocalStorage(b.config().brokerId(), CollectionConverters.asJava(b.config().logDirs().toSet()), + .map(b -> new BrokerLocalStorage(b.config().nodeId(), CollectionConverters.asJava(b.config().logDirs().toSet()), STORAGE_WAIT_TIMEOUT_SEC)) .collect(Collectors.toList()); } diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/TransactionsWithTieredStoreTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/TransactionsWithTieredStoreTest.java index 7d60c55f4f59b..f1cd7a0dc051b 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/TransactionsWithTieredStoreTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/TransactionsWithTieredStoreTest.java @@ -76,7 +76,7 @@ public Properties topicConfig() { public void maybeWaitForAtLeastOneSegmentUpload(scala.collection.Seq topicPartitions) { CollectionConverters.asJava(topicPartitions).forEach(topicPartition -> { List localStorages = CollectionConverters.asJava(brokers()).stream() - .map(b -> new BrokerLocalStorage(b.config().brokerId(), CollectionConverters.asJava(b.config().logDirs().toSet()), STORAGE_WAIT_TIMEOUT_SEC)) + .map(b -> new BrokerLocalStorage(b.config().nodeId(), CollectionConverters.asJava(b.config().logDirs().toSet()), STORAGE_WAIT_TIMEOUT_SEC)) .collect(Collectors.toList()); localStorages .stream() @@ -99,7 +99,7 @@ public void maybeVerifyLocalLogStartOffsets(scala.collection.immutable.Map { long offset = broker.replicaManager().localLog(entry.getKey()).get().localLogStartOffset(); - offsets.put(broker.config().brokerId(), offset); + offsets.put(broker.config().nodeId(), offset); return entry.getValue() == offset; }) ), () -> "local log start offset doesn't change to the expected position:" + partitionLocalStartOffsets + ", current position:" + offsets); @@ -109,7 +109,7 @@ private boolean isAssignedReplica(TopicPartition topicPartition, Integer replicaId) { Optional brokerOpt = CollectionConverters.asJava(brokers()) .stream() - .filter(b -> b.config().brokerId() == replicaId).findFirst(); + .filter(b -> b.config().nodeId() == replicaId).findFirst(); boolean isAssigned = false; if (brokerOpt.isPresent()) { HostedPartition hostedPartition = brokerOpt.get().replicaManager().getPartition(topicPartition); @@ -121,6 +121,6 @@ private boolean isAssignedReplica(TopicPartition topicPartition, } private boolean isAlive(Integer brokerId) { - return aliveBrokers().exists(b -> b.config().brokerId() == brokerId); + return aliveBrokers().exists(b -> b.config().nodeId() == brokerId); } } diff --git a/test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java b/test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java index 097f8c3e26d77..55fe1950fb959 100644 --- a/test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java +++ b/test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java @@ -596,7 +596,7 @@ public Map brokers() { public Map> raftManagers() { Map> results = new HashMap<>(); for (BrokerServer brokerServer : brokers().values()) { - results.put(brokerServer.config().brokerId(), brokerServer.sharedServer().raftManager()); + results.put(brokerServer.config().nodeId(), brokerServer.sharedServer().raftManager()); } for (ControllerServer controllerServer : controllers().values()) { if (!results.containsKey(controllerServer.config().nodeId())) {