diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index bc4d1860c97a8..0ba81cf87980a 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -314,9 +314,9 @@
+ files="(ClientQuotasImage|KafkaEventQueue|MetadataDelta|QuorumController|ReplicationControlManager|KRaftMigrationDriver|ClusterControlManager|MetaPropertiesEnsemble).java"/>
+ files="(ClientQuotasImage|KafkaEventQueue|ReplicationControlManager|FeatureControlManager|KRaftMigrationDriver|ScramControlManager|ClusterControlManager|MetadataDelta|MetaPropertiesEnsemble).java"/>
(dir, new OffsetCheckpointFile(new File(dir, RecoveryPointCheckpointFile), logDirFailureChannel))).toMap
@volatile private var logStartOffsetCheckpoints = liveLogDirs.map(dir =>
@@ -272,35 +271,31 @@ class LogManager(logDirs: Seq[File],
/**
* Retrieves the Uuid for the directory, given its absolute path.
*/
- def directoryId(dir: String): Option[Uuid] = dirIds.get(dir)
+ def directoryId(dir: String): Option[Uuid] = directoryIds.get(dir)
/**
* Determine directory ID for each directory with a meta.properties.
* If meta.properties does not include a directory ID, one is generated and persisted back to meta.properties.
* Directories without a meta.properties don't get a directory ID assigned.
*/
- private def directoryIds(dirs: Seq[File]): Map[String, Uuid] = {
- dirs.flatMap { dir =>
+ private def loadDirectoryIds(logDirs: Seq[File]): Map[String, Uuid] = {
+ val result = mutable.HashMap[String, Uuid]()
+ logDirs.foreach(logDir => {
try {
- val metadataCheckpoint = new BrokerMetadataCheckpoint(new File(dir, KafkaServer.brokerMetaPropsFile))
- metadataCheckpoint.read().map { props =>
- val rawMetaProperties = new RawMetaProperties(props)
- val uuid = rawMetaProperties.directoryId match {
- case Some(uuidStr) => Uuid.fromString(uuidStr)
- case None =>
- val uuid = DirectoryId.random()
- rawMetaProperties.directoryId = uuid.toString
- metadataCheckpoint.write(rawMetaProperties.props)
- uuid
- }
- dir.getAbsolutePath -> uuid
- }.toMap
+ val props = PropertiesUtils.readPropertiesFile(
+ new File(logDir, MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath)
+ val metaProps = new MetaProperties.Builder(props).build()
+ metaProps.directoryId().ifPresent(directoryId => {
+ result += (logDir.getAbsolutePath -> directoryId)
+ })
} catch {
- case e: IOException =>
- logDirFailureChannel.maybeAddOfflineLogDir(dir.getAbsolutePath, s"Disk error while loading ID $dir", e)
- None
- }
- }.toMap
+ case e: NoSuchFileException =>
+ info(s"No meta.properties file found in ${logDir}.")
+ case e: IOException =>
+ logDirFailureChannel.maybeAddOfflineLogDir(logDir.getAbsolutePath, s"Disk error while loading ID $logDir", e)
+ }
+ })
+ result
}
private def addLogToBeDeleted(log: UnifiedLog): Unit = {
diff --git a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala
deleted file mode 100755
index 3d00ebdf3f7c9..0000000000000
--- a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala
+++ /dev/null
@@ -1,273 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.server
-
-import java.io._
-import java.nio.file.{Files, NoSuchFileException}
-import java.util.Properties
-
-import kafka.common.InconsistentBrokerMetadataException
-import kafka.server.RawMetaProperties._
-import kafka.utils._
-import org.apache.kafka.common.KafkaException
-import org.apache.kafka.common.utils.Utils
-
-import scala.collection.mutable
-import scala.jdk.CollectionConverters._
-
-object RawMetaProperties {
- val ClusterIdKey = "cluster.id"
- val BrokerIdKey = "broker.id"
- val NodeIdKey = "node.id"
- val DirectoryIdKey = "directory.id"
- val VersionKey = "version"
-}
-
-class RawMetaProperties(val props: Properties = new Properties()) {
-
- def clusterId: Option[String] = {
- Option(props.getProperty(ClusterIdKey))
- }
-
- def clusterId_=(id: String): Unit = {
- props.setProperty(ClusterIdKey, id)
- }
-
- def brokerId: Option[Int] = {
- intValue(BrokerIdKey)
- }
-
- def brokerId_=(id: Int): Unit = {
- props.setProperty(BrokerIdKey, id.toString)
- }
-
- def nodeId: Option[Int] = {
- intValue(NodeIdKey)
- }
-
- def nodeId_=(id: Int): Unit = {
- props.setProperty(NodeIdKey, id.toString)
- }
-
- def directoryId: Option[String] = {
- Option(props.getProperty(DirectoryIdKey))
- }
-
- def directoryId_=(id: String): Unit = {
- props.setProperty(DirectoryIdKey, id)
- }
-
- def version: Int = {
- intValue(VersionKey).getOrElse(0)
- }
-
- def version_=(ver: Int): Unit = {
- props.setProperty(VersionKey, ver.toString)
- }
-
- private def intValue(key: String): Option[Int] = {
- try {
- Option(props.getProperty(key)).map(Integer.parseInt)
- } catch {
- case e: Throwable => throw new RuntimeException(s"Failed to parse $key property " +
- s"as an int: ${e.getMessage}")
- }
- }
-
- override def equals(that: Any): Boolean = that match {
- case other: RawMetaProperties => props.equals(other.props)
- case _ => false
- }
-
- override def hashCode(): Int = props.hashCode
-
- override def toString: String = {
- "{" + props.keySet().asScala.toList.asInstanceOf[List[String]].sorted.map {
- key => key + "=" + props.get(key)
- }.mkString(", ") + "}"
- }
-}
-
-object MetaProperties {
- def parse(properties: RawMetaProperties): MetaProperties = {
- val clusterId = require(ClusterIdKey, properties.clusterId)
- if (properties.version == 1) {
- val nodeId = require(NodeIdKey, properties.nodeId)
- new MetaProperties(clusterId, nodeId)
- } else if (properties.version == 0) {
- val brokerId = require(BrokerIdKey, properties.brokerId)
- new MetaProperties(clusterId, brokerId)
- } else {
- throw new RuntimeException(s"Expected version 0 or 1, but got version ${properties.version}")
- }
- }
-
- def require[T](key: String, value: Option[T]): T = {
- value.getOrElse(throw new RuntimeException(s"Failed to find required property $key."))
- }
-}
-
-case class ZkMetaProperties(
- clusterId: String,
- brokerId: Int
-) {
- def toProperties: Properties = {
- val properties = new RawMetaProperties()
- properties.version = 0
- properties.clusterId = clusterId
- properties.brokerId = brokerId
- properties.props
- }
-
- override def toString: String = {
- s"ZkMetaProperties(brokerId=$brokerId, clusterId=$clusterId)"
- }
-}
-
-case class MetaProperties(
- clusterId: String,
- nodeId: Int,
-) {
- private def toRawMetaProperties: RawMetaProperties = {
- val properties = new RawMetaProperties()
- properties.version = 1
- properties.clusterId = clusterId
- properties.nodeId = nodeId
- properties
- }
-
- def toProperties: Properties = {
- toRawMetaProperties.props
- }
-
- def toPropertiesWithDirectoryId(directoryId: String): Properties = {
- val properties = toRawMetaProperties
- properties.directoryId = directoryId
- properties.props
- }
-
- override def toString: String = {
- s"MetaProperties(clusterId=$clusterId, nodeId=$nodeId)"
- }
-}
-
-object BrokerMetadataCheckpoint extends Logging {
- def getBrokerMetadataAndOfflineDirs(
- logDirs: collection.Seq[String],
- ignoreMissing: Boolean,
- kraftMode: Boolean
- ): (RawMetaProperties, collection.Seq[String]) = {
- require(logDirs.nonEmpty, "Must have at least one log dir to read meta.properties")
-
- val brokerMetadataMap = mutable.HashMap[String, Properties]()
- val offlineDirs = mutable.ArrayBuffer.empty[String]
-
- for (logDir <- logDirs) {
- val brokerCheckpointFile = new File(logDir, KafkaServer.brokerMetaPropsFile)
- val brokerCheckpoint = new BrokerMetadataCheckpoint(brokerCheckpointFile)
-
- try {
- brokerCheckpoint.read() match {
- case Some(properties) =>
- brokerMetadataMap += logDir -> properties
- case None =>
- if (!ignoreMissing) {
- throw new KafkaException(s"No `meta.properties` found in $logDir " +
- "(have you run `kafka-storage.sh` to format the directory?)")
- }
- }
- } catch {
- case e: IOException =>
- offlineDirs += logDir
- error(s"Failed to read $brokerCheckpointFile", e)
- }
- }
-
- if (brokerMetadataMap.isEmpty) {
- (new RawMetaProperties(), offlineDirs)
- } else {
- // KRaft mode has to support handling both meta.properties versions 0 and 1 and has to
- // reconcile have multiple versions in different directories.
- val numDistinctMetaProperties = if (kraftMode) {
- brokerMetadataMap.values.map(props => MetaProperties.parse(new RawMetaProperties(props))).toSet.size
- } else {
- brokerMetadataMap.values.toSet.size
- }
- if (numDistinctMetaProperties > 1) {
- val builder = new StringBuilder
-
- for ((logDir, brokerMetadata) <- brokerMetadataMap)
- builder ++= s"- $logDir -> $brokerMetadata\n"
-
- throw new InconsistentBrokerMetadataException(
- s"BrokerMetadata is not consistent across log.dirs. This could happen if multiple brokers shared a log directory (log.dirs) " +
- s"or partial data was manually copied from another broker. Found:\n${builder.toString()}"
- )
- }
-
- val rawProps = new RawMetaProperties(brokerMetadataMap.head._2)
- (rawProps, offlineDirs)
- }
- }
-}
-
-/**
- * This class saves the metadata properties to a file
- */
-class BrokerMetadataCheckpoint(val file: File) extends Logging {
- private val lock = new Object()
-
- def write(properties: Properties): Unit = {
- lock synchronized {
- try {
- val temp = new File(file.getAbsolutePath + ".tmp")
- val fileOutputStream = new FileOutputStream(temp)
- try {
- properties.store(fileOutputStream, "")
- fileOutputStream.flush()
- fileOutputStream.getFD.sync()
- } finally {
- Utils.closeQuietly(fileOutputStream, temp.getName)
- }
- Utils.atomicMoveWithFallback(temp.toPath, file.toPath)
- } catch {
- case ie: IOException =>
- error("Failed to write meta.properties due to", ie)
- throw ie
- }
- }
- }
-
- def read(): Option[Properties] = {
- Files.deleteIfExists(new File(file.getPath + ".tmp").toPath) // try to delete any existing temp files for cleanliness
-
- val absolutePath = file.getAbsolutePath
- lock synchronized {
- try {
- Some(Utils.loadProps(absolutePath))
- } catch {
- case _: NoSuchFileException =>
- warn(s"No meta.properties file under dir $absolutePath")
- None
- case e: Exception =>
- error(s"Failed to read meta.properties file under dir $absolutePath", e)
- throw e
- }
- }
- }
-}
diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
index a9601663ddcec..30c25044282c3 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -56,7 +56,7 @@ import java.util.Optional
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.ReentrantLock
import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit, TimeoutException}
-import scala.collection.{Map, Seq}
+import scala.collection.Map
import scala.compat.java8.OptionConverters.RichOptionForJava8
import scala.jdk.CollectionConverters._
@@ -65,8 +65,7 @@ import scala.jdk.CollectionConverters._
* A Kafka broker that runs in KRaft (Kafka Raft) mode.
*/
class BrokerServer(
- val sharedServer: SharedServer,
- val initialOfflineDirs: Seq[String],
+ val sharedServer: SharedServer
) extends KafkaBroker {
val config = sharedServer.brokerConfig
val time = sharedServer.time
@@ -134,7 +133,7 @@ class BrokerServer(
@volatile var brokerTopicStats: BrokerTopicStats = _
- val clusterId: String = sharedServer.metaProps.clusterId
+ val clusterId: String = sharedServer.metaPropsEnsemble.clusterId().get()
var brokerMetadataPublisher: BrokerMetadataPublisher = _
@@ -197,8 +196,14 @@ class BrokerServer(
// Create log manager, but don't start it because we need to delay any potential unclean shutdown log recovery
// until we catch up on the metadata log and have up-to-date topic and broker configs.
- logManager = LogManager(config, initialOfflineDirs, metadataCache, kafkaScheduler, time,
- brokerTopicStats, logDirFailureChannel, keepPartitionMetadataFile = true)
+ logManager = LogManager(config,
+ sharedServer.metaPropsEnsemble.errorLogDirs().asScala.toSeq,
+ metadataCache,
+ kafkaScheduler,
+ time,
+ brokerTopicStats,
+ logDirFailureChannel,
+ keepPartitionMetadataFile = true)
remoteLogManagerOpt = createRemoteLogManager()
@@ -338,7 +343,7 @@ class BrokerServer(
lifecycleManager.start(
() => sharedServer.loader.lastAppliedOffset(),
brokerLifecycleChannelManager,
- sharedServer.metaProps.clusterId,
+ clusterId,
listenerInfo.toBrokerRegistrationRequest,
featuresRemapped,
logManager.readBrokerEpochFromCleanShutdownFiles()
diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala
index 0e0dc575e1b53..7b28c3372e1ed 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -138,7 +138,7 @@ class ControllerServer(
true
}
- def clusterId: String = sharedServer.clusterId()
+ def clusterId: String = sharedServer.clusterId
def startup(): Unit = {
if (!maybeChangeStatus(SHUTDOWN, STARTING)) return
@@ -238,7 +238,7 @@ class ControllerServer(
quorumControllerMetrics = new QuorumControllerMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry), time, config.migrationEnabled)
- new QuorumController.Builder(config.nodeId, sharedServer.metaProps.clusterId).
+ new QuorumController.Builder(config.nodeId, sharedServer.clusterId).
setTime(time).
setThreadNamePrefix(s"quorum-controller-${config.nodeId}-").
setConfigSchema(configSchema).
diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala
index c28cfc6f54371..32bd6f4937b20 100644
--- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala
@@ -18,7 +18,6 @@ package kafka.server
import java.io.File
import java.util.concurrent.CompletableFuture
-import kafka.common.InconsistentNodeIdException
import kafka.log.UnifiedLog
import kafka.metrics.KafkaMetricsReporter
import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole}
@@ -29,13 +28,16 @@ import org.apache.kafka.common.utils.{AppInfoParser, Time}
import org.apache.kafka.common.{KafkaException, Uuid}
import org.apache.kafka.metadata.KafkaConfigSchema
import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata}
+import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag.{REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR}
+import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble}
import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.server.config.ServerTopicConfigSynonyms
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.storage.internals.log.LogConfig
+import org.slf4j.Logger
-import java.util.Optional
-import scala.collection.Seq
+import java.util
+import java.util.{Optional, OptionalInt}
import scala.jdk.CollectionConverters._
/**
@@ -54,12 +56,13 @@ class KafkaRaftServer(
KafkaMetricsReporter.startReporters(VerifiableProperties(config.originals))
KafkaYammerMetrics.INSTANCE.configure(config.originals)
- private val (metaProps, bootstrapMetadata, offlineDirs) = KafkaRaftServer.initializeLogDirs(config)
+ private val (metaPropsEnsemble, bootstrapMetadata) =
+ KafkaRaftServer.initializeLogDirs(config, this.logger.underlying, this.logIdent)
private val metrics = Server.initializeMetrics(
config,
time,
- metaProps.clusterId
+ metaPropsEnsemble.clusterId().get()
)
private val controllerQuorumVotersFuture = CompletableFuture.completedFuture(
@@ -67,7 +70,7 @@ class KafkaRaftServer(
private val sharedServer = new SharedServer(
config,
- metaProps,
+ metaPropsEnsemble,
time,
metrics,
controllerQuorumVotersFuture,
@@ -75,7 +78,7 @@ class KafkaRaftServer(
)
private val broker: Option[BrokerServer] = if (config.processRoles.contains(BrokerRole)) {
- Some(new BrokerServer(sharedServer, offlineDirs))
+ Some(new BrokerServer(sharedServer))
} else {
None
}
@@ -135,39 +138,65 @@ object KafkaRaftServer {
* @return A tuple containing the loaded meta properties (which are guaranteed to
* be consistent across all log dirs) and the offline directories
*/
- def initializeLogDirs(config: KafkaConfig): (MetaProperties, BootstrapMetadata, Seq[String]) = {
- val logDirs = (config.logDirs.toSet + config.metadataLogDir).toSeq
- val (rawMetaProperties, offlineDirs) = BrokerMetadataCheckpoint.
- getBrokerMetadataAndOfflineDirs(logDirs, ignoreMissing = false, kraftMode = true)
-
- if (offlineDirs.contains(config.metadataLogDir)) {
- throw new KafkaException("Cannot start server since `meta.properties` could not be " +
- s"loaded from ${config.metadataLogDir}")
- }
-
+ def initializeLogDirs(
+ config: KafkaConfig,
+ log: Logger,
+ logPrefix: String
+ ): (MetaPropertiesEnsemble, BootstrapMetadata) = {
+ // Load and verify the original ensemble.
+ val loader = new MetaPropertiesEnsemble.Loader()
+ loader.addMetadataLogDir(config.metadataLogDir)
+ config.logDirs.foreach(loader.addLogDir(_))
+ val initialMetaPropsEnsemble = loader.load()
+ val verificationFlags = util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR)
+ initialMetaPropsEnsemble.verify(Optional.empty(), OptionalInt.of(config.nodeId), verificationFlags);
+
+ // Check that the __cluster_metadata-0 topic does not appear outside the metadata directory.
val metadataPartitionDirName = UnifiedLog.logDirName(MetadataPartition)
- val onlineNonMetadataDirs = logDirs.diff(offlineDirs :+ config.metadataLogDir)
- onlineNonMetadataDirs.foreach { logDir =>
- val metadataDir = new File(logDir, metadataPartitionDirName)
- if (metadataDir.exists) {
- throw new KafkaException(s"Found unexpected metadata location in data directory `$metadataDir` " +
- s"(the configured metadata directory is ${config.metadataLogDir}).")
+ initialMetaPropsEnsemble.logDirProps().keySet().forEach(logDir => {
+ if (!logDir.equals(config.metadataLogDir)) {
+ val clusterMetadataTopic = new File(logDir, metadataPartitionDirName)
+ if (clusterMetadataTopic.exists) {
+ throw new KafkaException(s"Found unexpected metadata location in data directory `$clusterMetadataTopic` " +
+ s"(the configured metadata directory is ${config.metadataLogDir}).")
+ }
}
+ })
+
+ // Set directory IDs on all directories. Rewrite the files if needed.
+ val metaPropsEnsemble = {
+ val copier = new MetaPropertiesEnsemble.Copier(initialMetaPropsEnsemble)
+ initialMetaPropsEnsemble.nonFailedDirectoryProps().forEachRemaining(e => {
+ val logDir = e.getKey
+ val metaProps = e.getValue
+ if (!metaProps.isPresent()) {
+ throw new RuntimeException(s"No `meta.properties` found in $logDir (have you run `kafka-storage.sh` " +
+ "to format the directory?)")
+ }
+ if (!metaProps.get().nodeId().isPresent()) {
+ throw new RuntimeException(s"Error: node ID not found in $logDir")
+ }
+ if (!metaProps.get().clusterId().isPresent()) {
+ throw new RuntimeException(s"Error: cluster ID not found in $logDir")
+ }
+ val builder = new MetaProperties.Builder(metaProps.get())
+ if (!builder.directoryId().isPresent()) {
+ builder.setDirectoryId(copier.generateValidDirectoryId())
+ }
+ copier.setLogDirProps(logDir, builder.build())
+ copier.setPreWriteHandler((logDir, _, _) => {
+ log.info("{}Rewriting {}{}meta.properties", logPrefix, logDir, File.separator)
+ })
+ })
+ copier.writeLogDirChanges()
+ copier.copy()
}
- val metaProperties = MetaProperties.parse(rawMetaProperties)
- if (config.nodeId != metaProperties.nodeId) {
- throw new InconsistentNodeIdException(
- s"Configured node.id `${config.nodeId}` doesn't match stored node.id `${metaProperties.nodeId}' in " +
- "meta.properties. If you moved your data, make sure your configured controller.id matches. " +
- "If you intend to create a new broker, you should remove all data in your data directories (log.dirs).")
- }
-
+ // Load the BootstrapMetadata.
val bootstrapDirectory = new BootstrapDirectory(config.metadataLogDir,
Optional.ofNullable(config.interBrokerProtocolVersionString))
val bootstrapMetadata = bootstrapDirectory.read()
-
- (metaProperties, bootstrapMetadata, offlineDirs.toSeq)
+ (metaPropsEnsemble, bootstrapMetadata)
}
val configSchema = new KafkaConfigSchema(Map(
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index 1bce5a600689f..322611ef3a3a0 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -18,7 +18,7 @@
package kafka.server
import kafka.cluster.{Broker, EndPoint}
-import kafka.common.{GenerateBrokerIdException, InconsistentBrokerIdException, InconsistentClusterIdException}
+import kafka.common.GenerateBrokerIdException
import kafka.controller.KafkaController
import kafka.coordinator.group.GroupCoordinatorAdapter
import kafka.coordinator.transaction.{ProducerIdManager, TransactionCoordinator}
@@ -47,6 +47,9 @@ import org.apache.kafka.common.security.{JaasContext, JaasUtils}
import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time, Utils}
import org.apache.kafka.common.{Endpoint, KafkaException, Node, TopicPartition}
import org.apache.kafka.coordinator.group.GroupCoordinator
+import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag
+import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag.REQUIRE_V0
+import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble}
import org.apache.kafka.metadata.{BrokerState, MetadataRecordSerde, VersionRange}
import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.server.authorizer.Authorizer
@@ -61,7 +64,9 @@ import org.apache.zookeeper.client.ZKClientConfig
import java.io.{File, IOException}
import java.net.{InetAddress, SocketTimeoutException}
-import java.util.OptionalLong
+import java.nio.file.{Files, Paths}
+import java.util
+import java.util.{Optional, OptionalInt, OptionalLong}
import java.util.concurrent._
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
import scala.collection.{Map, Seq}
@@ -69,9 +74,6 @@ import scala.compat.java8.OptionConverters.RichOptionForJava8
import scala.jdk.CollectionConverters._
object KafkaServer {
-
- val brokerMetaPropsFile = "meta.properties"
-
def zkClientConfigFromKafkaConfig(config: KafkaConfig, forceZkSslClientEnable: Boolean = false): ZKClientConfig = {
val clientConfig = new ZKClientConfig
if (config.zkSslClientEnable || forceZkSslClientEnable) {
@@ -168,9 +170,6 @@ class KafkaServer(
private var configRepository: ZkConfigRepository = _
val correlationId: AtomicInteger = new AtomicInteger(0)
- val brokerMetadataCheckpoints = config.logDirs.map { logDir =>
- (logDir, new BrokerMetadataCheckpoint(new File(logDir + File.separator + KafkaServer.brokerMetaPropsFile)))
- }.toMap
private var _clusterId: String = _
@volatile var _brokerTopicStats: BrokerTopicStats = _
@@ -229,23 +228,26 @@ class KafkaServer(
info(s"Cluster ID = $clusterId")
/* load metadata */
- val (preloadedBrokerMetadataCheckpoint, initialOfflineDirs) =
- BrokerMetadataCheckpoint.getBrokerMetadataAndOfflineDirs(config.logDirs, ignoreMissing = true, kraftMode = false)
-
- if (preloadedBrokerMetadataCheckpoint.version != 0) {
- throw new RuntimeException(s"Found unexpected version in loaded `meta.properties`: " +
- s"$preloadedBrokerMetadataCheckpoint. Zk-based brokers only support version 0 " +
- "(which is implicit when the `version` field is missing).")
+ val initialMetaPropsEnsemble = {
+ val loader = new MetaPropertiesEnsemble.Loader()
+ config.logDirs.foreach(loader.addLogDir(_))
+ loader.load()
}
- /* check cluster id */
- if (preloadedBrokerMetadataCheckpoint.clusterId.isDefined && preloadedBrokerMetadataCheckpoint.clusterId.get != clusterId)
- throw new InconsistentClusterIdException(
- s"The Cluster ID $clusterId doesn't match stored clusterId ${preloadedBrokerMetadataCheckpoint.clusterId} in meta.properties. " +
- s"The broker is trying to join the wrong cluster. Configured zookeeper.connect may be wrong.")
+ val verificationId = if (config.brokerId < 0) {
+ OptionalInt.empty()
+ } else {
+ OptionalInt.of(config.brokerId)
+ }
+ val verificationFlags = if (config.migrationEnabled) {
+ util.EnumSet.noneOf(classOf[VerificationFlag])
+ } else {
+ util.EnumSet.of(REQUIRE_V0)
+ }
+ initialMetaPropsEnsemble.verify(Optional.of(_clusterId), verificationId, verificationFlags)
/* generate brokerId */
- config.brokerId = getOrGenerateBrokerId(preloadedBrokerMetadataCheckpoint)
+ config.brokerId = getOrGenerateBrokerId(initialMetaPropsEnsemble)
logContext = new LogContext(s"[KafkaServer id=${config.brokerId}] ")
this.logIdent = logContext.logPrefix
@@ -270,10 +272,36 @@ class KafkaServer(
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size)
+ // Make sure all storage directories have meta.properties files.
+ val metaPropsEnsemble = {
+ val copier = new MetaPropertiesEnsemble.Copier(initialMetaPropsEnsemble)
+ initialMetaPropsEnsemble.nonFailedDirectoryProps().forEachRemaining(e => {
+ val logDir = e.getKey
+ val builder = new MetaProperties.Builder(e.getValue).
+ setClusterId(_clusterId).
+ setNodeId(config.brokerId)
+ if (!builder.directoryId().isPresent()) {
+ builder.setDirectoryId(copier.generateValidDirectoryId())
+ }
+ copier.setLogDirProps(logDir, builder.build())
+ })
+ copier.emptyLogDirs().clear()
+ copier.setPreWriteHandler((logDir, _, _) => {
+ info(s"Rewriting ${logDir}${File.separator}meta.properties")
+ Files.createDirectories(Paths.get(logDir))
+ })
+ copier.setWriteErrorHandler((logDir, e) => {
+ logDirFailureChannel.maybeAddOfflineLogDir(logDir, s"Error while writing meta.properties to $logDir", e)
+ })
+ copier.writeLogDirChanges()
+ copier.copy()
+ }
+ metaPropsEnsemble.verify(Optional.of(_clusterId), OptionalInt.of(config.brokerId), verificationFlags)
+
/* start log manager */
_logManager = LogManager(
config,
- initialOfflineDirs,
+ metaPropsEnsemble.errorLogDirs().asScala.toSeq,
configRepository,
kafkaScheduler,
time,
@@ -367,10 +395,6 @@ class KafkaServer(
val brokerInfo = createBrokerInfo
val brokerEpoch = zkClient.registerBroker(brokerInfo)
- // Now that the broker is successfully registered, checkpoint its metadata
- val zkMetaProperties = ZkMetaProperties(clusterId, config.brokerId)
- checkpointBrokerMetadata(zkMetaProperties)
-
/* start token manager */
tokenManager = new DelegationTokenManagerZk(config, tokenCache, time , zkClient)
tokenManager.startup()
@@ -390,7 +414,7 @@ class KafkaServer(
val controllerQuorumVotersFuture = CompletableFuture.completedFuture(
RaftConfig.parseVoterConnections(config.quorumVoters))
val raftManager = new KafkaRaftManager[ApiMessageAndVersion](
- clusterId,
+ metaPropsEnsemble.clusterId().get(),
config,
new MetadataRecordSerde,
KafkaRaftServer.MetadataPartition,
@@ -1020,24 +1044,6 @@ class KafkaServer(
}
}
- /**
- * Checkpoint the BrokerMetadata to all the online log.dirs
- *
- * @param brokerMetadata
- */
- private def checkpointBrokerMetadata(brokerMetadata: ZkMetaProperties) = {
- for (logDir <- config.logDirs if logManager.isLogDirOnline(new File(logDir).getAbsolutePath)) {
- val checkpoint = brokerMetadataCheckpoints(logDir)
- try {
- checkpoint.write(brokerMetadata.toProperties)
- } catch {
- case e: IOException =>
- val dirPath = checkpoint.file.getAbsolutePath
- logDirFailureChannel.maybeAddOfflineLogDir(dirPath, s"Error while writing meta.properties to $dirPath", e)
- }
- }
- }
-
/**
* Generates new brokerId if enabled or reads from meta.properties based on following conditions
*
@@ -1048,20 +1054,15 @@ class KafkaServer(
*
* @return The brokerId.
*/
- private def getOrGenerateBrokerId(brokerMetadata: RawMetaProperties): Int = {
- val brokerId = config.brokerId
-
- if (brokerId >= 0 && brokerMetadata.brokerId.exists(_ != brokerId))
- throw new InconsistentBrokerIdException(
- s"Configured broker.id $brokerId doesn't match stored broker.id ${brokerMetadata.brokerId} in meta.properties. " +
- s"If you moved your data, make sure your configured broker.id matches. " +
- s"If you intend to create a new broker, you should remove all data in your data directories (log.dirs).")
- else if (brokerMetadata.brokerId.isDefined)
- brokerMetadata.brokerId.get
- else if (brokerId < 0 && config.brokerIdGenerationEnable) // generate a new brokerId from Zookeeper
+ private def getOrGenerateBrokerId(metaPropsEnsemble: MetaPropertiesEnsemble): Int = {
+ if (config.brokerId >= 0) {
+ config.brokerId
+ } else if (metaPropsEnsemble.nodeId().isPresent) {
+ metaPropsEnsemble.nodeId().getAsInt()
+ } else if (config.brokerIdGenerationEnable) {
generateBrokerId()
- else
- brokerId
+ } else
+ throw new RuntimeException(s"No broker ID found, and ${config.brokerIdGenerationEnable} is disabled.")
}
/**
diff --git a/core/src/main/scala/kafka/server/SharedServer.scala b/core/src/main/scala/kafka/server/SharedServer.scala
index 7bbccf93bf027..abf7e0fa0decb 100644
--- a/core/src/main/scala/kafka/server/SharedServer.scala
+++ b/core/src/main/scala/kafka/server/SharedServer.scala
@@ -30,6 +30,7 @@ import org.apache.kafka.image.loader.MetadataLoader
import org.apache.kafka.image.loader.metrics.MetadataLoaderMetrics
import org.apache.kafka.image.publisher.{SnapshotEmitter, SnapshotGenerator}
import org.apache.kafka.metadata.MetadataRecordSerde
+import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble
import org.apache.kafka.raft.RaftConfig.AddressSpec
import org.apache.kafka.server.common.ApiMessageAndVersion
import org.apache.kafka.server.fault.{FaultHandler, LoggingFaultHandler, ProcessTerminatingFaultHandler}
@@ -87,7 +88,7 @@ class StandardFaultHandlerFactory extends FaultHandlerFactory {
*/
class SharedServer(
private val sharedServerConfig: KafkaConfig,
- val metaProps: MetaProperties,
+ val metaPropsEnsemble: MetaPropertiesEnsemble,
val time: Time,
private val _metrics: Metrics,
val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]],
@@ -110,7 +111,9 @@ class SharedServer(
@volatile var snapshotGenerator: SnapshotGenerator = _
@volatile var metadataLoaderMetrics: MetadataLoaderMetrics = _
- def clusterId(): String = metaProps.clusterId
+ def clusterId: String = metaPropsEnsemble.clusterId().get()
+
+ def nodeId: Int = metaPropsEnsemble.nodeId().getAsInt()
def isUsed(): Boolean = synchronized {
usedByController || usedByBroker
@@ -250,7 +253,7 @@ class SharedServer(
controllerServerMetrics = new ControllerMetadataMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry()))
}
val _raftManager = new KafkaRaftManager[ApiMessageAndVersion](
- clusterId(),
+ clusterId,
sharedServerConfig,
new MetadataRecordSerde,
KafkaRaftServer.MetadataPartition,
@@ -276,7 +279,7 @@ class SharedServer(
new AtomicReference[MetadataProvenance](MetadataProvenance.EMPTY))
}
val loaderBuilder = new MetadataLoader.Builder().
- setNodeId(metaProps.nodeId).
+ setNodeId(nodeId).
setTime(time).
setThreadNamePrefix(s"kafka-${sharedServerConfig.nodeId}-").
setFaultHandler(metadataLoaderFaultHandler).
@@ -284,11 +287,11 @@ class SharedServer(
setMetrics(metadataLoaderMetrics)
loader = loaderBuilder.build()
snapshotEmitter = new SnapshotEmitter.Builder().
- setNodeId(metaProps.nodeId).
+ setNodeId(nodeId).
setRaftClient(_raftManager.client).
build()
snapshotGenerator = new SnapshotGenerator.Builder(snapshotEmitter).
- setNodeId(metaProps.nodeId).
+ setNodeId(nodeId).
setTime(time).
setFaultHandler(metadataPublishingFaultHandler).
setMaxBytesSinceLastSnapshot(sharedServerConfig.metadataSnapshotMaxNewRecordBytes).
diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala
index e6a10e21a51a5..b2db8b6e8a1e9 100644
--- a/core/src/main/scala/kafka/tools/StorageTool.scala
+++ b/core/src/main/scala/kafka/tools/StorageTool.scala
@@ -17,14 +17,15 @@
package kafka.tools
+import kafka.server.KafkaConfig
+
import java.io.PrintStream
import java.nio.file.{Files, Paths}
-import kafka.server.{BrokerMetadataCheckpoint, KafkaConfig, KafkaServer, MetaProperties, RawMetaProperties}
import kafka.utils.{Exit, Logging}
import net.sourceforge.argparse4j.ArgumentParsers
import net.sourceforge.argparse4j.impl.Arguments.{append, store, storeTrue}
import net.sourceforge.argparse4j.inf.Namespace
-import org.apache.kafka.common.{DirectoryId, Uuid}
+import org.apache.kafka.common.Uuid
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata}
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
@@ -32,6 +33,8 @@ import org.apache.kafka.common.metadata.FeatureLevelRecord
import org.apache.kafka.common.metadata.UserScramCredentialRecord
import org.apache.kafka.common.security.scram.internals.ScramMechanism
import org.apache.kafka.common.security.scram.internals.ScramFormatter
+import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag
+import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import java.util
import java.util.Base64
@@ -60,7 +63,11 @@ object StorageTool extends Logging {
if (!metadataVersion.isKRaftSupported) {
throw new TerseFailure(s"Must specify a valid KRaft metadata version of at least 3.0.")
}
- val metaProperties = buildMetadataProperties(clusterId, config.get)
+ val metaProperties = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId).
+ setNodeId(config.get.nodeId).
+ build()
val metadataRecords : ArrayBuffer[ApiMessageAndVersion] = ArrayBuffer()
getUserScramCredentialRecords(namespace).foreach(userScramCredentialRecords => {
if (!metadataVersion.isScramSupported()) {
@@ -269,7 +276,7 @@ object StorageTool extends Logging {
def infoCommand(stream: PrintStream, selfManagedMode: Boolean, directories: Seq[String]): Int = {
val problems = new mutable.ArrayBuffer[String]
val foundDirectories = new mutable.ArrayBuffer[String]
- var prevMetadata: Option[RawMetaProperties] = None
+ var prevMetadata: Option[MetaProperties] = None
directories.sorted.foreach(directory => {
val directoryPath = Paths.get(directory)
if (!Files.isDirectory(directoryPath)) {
@@ -280,27 +287,26 @@ object StorageTool extends Logging {
}
} else {
foundDirectories += directoryPath.toString
- val metaPath = directoryPath.resolve(KafkaServer.brokerMetaPropsFile)
+ val metaPath = directoryPath.resolve(MetaPropertiesEnsemble.META_PROPERTIES_NAME)
if (!Files.exists(metaPath)) {
problems += s"$directoryPath is not formatted."
} else {
- val properties = Utils.loadProps(metaPath.toString)
- val rawMetaProperties = new RawMetaProperties(properties)
-
- val curMetadata = rawMetaProperties.version match {
- case 0 | 1 => Some(rawMetaProperties)
- case v =>
- problems += s"Unsupported version for $metaPath: $v"
- None
- }
-
- if (prevMetadata.isEmpty) {
- prevMetadata = curMetadata
- } else {
- if (!prevMetadata.get.equals(curMetadata.get)) {
- problems += s"Metadata for $metaPath was ${curMetadata.get}, " +
- s"but other directories featured ${prevMetadata.get}"
+ val properties = PropertiesUtils.readPropertiesFile(metaPath.toString)
+ try {
+ val curMetadata = new MetaProperties.Builder(properties).build()
+ if (prevMetadata.isEmpty) {
+ prevMetadata = Some(curMetadata)
+ } else {
+ if (!prevMetadata.get.clusterId().equals(curMetadata.clusterId())) {
+ problems += s"Mismatched cluster IDs between storage directories."
+ } else if (!prevMetadata.get.nodeId().equals(curMetadata.nodeId())) {
+ problems += s"Mismatched node IDs between storage directories."
+ }
}
+ } catch {
+ case e: Exception =>
+ e.printStackTrace(System.out)
+ problems += s"Error loading $metaPath: ${e.getMessage}"
}
}
}
@@ -308,11 +314,11 @@ object StorageTool extends Logging {
prevMetadata.foreach { prev =>
if (selfManagedMode) {
- if (prev.version == 0) {
+ if (prev.version.equals(MetaPropertiesVersion.V0)) {
problems += "The kafka configuration file appears to be for a cluster in KRaft mode, but " +
"the directories are formatted for legacy mode."
}
- } else if (prev.version == 1) {
+ } else if (prev.version.equals(MetaPropertiesVersion.V1)) {
problems += "The kafka configuration file appears to be for a legacy cluster, but " +
"the directories are formatted for a cluster in KRaft mode."
}
@@ -333,7 +339,9 @@ object StorageTool extends Logging {
}
prevMetadata.foreach { prev =>
- stream.println(s"Found metadata: ${prev}")
+ val sortedOutput = new util.TreeMap[String, String]()
+ prev.toProperties().entrySet().forEach(e => sortedOutput.put(e.getKey.toString, e.getValue.toString))
+ stream.println(s"Found metadata: ${sortedOutput}")
stream.println("")
}
@@ -382,7 +390,10 @@ object StorageTool extends Logging {
if (config.nodeId < 0) {
throw new TerseFailure(s"The node.id must be set to a non-negative integer. We saw ${config.nodeId}")
}
- new MetaProperties(effectiveClusterId.toString, config.nodeId)
+ new MetaProperties.Builder().
+ setClusterId(effectiveClusterId.toString).
+ setNodeId(config.nodeId).
+ build()
}
def formatCommand(
@@ -407,36 +418,42 @@ object StorageTool extends Logging {
if (directories.isEmpty) {
throw new TerseFailure("No log directories found in the configuration.")
}
-
- val unformattedDirectories = directories.filter(directory => {
- if (!Files.isDirectory(Paths.get(directory)) || !Files.exists(Paths.get(directory, KafkaServer.brokerMetaPropsFile))) {
- true
- } else if (!ignoreFormatted) {
- throw new TerseFailure(s"Log directory $directory is already formatted. " +
- "Use --ignore-formatted to ignore this directory and format the others.")
- } else {
- false
- }
- })
- if (unformattedDirectories.isEmpty) {
+ val loader = new MetaPropertiesEnsemble.Loader()
+ directories.foreach(loader.addLogDir(_))
+ val metaPropertiesEnsemble = loader.load()
+ metaPropertiesEnsemble.verify(metaProperties.clusterId(), metaProperties.nodeId(),
+ util.EnumSet.noneOf(classOf[VerificationFlag]))
+
+ System.out.println(s"metaPropertiesEnsemble=${metaPropertiesEnsemble}")
+ val copier = new MetaPropertiesEnsemble.Copier(metaPropertiesEnsemble)
+ if (!(ignoreFormatted || copier.logDirProps().isEmpty)) {
+ val firstLogDir = copier.logDirProps().keySet().iterator().next()
+ throw new TerseFailure(s"Log directory ${firstLogDir} is already formatted. " +
+ "Use --ignore-formatted to ignore this directory and format the others.")
+ }
+ if (!copier.errorLogDirs().isEmpty) {
+ val firstLogDir = copier.errorLogDirs().iterator().next()
+ throw new TerseFailure(s"I/O error trying to read log directory ${firstLogDir}.")
+ }
+ if (metaPropertiesEnsemble.emptyLogDirs().isEmpty) {
stream.println("All of the log directories are already formatted.")
+ } else {
+ metaPropertiesEnsemble.emptyLogDirs().forEach(logDir => {
+ copier.setLogDirProps(logDir, new MetaProperties.Builder(metaProperties).
+ setDirectoryId(copier.generateValidDirectoryId()).
+ build())
+ copier.setPreWriteHandler((logDir, isNew, metaProperties) => {
+ stream.println(s"Formatting ${logDir} with metadata.version ${metadataVersion}.")
+ Files.createDirectories(Paths.get(logDir))
+ val bootstrapDirectory = new BootstrapDirectory(logDir, Optional.empty())
+ bootstrapDirectory.writeBinaryFile(bootstrapMetadata)
+ })
+ copier.setWriteErrorHandler((logDir, e) => {
+ throw new TerseFailure(s"Error while writing meta.properties file ${logDir}: ${e.getMessage}")
+ })
+ copier.writeLogDirChanges()
+ })
}
- unformattedDirectories.foreach(directory => {
- try {
- Files.createDirectories(Paths.get(directory))
- } catch {
- case e: Throwable => throw new TerseFailure(s"Unable to create storage " +
- s"directory $directory: ${e.getMessage}")
- }
- val metaPropertiesPath = Paths.get(directory, KafkaServer.brokerMetaPropsFile)
- val checkpoint = new BrokerMetadataCheckpoint(metaPropertiesPath.toFile)
- checkpoint.write(metaProperties.toPropertiesWithDirectoryId(DirectoryId.random().toString))
-
- val bootstrapDirectory = new BootstrapDirectory(directory, Optional.empty())
- bootstrapDirectory.writeBinaryFile(bootstrapMetadata)
-
- stream.println(s"Formatting ${directory} with metadata.version ${metadataVersion}.")
- })
0
}
}
diff --git a/core/src/test/java/kafka/testkit/BrokerNode.java b/core/src/test/java/kafka/testkit/BrokerNode.java
index 1cc90276a7976..371cd3b7b4841 100644
--- a/core/src/test/java/kafka/testkit/BrokerNode.java
+++ b/core/src/test/java/kafka/testkit/BrokerNode.java
@@ -18,29 +18,40 @@
package kafka.testkit;
import org.apache.kafka.common.Uuid;
+import org.apache.kafka.metadata.properties.MetaProperties;
+import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble;
+import org.apache.kafka.metadata.properties.MetaPropertiesVersion;
+import java.io.File;
+import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-
-import static java.util.Collections.emptyMap;
+import java.util.Optional;
public class BrokerNode implements TestKitNode {
public static class Builder {
private int id = -1;
+ private String baseDirectory = null;
+ private Uuid clusterId = null;
private Uuid incarnationId = null;
- private String metadataDirectory = null;
private List logDataDirectories = null;
+ private String metadataDirectory = null;
+ private Map propertyOverrides = new HashMap<>();
+
+ public int id() {
+ return id;
+ }
public Builder setId(int id) {
this.id = id;
return this;
}
- public Builder setLogDirectories(List logDataDirectories) {
- this.logDataDirectories = logDataDirectories;
+ public Builder setIncarnationId(Uuid incarnationId) {
+ this.incarnationId = incarnationId;
return this;
}
@@ -49,70 +60,97 @@ public Builder setMetadataDirectory(String metadataDirectory) {
return this;
}
- BrokerNode build(
- String baseDirectory
+ public Builder setLogDirectories(List logDataDirectories) {
+ this.logDataDirectories = logDataDirectories;
+ return this;
+ }
+
+ public BrokerNode build(
+ String baseDirectory,
+ Uuid clusterId,
+ boolean combined
) {
if (id == -1) {
- throw new RuntimeException("You must set the node id");
+ throw new RuntimeException("You must set the node id.");
}
if (incarnationId == null) {
incarnationId = Uuid.randomUuid();
}
if (logDataDirectories == null) {
- logDataDirectories = Collections.
- singletonList(String.format("broker_%d_data0", id));
+ if (combined) {
+ logDataDirectories = Collections.
+ singletonList(String.format("combined_%d", id));
+ } else {
+ logDataDirectories = Collections.
+ singletonList(String.format("broker_%d_data0", id));
+ }
+ }
+ List absoluteLogDataDirectories = new ArrayList<>();
+ for (String logDir : logDataDirectories) {
+ if (Paths.get(logDir).isAbsolute()) {
+ absoluteLogDataDirectories.add(logDir);
+ } else {
+ absoluteLogDataDirectories.add(new File(baseDirectory, logDir).getAbsolutePath());
+ }
}
- logDataDirectories = TestKitNodes.absolutize(baseDirectory, logDataDirectories);
+ this.logDataDirectories = absoluteLogDataDirectories;
if (metadataDirectory == null) {
metadataDirectory = logDataDirectories.get(0);
+ } else if (!Paths.get(metadataDirectory).isAbsolute()) {
+ metadataDirectory = new File(baseDirectory, metadataDirectory).getAbsolutePath();
+ }
+ MetaPropertiesEnsemble.Copier copier =
+ new MetaPropertiesEnsemble.Copier(MetaPropertiesEnsemble.EMPTY);
+ copier.setMetaLogDir(Optional.of(metadataDirectory));
+ for (String logDir : logDataDirectories) {
+ copier.setLogDirProps(logDir, new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId.toString()).
+ setNodeId(id).
+ setDirectoryId(copier.generateValidDirectoryId()).
+ build());
}
- metadataDirectory = TestKitNodes.absolutize(baseDirectory, metadataDirectory);
- return new BrokerNode(id, incarnationId, metadataDirectory,
- logDataDirectories);
+ copier.setMetaLogDir(Optional.of(metadataDirectory));
+ return new BrokerNode(incarnationId,
+ copier.copy(),
+ combined,
+ propertyOverrides);
}
}
- private final int id;
private final Uuid incarnationId;
- private final String metadataDirectory;
- private final List logDataDirectories;
+ private final MetaPropertiesEnsemble initialMetaPropertiesEnsemble;
+ private final boolean combined;
private final Map propertyOverrides;
- BrokerNode(int id,
- Uuid incarnationId,
- String metadataDirectory,
- List logDataDirectories) {
- this(id, incarnationId, metadataDirectory, logDataDirectories, emptyMap());
- }
-
- BrokerNode(int id,
- Uuid incarnationId,
- String metadataDirectory,
- List logDataDirectories,
- Map propertyOverrides) {
- this.id = id;
+ BrokerNode(
+ Uuid incarnationId,
+ MetaPropertiesEnsemble initialMetaPropertiesEnsemble,
+ boolean combined,
+ Map propertyOverrides
+ ) {
this.incarnationId = incarnationId;
- this.metadataDirectory = metadataDirectory;
- this.logDataDirectories = new ArrayList<>(logDataDirectories);
+ this.initialMetaPropertiesEnsemble = initialMetaPropertiesEnsemble;
+ this.combined = combined;
this.propertyOverrides = new HashMap<>(propertyOverrides);
}
- @Override
- public int id() {
- return id;
- }
-
public Uuid incarnationId() {
return incarnationId;
}
@Override
- public String metadataDirectory() {
- return metadataDirectory;
+ public MetaPropertiesEnsemble initialMetaPropertiesEnsemble() {
+ return initialMetaPropertiesEnsemble;
+ }
+
+ @Override
+ public boolean combined() {
+ return combined;
}
public List logDataDirectories() {
- return logDataDirectories;
+ return new ArrayList<>(initialMetaPropertiesEnsemble.logDirProps().keySet());
}
public Map propertyOverrides() {
diff --git a/core/src/test/java/kafka/testkit/ControllerNode.java b/core/src/test/java/kafka/testkit/ControllerNode.java
index 11901d1f38731..02ac9a6dadbf7 100644
--- a/core/src/test/java/kafka/testkit/ControllerNode.java
+++ b/core/src/test/java/kafka/testkit/ControllerNode.java
@@ -17,10 +17,25 @@
package kafka.testkit;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.metadata.properties.MetaProperties;
+import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble;
+import org.apache.kafka.metadata.properties.MetaPropertiesVersion;
+
+import java.io.File;
+import java.nio.file.Paths;
+import java.util.Optional;
+
public class ControllerNode implements TestKitNode {
public static class Builder {
private int id = -1;
+ private String baseDirectory = null;
private String metadataDirectory = null;
+ private Uuid clusterId = null;
+
+ public int id() {
+ return id;
+ }
public Builder setId(int id) {
this.id = id;
@@ -33,34 +48,58 @@ public Builder setMetadataDirectory(String metadataDirectory) {
}
public ControllerNode build(
- String baseDirectory
+ String baseDirectory,
+ Uuid clusterId,
+ boolean combined
) {
if (id == -1) {
- throw new RuntimeException("You must set the node id");
+ throw new RuntimeException("You must set the node id.");
+ }
+ if (baseDirectory == null) {
+ throw new RuntimeException("You must set the base directory.");
}
if (metadataDirectory == null) {
- metadataDirectory = String.format("controller_%d", id);
+ if (combined) {
+ metadataDirectory = String.format("combined_%d", id);
+ } else {
+ metadataDirectory = String.format("controller_%d", id);
+ }
+ }
+ if (!Paths.get(metadataDirectory).isAbsolute()) {
+ metadataDirectory = new File(baseDirectory, metadataDirectory).getAbsolutePath();
}
- metadataDirectory = TestKitNodes.absolutize(baseDirectory, metadataDirectory);
- return new ControllerNode(id, metadataDirectory);
+ MetaPropertiesEnsemble.Copier copier =
+ new MetaPropertiesEnsemble.Copier(MetaPropertiesEnsemble.EMPTY);
+ copier.setMetaLogDir(Optional.of(metadataDirectory));
+ copier.setLogDirProps(metadataDirectory, new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId.toString()).
+ setNodeId(id).
+ setDirectoryId(copier.generateValidDirectoryId()).
+ build());
+ return new ControllerNode(copier.copy(), combined);
}
}
- private final int id;
- private final String metadataDirectory;
+ private final MetaPropertiesEnsemble initialMetaPropertiesEnsemble;
+
+ private final boolean combined;
- ControllerNode(int id, String metadataDirectory) {
- this.id = id;
- this.metadataDirectory = metadataDirectory;
+ ControllerNode(
+ MetaPropertiesEnsemble initialMetaPropertiesEnsemble,
+ boolean combined
+ ) {
+ this.initialMetaPropertiesEnsemble = initialMetaPropertiesEnsemble;
+ this.combined = combined;
}
@Override
- public int id() {
- return id;
+ public MetaPropertiesEnsemble initialMetaPropertiesEnsemble() {
+ return initialMetaPropertiesEnsemble;
}
@Override
- public String metadataDirectory() {
- return metadataDirectory;
+ public boolean combined() {
+ return combined;
}
}
diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
index 7b6bef6f64c62..77f1832bbe323 100644
--- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
+++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
@@ -25,9 +25,6 @@
import kafka.server.KafkaConfig;
import kafka.server.KafkaConfig$;
import kafka.server.KafkaRaftServer;
-import kafka.server.MetaProperties;
-import kafka.tools.StorageTool;
-import kafka.utils.Logging;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.admin.AdminClientConfig;
import org.apache.kafka.common.Node;
@@ -37,10 +34,11 @@
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.controller.Controller;
-import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
+import org.apache.kafka.metadata.bootstrap.BootstrapDirectory;
+import org.apache.kafka.metadata.properties.MetaProperties;
+import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble;
import org.apache.kafka.raft.RaftConfig;
import org.apache.kafka.server.common.ApiMessageAndVersion;
-import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.fault.FaultHandler;
import org.apache.kafka.server.fault.MockFaultHandler;
import org.apache.kafka.test.TestUtils;
@@ -48,12 +46,8 @@
import org.slf4j.LoggerFactory;
import org.slf4j.event.Level;
import scala.Option;
-import scala.collection.JavaConverters;
-import java.io.ByteArrayOutputStream;
import java.io.File;
-import java.io.IOException;
-import java.io.PrintStream;
import java.net.InetSocketAddress;
import java.nio.file.Files;
import java.nio.file.Paths;
@@ -65,6 +59,7 @@
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
+import java.util.Optional;
import java.util.Properties;
import java.util.TreeMap;
import java.util.concurrent.CompletableFuture;
@@ -74,7 +69,6 @@
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.Consumer;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -173,10 +167,14 @@ private KafkaConfig createNodeConfig(TestKitNode node) {
props.put(KafkaConfig$.MODULE$.MetadataLogDirProp(),
node.metadataDirectory());
}
- // Set the log.dirs according to the broker node setting (if there is a broker node)
if (brokerNode != null) {
+ // Set the log.dirs according to the broker node setting (if there is a broker node)
props.put(KafkaConfig$.MODULE$.LogDirsProp(),
String.join(",", brokerNode.logDataDirectories()));
+ } else {
+ // Set log.dirs equal to the metadata directory if there is just a controller.
+ props.put(KafkaConfig$.MODULE$.LogDirsProp(),
+ controllerNode.metadataDirectory());
}
props.put(KafkaConfig$.MODULE$.ListenerSecurityProtocolMapProp(),
"EXTERNAL:PLAINTEXT,CONTROLLER:PLAINTEXT");
@@ -216,17 +214,16 @@ public KafkaClusterTestKit build() throws Exception {
ExecutorService executorService = null;
ControllerQuorumVotersFutureManager connectFutureManager =
new ControllerQuorumVotersFutureManager(nodes.controllerNodes().size());
- File baseDirectory = new File(nodes.baseDirectory());
+ File baseDirectory = null;
try {
+ baseDirectory = new File(nodes.baseDirectory());
executorService = Executors.newFixedThreadPool(numOfExecutorThreads,
ThreadUtils.createThreadFactory("kafka-cluster-test-kit-executor-%d", false));
for (ControllerNode node : nodes.controllerNodes().values()) {
setupNodeDirectories(baseDirectory, node.metadataDirectory(), Collections.emptyList());
- BootstrapMetadata bootstrapMetadata = BootstrapMetadata.
- fromVersion(nodes.bootstrapMetadataVersion(), "testkit");
SharedServer sharedServer = new SharedServer(createNodeConfig(node),
- MetaProperties.apply(nodes.clusterId().toString(), node.id()),
+ node.initialMetaPropertiesEnsemble(),
Time.SYSTEM,
new Metrics(),
connectFutureManager.future,
@@ -236,7 +233,7 @@ public KafkaClusterTestKit build() throws Exception {
controller = new ControllerServer(
sharedServer,
KafkaRaftServer.configSchema(),
- bootstrapMetadata);
+ nodes.bootstrapMetadata());
} catch (Throwable e) {
log.error("Error creating controller {}", node.id(), e);
Utils.swallow(log, Level.WARN, "sharedServer.stopForController error", () -> sharedServer.stopForController());
@@ -256,16 +253,14 @@ public KafkaClusterTestKit build() throws Exception {
for (BrokerNode node : nodes.brokerNodes().values()) {
SharedServer sharedServer = jointServers.computeIfAbsent(node.id(),
id -> new SharedServer(createNodeConfig(node),
- MetaProperties.apply(nodes.clusterId().toString(), id),
+ node.initialMetaPropertiesEnsemble(),
Time.SYSTEM,
new Metrics(),
connectFutureManager.future,
faultHandlerFactory));
BrokerServer broker = null;
try {
- broker = new BrokerServer(
- sharedServer,
- JavaConverters.asScalaBuffer(Collections.emptyList()).toSeq());
+ broker = new BrokerServer(sharedServer);
} catch (Throwable e) {
log.error("Error creating broker {}", node.id(), e);
Utils.swallow(log, Level.WARN, "sharedServer.stopForBroker error", () -> sharedServer.stopForBroker());
@@ -360,19 +355,17 @@ private KafkaClusterTestKit(
public void format() throws Exception {
List> futures = new ArrayList<>();
try {
- for (Entry entry : controllers.entrySet()) {
- int nodeId = entry.getKey();
- ControllerServer controller = entry.getValue();
- formatNodeAndLog(nodes.controllerProperties(nodeId), controller.config().metadataLogDir(),
- controller, futures::add);
+ for (ControllerServer controller : controllers.values()) {
+ futures.add(executorService.submit(() -> {
+ formatNode(controller.sharedServer().metaPropsEnsemble(), true);
+ }));
}
for (Entry entry : brokers.entrySet()) {
- int nodeId = entry.getKey();
- if (!controllers.containsKey(nodeId)) {
- BrokerServer broker = entry.getValue();
- formatNodeAndLog(nodes.brokerProperties(nodeId), broker.config().metadataLogDir(),
- broker, futures::add);
- }
+ BrokerServer broker = entry.getValue();
+ futures.add(executorService.submit(() -> {
+ formatNode(broker.sharedServer().metaPropsEnsemble(),
+ !nodes().brokerNodes().get(entry.getKey()).combined());
+ }));
}
for (Future> future: futures) {
future.get();
@@ -385,25 +378,30 @@ public void format() throws Exception {
}
}
- private void formatNodeAndLog(MetaProperties properties, String metadataLogDir, Logging loggingMixin,
- Consumer> futureConsumer) {
- futureConsumer.accept(executorService.submit(() -> {
- try (ByteArrayOutputStream stream = new ByteArrayOutputStream()) {
- try (PrintStream out = new PrintStream(stream)) {
- StorageTool.formatCommand(out,
- JavaConverters.asScalaBuffer(Collections.singletonList(metadataLogDir)).toSeq(),
- properties,
- MetadataVersion.MINIMUM_BOOTSTRAP_VERSION,
- false);
- } finally {
- for (String line : stream.toString().split(String.format("%n"))) {
- loggingMixin.info(() -> line);
- }
+ private void formatNode(
+ MetaPropertiesEnsemble ensemble,
+ boolean writeMetadataDirectory
+ ) {
+ try {
+ MetaPropertiesEnsemble.Copier copier =
+ new MetaPropertiesEnsemble.Copier(MetaPropertiesEnsemble.EMPTY);
+ for (Entry entry : ensemble.logDirProps().entrySet()) {
+ String logDir = entry.getKey();
+ if (writeMetadataDirectory || (!ensemble.metadataLogDir().equals(Optional.of(logDir)))) {
+ log.trace("Adding {} to the list of directories to format.", logDir);
+ copier.setLogDirProps(logDir, entry.getValue());
}
- } catch (IOException e) {
- throw new RuntimeException(e);
}
- }));
+ copier.setPreWriteHandler((logDir, isNew, metaProperties) -> {
+ log.info("Formatting {}.", logDir);
+ Files.createDirectories(Paths.get(logDir));
+ BootstrapDirectory bootstrapDirectory = new BootstrapDirectory(logDir, Optional.empty());
+ bootstrapDirectory.writeBinaryFile(nodes.bootstrapMetadata());
+ });
+ copier.writeLogDirChanges();
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to format node " + ensemble.nodeId(), e);
+ }
}
public void startup() throws ExecutionException, InterruptedException {
diff --git a/core/src/test/java/kafka/testkit/TestKitNode.java b/core/src/test/java/kafka/testkit/TestKitNode.java
index a5423d135f9cb..17f73e8256984 100644
--- a/core/src/test/java/kafka/testkit/TestKitNode.java
+++ b/core/src/test/java/kafka/testkit/TestKitNode.java
@@ -17,7 +17,18 @@
package kafka.testkit;
+import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble;
+
public interface TestKitNode {
- int id();
- String metadataDirectory();
+ default int id() {
+ return initialMetaPropertiesEnsemble().nodeId().getAsInt();
+ }
+
+ default String metadataDirectory() {
+ return initialMetaPropertiesEnsemble().metadataLogDir().get();
+ }
+
+ MetaPropertiesEnsemble initialMetaPropertiesEnsemble();
+
+ boolean combined();
}
diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java b/core/src/test/java/kafka/testkit/TestKitNodes.java
index 29aee50fb9cd8..0f0d8939fe960 100644
--- a/core/src/test/java/kafka/testkit/TestKitNodes.java
+++ b/core/src/test/java/kafka/testkit/TestKitNodes.java
@@ -17,19 +17,14 @@
package kafka.testkit;
-import kafka.server.MetaProperties;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.network.ListenerName;
-import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.test.TestUtils;
-import java.io.File;
-import java.io.IOException;
+import java.nio.file.Files;
import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
@@ -70,8 +65,9 @@ public Builder setNumControllerNodes(int numControllerNodes) {
if (!controllerNodeBuilders.isEmpty()) {
nextId = controllerNodeBuilders.lastKey() + 1;
}
- controllerNodeBuilders.put(nextId, new ControllerNode.Builder().
- setId(nextId));
+ controllerNodeBuilders.put(nextId,
+ new ControllerNode.Builder().
+ setId(nextId));
}
return this;
}
@@ -88,33 +84,36 @@ public Builder setNumBrokerNodes(int numBrokerNodes) {
if (!brokerNodeBuilders.isEmpty()) {
nextId = brokerNodeBuilders.lastKey() + 1;
}
- brokerNodeBuilders.put(nextId, new BrokerNode.Builder().
- setId(nextId));
+ brokerNodeBuilders.put(nextId,
+ new BrokerNode.Builder().
+ setId(nextId));
}
return this;
}
public TestKitNodes build() {
- if (clusterId == null) {
- clusterId = Uuid.randomUuid();
- }
- if (bootstrapMetadataVersion == null) {
- bootstrapMetadataVersion = MetadataVersion.latest();
- }
- String baseDirectory = TestUtils.tempDirectory("kafka_" + clusterId).getAbsolutePath();
+ String baseDirectory = TestUtils.tempDirectory().getAbsolutePath();
try {
- NavigableMap controllerNodes = new TreeMap<>();
- for (ControllerNode.Builder controllerNodeBuilder : controllerNodeBuilders.values()) {
- ControllerNode controllerNode = controllerNodeBuilder.build(baseDirectory);
- if (controllerNodes.put(controllerNode.id(), controllerNode) != null) {
- throw new RuntimeException("More than one controller claimed ID " + controllerNode.id());
+ if (clusterId == null) {
+ clusterId = Uuid.randomUuid();
+ }
+ if (bootstrapMetadataVersion == null) {
+ bootstrapMetadataVersion = MetadataVersion.latest();
+ }
+ TreeMap controllerNodes = new TreeMap<>();
+ for (ControllerNode.Builder builder : controllerNodeBuilders.values()) {
+ ControllerNode node = builder.
+ build(baseDirectory, clusterId, brokerNodeBuilders.containsKey(builder.id()));
+ if (controllerNodes.put(node.id(), node) != null) {
+ throw new RuntimeException("Duplicate builder for controller " + node.id());
}
}
- NavigableMap brokerNodes = new TreeMap<>();
- for (BrokerNode.Builder brokerNodeBuilder : brokerNodeBuilders.values()) {
- BrokerNode brokerNode = brokerNodeBuilder.build(baseDirectory);
- if (brokerNodes.put(brokerNode.id(), brokerNode) != null) {
- throw new RuntimeException("More than one broker claimed ID " + brokerNode.id());
+ TreeMap brokerNodes = new TreeMap<>();
+ for (BrokerNode.Builder builder : brokerNodeBuilders.values()) {
+ BrokerNode node = builder.
+ build(baseDirectory, clusterId, controllerNodeBuilders.containsKey(builder.id()));
+ if (brokerNodes.put(node.id(), node) != null) {
+ throw new RuntimeException("Duplicate builder for broker " + node.id());
}
}
return new TestKitNodes(baseDirectory,
@@ -124,9 +123,9 @@ public TestKitNodes build() {
brokerNodes);
} catch (Exception e) {
try {
- Utils.delete(new File(baseDirectory));
- } catch (IOException x) {
- throw new RuntimeException(x);
+ Files.delete(Paths.get(baseDirectory));
+ } catch (Exception x) {
+ throw new RuntimeException("Failed to delete base directory " + baseDirectory, x);
}
throw e;
}
@@ -150,10 +149,6 @@ private int startControllerId() {
private final NavigableMap controllerNodes;
private final NavigableMap brokerNodes;
- public boolean isCombined(int node) {
- return controllerNodes.containsKey(node) && brokerNodes.containsKey(node);
- }
-
private TestKitNodes(
String baseDirectory,
Uuid clusterId,
@@ -168,6 +163,10 @@ private TestKitNodes(
this.brokerNodes = brokerNodes;
}
+ public boolean isCombined(int node) {
+ return controllerNodes.containsKey(node) && brokerNodes.containsKey(node);
+ }
+
public String baseDirectory() {
return baseDirectory;
}
@@ -184,16 +183,12 @@ public Map controllerNodes() {
return controllerNodes;
}
- public NavigableMap brokerNodes() {
- return brokerNodes;
- }
-
- public MetaProperties controllerProperties(int id) {
- return MetaProperties.apply(clusterId.toString(), id);
+ public BootstrapMetadata bootstrapMetadata() {
+ return BootstrapMetadata.fromVersion(bootstrapMetadataVersion(), "testkit");
}
- public MetaProperties brokerProperties(int id) {
- return MetaProperties.apply(clusterId.toString(), id);
+ public NavigableMap brokerNodes() {
+ return brokerNodes;
}
public ListenerName interBrokerListenerName() {
@@ -207,19 +202,4 @@ public ListenerName externalListenerName() {
public ListenerName controllerListenerName() {
return new ListenerName("CONTROLLER");
}
-
- static List absolutize(String base, Collection directories) {
- List newDirectories = new ArrayList<>();
- for (String directory : directories) {
- newDirectories.add(absolutize(base, directory));
- }
- return newDirectories;
- }
-
- static String absolutize(String base, String directory) {
- if (Paths.get(directory).isAbsolute()) {
- return directory;
- }
- return Paths.get(base, directory).toAbsolutePath().toString();
- }
}
diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
index 0a2cf270bf909..19836ac299ca5 100755
--- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
@@ -20,7 +20,7 @@ package kafka.server
import java.io.{ByteArrayOutputStream, File, PrintStream}
import java.net.InetSocketAddress
import java.util
-import java.util.{Collections, Properties}
+import java.util.{Collections, Optional, OptionalInt, Properties}
import java.util.concurrent.{CompletableFuture, TimeUnit}
import javax.security.auth.login.Configuration
import kafka.tools.StorageTool
@@ -33,6 +33,8 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.{Exit, Time}
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata
import org.apache.kafka.common.metadata.FeatureLevelRecord
+import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag.{REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR}
+import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion}
import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec}
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler}
@@ -41,6 +43,7 @@ import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterAll, AfterEach, BeforeAll, BeforeEach, Tag, TestInfo}
+import java.nio.file.{Files, Paths}
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.ListBuffer
import scala.collection.{Seq, immutable}
@@ -96,16 +99,37 @@ class KRaftQuorumImplementation(
startup: Boolean,
threadNamePrefix: Option[String],
): KafkaBroker = {
+ val metaPropertiesEnsemble = {
+ val loader = new MetaPropertiesEnsemble.Loader()
+ config.logDirs.foreach(loader.addLogDir(_))
+ loader.addMetadataLogDir(config.metadataLogDir)
+ val ensemble = loader.load()
+ val copier = new MetaPropertiesEnsemble.Copier(ensemble)
+ ensemble.emptyLogDirs().forEach(logDir => {
+ copier.setLogDirProps(logDir, new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId).
+ setNodeId(config.nodeId).
+ build())
+ })
+ copier.setPreWriteHandler((logDir, _, _) => {
+ Files.createDirectories(Paths.get(logDir));
+ })
+ copier.writeLogDirChanges()
+ copier.copy()
+ }
+ metaPropertiesEnsemble.verify(Optional.of(clusterId),
+ OptionalInt.of(config.nodeId),
+ util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR))
val sharedServer = new SharedServer(config,
- new MetaProperties(clusterId, config.nodeId),
+ metaPropertiesEnsemble,
Time.SYSTEM,
new Metrics(),
controllerQuorumVotersFuture,
faultHandlerFactory)
var broker: BrokerServer = null
try {
- broker = new BrokerServer(sharedServer,
- initialOfflineDirs = Seq())
+ broker = new BrokerServer(sharedServer)
if (startup) broker.startup()
broker
} catch {
@@ -308,7 +332,11 @@ abstract class QuorumTestHarness extends Logging {
}
val nodeId = Integer.parseInt(props.getProperty(KafkaConfig.NodeIdProp))
val metadataDir = TestUtils.tempDir()
- val metaProperties = new MetaProperties(Uuid.randomUuid().toString, nodeId)
+ val metaProperties = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(Uuid.randomUuid().toString).
+ setNodeId(nodeId).
+ build()
formatDirectories(immutable.Seq(metadataDir.getAbsolutePath), metaProperties)
val metadataRecords = new util.ArrayList[ApiMessageAndVersion]
@@ -330,8 +358,14 @@ abstract class QuorumTestHarness extends Logging {
props.setProperty(KafkaConfig.QuorumVotersProp, s"${nodeId}@localhost:0")
val config = new KafkaConfig(props)
val controllerQuorumVotersFuture = new CompletableFuture[util.Map[Integer, AddressSpec]]
+ val metaPropertiesEnsemble = new MetaPropertiesEnsemble.Loader().
+ addMetadataLogDir(metadataDir.getAbsolutePath).
+ load()
+ metaPropertiesEnsemble.verify(Optional.of(metaProperties.clusterId().get()),
+ OptionalInt.of(nodeId),
+ util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR))
val sharedServer = new SharedServer(config,
- metaProperties,
+ metaPropertiesEnsemble,
Time.SYSTEM,
new Metrics(),
controllerQuorumVotersFuture,
@@ -363,7 +397,7 @@ abstract class QuorumTestHarness extends Logging {
faultHandlerFactory,
metadataDir,
controllerQuorumVotersFuture,
- metaProperties.clusterId,
+ metaProperties.clusterId.get(),
this,
faultHandler)
}
diff --git a/core/src/test/scala/kafka/server/BrokerMetadataCheckpointTest.scala b/core/src/test/scala/kafka/server/BrokerMetadataCheckpointTest.scala
deleted file mode 100644
index bc87dd91168aa..0000000000000
--- a/core/src/test/scala/kafka/server/BrokerMetadataCheckpointTest.scala
+++ /dev/null
@@ -1,234 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
- * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
- * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
- * License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
- * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations under the License.
- */
-package kafka.server
-
-import kafka.utils.{CoreUtils, Logging}
-
-import java.io.{File, FileOutputStream}
-import java.util.Properties
-import org.apache.kafka.common.utils.Utils
-import org.apache.kafka.test.TestUtils
-import org.junit.jupiter.api.Assertions._
-import org.junit.jupiter.api.Test
-
-import scala.collection.mutable
-
-class BrokerMetadataCheckpointTest extends Logging {
- private val clusterIdBase64 = "H3KKO4NTRPaCWtEmm3vW7A"
-
- @Test
- def testReadWithNonExistentFile(): Unit = {
- assertEquals(None, new BrokerMetadataCheckpoint(new File("path/that/does/not/exist")).read())
- }
-
- @Test
- def testCreateZkMetadataProperties(): Unit = {
- val meta = ZkMetaProperties("7bc79ca1-9746-42a3-a35a-efb3cde44492", 3)
- val properties = meta.toProperties
- val parsed = new RawMetaProperties(properties)
- assertEquals(0, parsed.version)
- assertEquals(Some(meta.clusterId), parsed.clusterId)
- assertEquals(Some(meta.brokerId), parsed.brokerId)
- }
-
- @Test
- def testParseRawMetaPropertiesWithoutVersion(): Unit = {
- val brokerId = 1
- val clusterId = "7bc79ca1-9746-42a3-a35a-efb3cde44492"
-
- val properties = new Properties()
- properties.put(RawMetaProperties.BrokerIdKey, brokerId.toString)
- properties.put(RawMetaProperties.ClusterIdKey, clusterId)
-
- val parsed = new RawMetaProperties(properties)
- assertEquals(Some(brokerId), parsed.brokerId)
- assertEquals(Some(clusterId), parsed.clusterId)
- assertEquals(0, parsed.version)
- }
-
- @Test
- def testRawPropertiesWithInvalidBrokerId(): Unit = {
- val properties = new Properties()
- properties.put(RawMetaProperties.BrokerIdKey, "oof")
- val parsed = new RawMetaProperties(properties)
- assertThrows(classOf[RuntimeException], () => parsed.brokerId)
- }
-
- @Test
- def testCreateMetadataProperties(): Unit = {
- confirmValidForMetaProperties(clusterIdBase64)
- }
-
- @Test
- def testMetaPropertiesWithMissingVersion(): Unit = {
- val properties = new RawMetaProperties()
- properties.clusterId = clusterIdBase64
- properties.nodeId = 1
- assertThrows(classOf[RuntimeException], () => MetaProperties.parse(properties))
- }
-
- @Test
- def testMetaPropertiesAllowsHexEncodedUUIDs(): Unit = {
- val clusterId = "7bc79ca1-9746-42a3-a35a-efb3cde44492"
- confirmValidForMetaProperties(clusterId)
- }
-
- @Test
- def testMetaPropertiesWithNonUuidClusterId(): Unit = {
- val clusterId = "not a valid uuid"
- confirmValidForMetaProperties(clusterId)
- }
-
- private def confirmValidForMetaProperties(clusterId: String) = {
- val meta = MetaProperties(
- clusterId = clusterId,
- nodeId = 5
- )
- val properties = new RawMetaProperties(meta.toProperties)
- val meta2 = MetaProperties.parse(properties)
- assertEquals(meta, meta2)
- }
-
- @Test
- def testMetaPropertiesWithMissingBrokerId(): Unit = {
- val properties = new RawMetaProperties()
- properties.version = 1
- properties.clusterId = clusterIdBase64
- assertThrows(classOf[RuntimeException], () => MetaProperties.parse(properties))
- }
-
- @Test
- def testMetaPropertiesWithMissingControllerId(): Unit = {
- val properties = new RawMetaProperties()
- properties.version = 1
- properties.clusterId = clusterIdBase64
- assertThrows(classOf[RuntimeException], () => MetaProperties.parse(properties))
- }
-
- @Test
- def testMetaPropertiesWithVersionZero(): Unit = {
- val properties = new RawMetaProperties()
- properties.version = 0
- properties.clusterId = clusterIdBase64
- properties.brokerId = 5
- val metaProps = MetaProperties.parse(properties)
- assertEquals(clusterIdBase64, metaProps.clusterId)
- assertEquals(5, metaProps.nodeId)
- }
-
- @Test
- def testValidMetaPropertiesWithMultipleVersionsInLogDirs(): Unit = {
- // Let's create two directories with meta.properties one in v0 and v1.
- val props1 = new RawMetaProperties()
- props1.version = 0
- props1.clusterId = clusterIdBase64
- props1.brokerId = 5
- val props2 = new RawMetaProperties()
- props2.version = 1
- props2.clusterId = clusterIdBase64
- props2.nodeId = 5
- for (ignoreMissing <- Seq(true, false)) {
- val (metaProps, offlineDirs) = getMetadataWithMultipleMetaPropLogDirs(Seq(props1, props2), ignoreMissing, kraftMode = true)
- assertEquals(MetaProperties.parse(props2), MetaProperties.parse(metaProps))
- assertEquals(Seq.empty, offlineDirs)
- }
- }
-
- @Test
- def testInvalidMetaPropertiesWithMultipleVersionsInLogDirs(): Unit = {
- // Let's create two directories with meta.properties one in v0 and v1.
- val props1 = new RawMetaProperties()
- props1.version = 0
- props1.brokerId = 5
- val props2 = new RawMetaProperties()
- props2.version = 1
- props2.clusterId = clusterIdBase64
- props2.nodeId = 5
- for (ignoreMissing <- Seq(true, false)) {
- assertThrows(classOf[RuntimeException],
- () => getMetadataWithMultipleMetaPropLogDirs(Seq(props1, props2), ignoreMissing, kraftMode = true))
- }
- }
-
- private def getMetadataWithMultipleMetaPropLogDirs(metaProperties: Seq[RawMetaProperties],
- ignoreMissing: Boolean,
- kraftMode: Boolean): (RawMetaProperties, collection.Seq[String]) = {
- val logDirs = mutable.Buffer[File]()
- try {
- for (mp <- metaProperties) {
- val logDir = TestUtils.tempDirectory()
- logDirs += logDir
- val propFile = new File(logDir.getAbsolutePath, KafkaServer.brokerMetaPropsFile)
- val fs = new FileOutputStream(propFile)
- try {
- mp.props.store(fs, "")
- fs.flush()
- fs.getFD.sync()
- } finally {
- Utils.closeQuietly(fs, propFile.getName)
- }
- }
- BrokerMetadataCheckpoint.getBrokerMetadataAndOfflineDirs(logDirs.map(_.getAbsolutePath), ignoreMissing, kraftMode)
- } finally {
- logDirs.foreach(logDir => CoreUtils.swallow(Utils.delete(logDir), this))
- }
- }
-
- @Test
- def testGetBrokerMetadataAndOfflineDirsWithNonexistentDirectories(): Unit = {
- // Use a regular file as an invalid log dir to trigger an IO error
- val invalidDir = TestUtils.tempFile("blah")
- try {
- // The `ignoreMissing` and `kraftMode` flag has no effect if there is an IO error
- testEmptyGetBrokerMetadataAndOfflineDirs(invalidDir,
- expectedOfflineDirs = Seq(invalidDir), ignoreMissing = true, kraftMode = true)
- testEmptyGetBrokerMetadataAndOfflineDirs(invalidDir,
- expectedOfflineDirs = Seq(invalidDir), ignoreMissing = false, kraftMode = true)
- } finally {
- Utils.delete(invalidDir)
- }
- }
-
- @Test
- def testGetBrokerMetadataAndOfflineDirsIgnoreMissing(): Unit = {
- val tempDir = TestUtils.tempDirectory()
- try {
- testEmptyGetBrokerMetadataAndOfflineDirs(tempDir,
- expectedOfflineDirs = Seq(), ignoreMissing = true, kraftMode = true)
- testEmptyGetBrokerMetadataAndOfflineDirs(tempDir,
- expectedOfflineDirs = Seq(), ignoreMissing = true, kraftMode = false)
-
- assertThrows(classOf[RuntimeException],
- () => BrokerMetadataCheckpoint.getBrokerMetadataAndOfflineDirs(
- Seq(tempDir.getAbsolutePath), ignoreMissing = false, kraftMode = false))
- assertThrows(classOf[RuntimeException],
- () => BrokerMetadataCheckpoint.getBrokerMetadataAndOfflineDirs(
- Seq(tempDir.getAbsolutePath), ignoreMissing = false, kraftMode = true))
- } finally {
- Utils.delete(tempDir)
- }
- }
-
- private def testEmptyGetBrokerMetadataAndOfflineDirs(
- logDir: File,
- expectedOfflineDirs: Seq[File],
- ignoreMissing: Boolean,
- kraftMode: Boolean
- ): Unit = {
- val (metaProperties, offlineDirs) = BrokerMetadataCheckpoint.getBrokerMetadataAndOfflineDirs(
- Seq(logDir.getAbsolutePath), ignoreMissing, kraftMode = false)
- assertEquals(expectedOfflineDirs.map(_.getAbsolutePath), offlineDirs)
- assertEquals(new Properties(), metaProperties.props)
- }
-
-}
diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
index b0dd81e1c5828..e14e2eb8e079d 100755
--- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
@@ -20,13 +20,14 @@ package kafka.log
import com.yammer.metrics.core.{Gauge, MetricName}
import kafka.server.checkpoints.OffsetCheckpointFile
import kafka.server.metadata.{ConfigRepository, MockConfigRepository}
-import kafka.server.{BrokerMetadataCheckpoint, BrokerTopicStats, KafkaServer, RawMetaProperties}
+import kafka.server.BrokerTopicStats
import kafka.utils._
import org.apache.directory.api.util.FileUtils
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.OffsetOutOfRangeException
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
+import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.mockito.ArgumentMatchers.any
@@ -36,7 +37,7 @@ import org.mockito.Mockito.{doAnswer, doNothing, mock, never, spy, times, verify
import java.io._
import java.nio.file.Files
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, Future}
-import java.util.{Collections, OptionalLong, Properties}
+import java.util.{Collections, Optional, OptionalLong, Properties}
import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.util.MockTime
import org.apache.kafka.storage.internals.log.{FetchDataInfo, FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig, RemoteIndexCache}
@@ -1043,27 +1044,34 @@ class LogManagerTest {
@Test
def testLoadDirectoryIds(): Unit = {
- def writeMetaProperties(dir: File, id: Option[String] = None): Unit = {
- val rawProps = new RawMetaProperties()
- rawProps.nodeId = 1
- rawProps.clusterId = "IVT1Seu3QjacxS7oBTKhDQ"
- id.foreach(v => rawProps.directoryId = v)
- new BrokerMetadataCheckpoint(new File(dir, KafkaServer.brokerMetaPropsFile)).write(rawProps.props)
+ def writeMetaProperties(
+ dir: File,
+ directoryId: Optional[Uuid] = Optional.empty()
+ ): Unit = {
+ val metaProps = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V0).
+ setClusterId("IVT1Seu3QjacxS7oBTKhDQ").
+ setNodeId(1).
+ setDirectoryId(directoryId).
+ build()
+ PropertiesUtils.writePropertiesFile(metaProps.toProperties,
+ new File(dir, MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath, false)
}
val dirs: Seq[File] = Seq.fill(5)(TestUtils.tempDir())
writeMetaProperties(dirs(0))
- writeMetaProperties(dirs(1), Some("ZwkGXjB0TvSF6mjVh6gO7Q"))
+ writeMetaProperties(dirs(1), Optional.of(Uuid.fromString("ZwkGXjB0TvSF6mjVh6gO7Q")))
// no meta.properties on dirs(2)
- writeMetaProperties(dirs(3), Some("kQfNPJ2FTHq_6Qlyyv6Jqg"))
+ writeMetaProperties(dirs(3), Optional.of(Uuid.fromString("kQfNPJ2FTHq_6Qlyyv6Jqg")))
writeMetaProperties(dirs(4))
logManager = createLogManager(dirs)
- assertTrue(logManager.directoryId(dirs(0).getAbsolutePath).isDefined)
+ assertFalse(logManager.directoryId(dirs(0).getAbsolutePath).isDefined)
+ assertTrue(logManager.directoryId(dirs(1).getAbsolutePath).isDefined)
assertEquals(Some(Uuid.fromString("ZwkGXjB0TvSF6mjVh6gO7Q")), logManager.directoryId(dirs(1).getAbsolutePath))
assertEquals(None, logManager.directoryId(dirs(2).getAbsolutePath))
assertEquals(Some(Uuid.fromString("kQfNPJ2FTHq_6Qlyyv6Jqg")), logManager.directoryId(dirs(3).getAbsolutePath))
- assertTrue(logManager.directoryId(dirs(4).getAbsolutePath).isDefined)
- assertEquals(4, logManager.directoryIds.size)
+ assertTrue(logManager.directoryId(dirs(3).getAbsolutePath).isDefined)
+ assertEquals(2, logManager.directoryIds.size)
}
}
diff --git a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala
index b2b17852b1c87..e9bdb1b733650 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaRaftServerTest.scala
@@ -19,11 +19,11 @@ package kafka.server
import java.io.File
import java.nio.file.Files
import java.util.{Optional, Properties}
-import kafka.common.{InconsistentBrokerMetadataException, InconsistentNodeIdException}
import kafka.log.UnifiedLog
import org.apache.kafka.common.{KafkaException, Uuid}
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata}
+import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.test.TestUtils
import org.junit.jupiter.api.Assertions._
@@ -36,7 +36,11 @@ class KafkaRaftServerTest {
def testSuccessfulLoadMetaProperties(): Unit = {
val clusterId = clusterIdBase64
val nodeId = 0
- val metaProperties = MetaProperties(clusterId, nodeId)
+ val metaProperties = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId).
+ setNodeId(nodeId).
+ build()
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker,controller")
@@ -45,11 +49,13 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
- val (loadedMetaProperties, _, offlineDirs) =
- invokeLoadMetaProperties(metaProperties, configProperties)
+ val metaPropertiesEnsemble =
+ invokeLoadMetaProperties(metaProperties, configProperties)._1
- assertEquals(metaProperties, loadedMetaProperties)
- assertEquals(Seq.empty, offlineDirs)
+ val loadedMetaProperties = metaPropertiesEnsemble.logDirProps().values().iterator().next()
+ assertEquals(metaProperties, new MetaProperties.Builder(loadedMetaProperties).
+ setDirectoryId(Optional.empty[Uuid]).build())
+ assertTrue(loadedMetaProperties.directoryId().isPresent)
}
@Test
@@ -58,7 +64,11 @@ class KafkaRaftServerTest {
val metaNodeId = 1
val configNodeId = 0
- val metaProperties = MetaProperties(clusterId, metaNodeId)
+ val metaProperties = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId).
+ setNodeId(metaNodeId).
+ build()
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "controller")
@@ -66,7 +76,7 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.QuorumVotersProp, s"$configNodeId@localhost:9092")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT")
- assertThrows(classOf[InconsistentNodeIdException], () =>
+ assertThrows(classOf[RuntimeException], () =>
invokeLoadMetaProperties(metaProperties, configProperties))
}
@@ -74,14 +84,14 @@ class KafkaRaftServerTest {
metaProperties: MetaProperties,
configProperties: Properties,
metadataVersion: Option[MetadataVersion] = Some(MetadataVersion.latest())
- ): (MetaProperties, BootstrapMetadata, collection.Seq[String]) = {
+ ): (MetaPropertiesEnsemble, BootstrapMetadata) = {
val tempLogDir = TestUtils.tempDirectory()
try {
writeMetaProperties(tempLogDir, metaProperties)
metadataVersion.foreach(mv => writeBootstrapMetadata(tempLogDir, mv))
configProperties.put(KafkaConfig.LogDirProp, tempLogDir.getAbsolutePath)
val config = KafkaConfig.fromProps(configProperties)
- KafkaRaftServer.initializeLogDirs(config)
+ KafkaRaftServer.initializeLogDirs(config, MetaPropertiesEnsemble.LOG, "")
} finally {
Utils.delete(tempLogDir)
}
@@ -91,9 +101,8 @@ class KafkaRaftServerTest {
logDir: File,
metaProperties: MetaProperties
): Unit = {
- val metaPropertiesFile = new File(logDir.getAbsolutePath, KafkaServer.brokerMetaPropsFile)
- val checkpoint = new BrokerMetadataCheckpoint(metaPropertiesFile)
- checkpoint.write(metaProperties.toProperties)
+ val metaPropertiesFile = new File(logDir.getAbsolutePath, MetaPropertiesEnsemble.META_PROPERTIES_NAME)
+ PropertiesUtils.writePropertiesFile(metaProperties.toProperties(), metaPropertiesFile.getAbsolutePath, false)
}
private def writeBootstrapMetadata(logDir: File, metadataVersion: MetadataVersion): Unit = {
@@ -110,7 +119,11 @@ class KafkaRaftServerTest {
// The other is online, but has no `meta.properties`.
val logDir1 = TestUtils.tempDirectory()
val logDir2 = TestUtils.tempDirectory()
- writeMetaProperties(logDir1, MetaProperties(clusterId, nodeId))
+ writeMetaProperties(logDir1, new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId).
+ setNodeId(nodeId).
+ build())
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker")
@@ -120,7 +133,8 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties)
- assertThrows(classOf[KafkaException], () => KafkaRaftServer.initializeLogDirs(config))
+ assertThrows(classOf[RuntimeException],
+ () => KafkaRaftServer.initializeLogDirs(config, MetaPropertiesEnsemble.LOG, ""))
}
@Test
@@ -130,7 +144,11 @@ class KafkaRaftServerTest {
// One log dir is online and has properly formatted `meta.properties`
val validDir = TestUtils.tempDirectory()
- writeMetaProperties(validDir, MetaProperties(clusterId, nodeId))
+ writeMetaProperties(validDir, new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId).
+ setNodeId(nodeId).
+ build())
// Use a regular file as an invalid log dir to trigger an IO error
val invalidDir = TestUtils.tempFile("blah")
@@ -143,7 +161,8 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties)
- assertThrows(classOf[KafkaException], () => KafkaRaftServer.initializeLogDirs(config))
+ assertThrows(classOf[RuntimeException],
+ () => KafkaRaftServer.initializeLogDirs(config, MetaPropertiesEnsemble.LOG, ""))
}
@Test
@@ -153,7 +172,12 @@ class KafkaRaftServerTest {
// One log dir is online and has properly formatted `meta.properties`
val validDir = TestUtils.tempDirectory()
- writeMetaProperties(validDir, MetaProperties(clusterId, nodeId))
+ writeMetaProperties(validDir, new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId).
+ setNodeId(nodeId).
+ build())
+
writeBootstrapMetadata(validDir, MetadataVersion.latest())
// Use a regular file as an invalid log dir to trigger an IO error
@@ -167,9 +191,11 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties)
- val (loadedProperties, _, offlineDirs) = KafkaRaftServer.initializeLogDirs(config)
- assertEquals(nodeId, loadedProperties.nodeId)
- assertEquals(Seq(invalidDir.getAbsolutePath), offlineDirs)
+ val (metaPropertiesEnsemble, _) =
+ KafkaRaftServer.initializeLogDirs(config, MetaPropertiesEnsemble.LOG, "")
+ assertEquals(nodeId, metaPropertiesEnsemble.nodeId().getAsInt())
+ assertEquals(invalidDir.getAbsolutePath,
+ String.join(", ", metaPropertiesEnsemble.errorLogDirs()))
}
@Test
@@ -182,7 +208,11 @@ class KafkaRaftServerTest {
val dataDir = TestUtils.tempDirectory()
Seq(metadataDir, dataDir).foreach { dir =>
- writeMetaProperties(dir, MetaProperties(clusterId, nodeId))
+ writeMetaProperties(dir, new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId).
+ setNodeId(nodeId).
+ build())
}
// Create the metadata dir in the data directory
@@ -197,7 +227,8 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties)
- assertThrows(classOf[KafkaException], () => KafkaRaftServer.initializeLogDirs(config))
+ assertThrows(classOf[KafkaException],
+ () => KafkaRaftServer.initializeLogDirs(config, MetaPropertiesEnsemble.LOG, ""))
}
@Test
@@ -208,7 +239,11 @@ class KafkaRaftServerTest {
// Create a random clusterId in each log dir
Seq(logDir1, logDir2).foreach { dir =>
- writeMetaProperties(dir, MetaProperties(clusterId = Uuid.randomUuid().toString, nodeId))
+ writeMetaProperties(dir, new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(Uuid.randomUuid().toString).
+ setNodeId(nodeId).
+ build())
}
val configProperties = new Properties
@@ -219,15 +254,20 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
val config = KafkaConfig.fromProps(configProperties)
- assertThrows(classOf[InconsistentBrokerMetadataException],
- () => KafkaRaftServer.initializeLogDirs(config))
+ assertThrows(classOf[RuntimeException],
+ () => KafkaRaftServer.initializeLogDirs(config, MetaPropertiesEnsemble.LOG, ""))
}
@Test
def testKRaftUpdateWithIBP(): Unit = {
val clusterId = clusterIdBase64
val nodeId = 0
- val metaProperties = MetaProperties(clusterId, nodeId)
+ val metaProperties = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId).
+ setNodeId(nodeId).
+ setDirectoryId(Uuid.fromString("4jm0e-YRYeB6CCKBvwoS8w")).
+ build()
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker,controller")
@@ -237,11 +277,12 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
configProperties.put(KafkaConfig.InterBrokerProtocolVersionProp, "3.3-IV1")
- val (loadedMetaProperties, bootstrapMetadata, offlineDirs) =
+ val (metaPropertiesEnsemble, bootstrapMetadata) =
invokeLoadMetaProperties(metaProperties, configProperties, None)
- assertEquals(metaProperties, loadedMetaProperties)
- assertEquals(Seq.empty, offlineDirs)
+ assertEquals(metaProperties, metaPropertiesEnsemble.logDirProps().values().iterator().next())
+ assertTrue(metaPropertiesEnsemble.errorLogDirs().isEmpty())
+ assertTrue(metaPropertiesEnsemble.emptyLogDirs().isEmpty())
assertEquals(bootstrapMetadata.metadataVersion(), MetadataVersion.IBP_3_3_IV1)
}
@@ -249,7 +290,12 @@ class KafkaRaftServerTest {
def testKRaftUpdateWithoutIBP(): Unit = {
val clusterId = clusterIdBase64
val nodeId = 0
- val metaProperties = MetaProperties(clusterId, nodeId)
+ val metaProperties = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId).
+ setNodeId(nodeId).
+ setDirectoryId(Uuid.fromString("4jm0e-YRYeB6CCKBvwoS8w")).
+ build()
val logDir = TestUtils.tempDirectory()
writeMetaProperties(logDir, metaProperties)
@@ -262,11 +308,12 @@ class KafkaRaftServerTest {
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
configProperties.put(KafkaConfig.LogDirProp, logDir.getAbsolutePath)
- val (loadedMetaProperties, bootstrapMetadata, offlineDirs) =
+ val (metaPropertiesEnsemble, bootstrapMetadata) =
invokeLoadMetaProperties(metaProperties, configProperties, None)
- assertEquals(metaProperties, loadedMetaProperties)
- assertEquals(Seq.empty, offlineDirs)
+ assertEquals(metaProperties, metaPropertiesEnsemble.logDirProps().values().iterator().next())
+ assertTrue(metaPropertiesEnsemble.errorLogDirs().isEmpty())
+ assertTrue(metaPropertiesEnsemble.emptyLogDirs().isEmpty())
assertEquals(bootstrapMetadata.metadataVersion(), MetadataVersion.latest())
}
}
diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala
index bdeab92e0e7d7..e424f77c3e79f 100755
--- a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala
@@ -16,16 +16,15 @@
*/
package kafka.server
-import java.util.Properties
-
+import java.util.{OptionalInt, Properties}
import scala.collection.Seq
-
-import kafka.server.QuorumTestHarness
import kafka.utils.TestUtils
+import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, PropertiesUtils}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
import org.junit.jupiter.api.Assertions._
-import java.io.File
+import java.io.File
import org.apache.zookeeper.KeeperException.NodeExistsException
class ServerGenerateBrokerIdTest extends QuorumTestHarness {
@@ -155,36 +154,36 @@ class ServerGenerateBrokerIdTest extends QuorumTestHarness {
assertThrows(classOf[NodeExistsException], () => serverB.startup())
servers = Seq(serverA)
- // verify no broker metadata was written
- serverB.config.logDirs.foreach { logDir =>
- val brokerMetaFile = new File(logDir + File.separator + KafkaServer.brokerMetaPropsFile)
- assertFalse(brokerMetaFile.exists())
- }
-
// adjust the broker config and start again
propsB.setProperty(KafkaConfig.BrokerIdProp, "2")
- val newConfigB = KafkaConfig.fromProps(propsB)
- val newServerB = TestUtils.createServer(newConfigB, threadNamePrefix = Option(this.getClass.getName))
- servers = Seq(serverA, newServerB)
+ val serverB2 = new KafkaServer(KafkaConfig.fromProps(propsB),
+ threadNamePrefix = Option(this.getClass.getName))
+ val startupException = assertThrows(classOf[RuntimeException], () => serverB2.startup())
+ assertTrue(startupException.getMessage.startsWith("Stored node id 1 doesn't match previous node id 2"),
+ "Unexpected exception message " + startupException.getMessage())
+ serverB2.config.logDirs.foreach(logDir => Utils.delete(new File(logDir)))
+ propsB.setProperty(KafkaConfig.BrokerIdProp, "3")
+ val serverB3 = new KafkaServer(KafkaConfig.fromProps(propsB),
+ threadNamePrefix = Option(this.getClass.getName))
+ serverB3.startup()
+ servers = Seq(serverA, serverB3)
serverA.shutdown()
- newServerB.shutdown()
+ serverB3.shutdown()
// verify correct broker metadata was written
assertTrue(verifyBrokerMetadata(serverA.config.logDirs, 1))
- assertTrue(verifyBrokerMetadata(newServerB.config.logDirs, 2))
+ assertTrue(verifyBrokerMetadata(serverB3.config.logDirs, 3))
TestUtils.assertNoNonDaemonThreads(this.getClass.getName)
}
def verifyBrokerMetadata(logDirs: Seq[String], brokerId: Int): Boolean = {
for (logDir <- logDirs) {
- val brokerMetadataOpt = new BrokerMetadataCheckpoint(
- new File(logDir + File.separator + KafkaServer.brokerMetaPropsFile)).read()
- brokerMetadataOpt match {
- case Some(properties) =>
- val brokerMetadata = new RawMetaProperties(properties)
- if (brokerMetadata.brokerId.exists(_ != brokerId)) return false
- case _ => return false
+ val properties = PropertiesUtils.readPropertiesFile(
+ new File(logDir, MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath)
+ val metaProps = new MetaProperties.Builder(properties).build()
+ if (!metaProps.nodeId().equals(OptionalInt.of(brokerId))) {
+ return false
}
}
true
diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateClusterIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateClusterIdTest.scala
index f537037d732f2..87d53dace58d3 100755
--- a/core/src/test/scala/unit/kafka/server/ServerGenerateClusterIdTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ServerGenerateClusterIdTest.scala
@@ -17,21 +17,18 @@
package kafka.server
import java.io.File
-
-
import scala.collection.Seq
import scala.concurrent._
import scala.concurrent.duration._
import ExecutionContext.Implicits._
-
-import kafka.common.{InconsistentBrokerMetadataException, InconsistentClusterIdException}
import kafka.utils.TestUtils
-import kafka.server.QuorumTestHarness
-
+import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
import org.apache.kafka.test.TestUtils.isValidClusterId
+import java.util.Optional
+
class ServerGenerateClusterIdTest extends QuorumTestHarness {
var config1: KafkaConfig = _
@@ -173,7 +170,7 @@ class ServerGenerateClusterIdTest extends QuorumTestHarness {
val server = new KafkaServer(config1, threadNamePrefix = Option(this.getClass.getName))
// Startup fails
- assertThrows(classOf[InconsistentClusterIdException], () => server.startup())
+ assertThrows(classOf[RuntimeException], () => server.startup())
server.shutdown()
@@ -197,7 +194,7 @@ class ServerGenerateClusterIdTest extends QuorumTestHarness {
val server = new KafkaServer(config, threadNamePrefix = Option(this.getClass.getName))
// Startup fails
- assertThrows(classOf[InconsistentBrokerMetadataException], () => server.startup())
+ assertThrows(classOf[RuntimeException], () => server.startup())
server.shutdown()
@@ -211,23 +208,24 @@ class ServerGenerateClusterIdTest extends QuorumTestHarness {
}
def forgeBrokerMetadata(logDir: String, brokerId: Int, clusterId: String): Unit = {
- val checkpoint = new BrokerMetadataCheckpoint(
- new File(logDir + File.separator + KafkaServer.brokerMetaPropsFile))
- checkpoint.write(ZkMetaProperties(clusterId, brokerId).toProperties)
+ val metaProps = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V0).
+ setNodeId(brokerId).
+ setClusterId(clusterId).
+ build()
+ PropertiesUtils.writePropertiesFile(metaProps.toProperties(),
+ new File(logDir, MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath, false)
}
def verifyBrokerMetadata(logDirs: Seq[String], clusterId: String): Boolean = {
for (logDir <- logDirs) {
- val brokerMetadataOpt = new BrokerMetadataCheckpoint(
- new File(logDir + File.separator + KafkaServer.brokerMetaPropsFile)).read()
- brokerMetadataOpt match {
- case Some(properties) =>
- val brokerMetadata = new RawMetaProperties(properties)
- if (brokerMetadata.clusterId.exists(_ != clusterId)) return false
- case _ => return false
+ val properties = PropertiesUtils.readPropertiesFile(
+ new File(logDir, MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath)
+ val metaProps = new MetaProperties.Builder(properties).build()
+ if (!metaProps.clusterId().equals(Optional.of(clusterId))) {
+ return false
}
}
true
}
-
}
diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala
index 8cb5d156e98d4..5a4cca14c58e4 100644
--- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala
+++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala
@@ -22,14 +22,15 @@ import java.nio.charset.StandardCharsets
import java.nio.file.{Files, Paths}
import java.util
import java.util.Properties
-import org.apache.kafka.common.{KafkaException, Uuid}
-import kafka.server.{BrokerMetadataCheckpoint, KafkaConfig, KafkaServer, MetaProperties}
+import org.apache.kafka.common.{DirectoryId, KafkaException}
+import kafka.server.KafkaConfig
import kafka.utils.Exit
import kafka.utils.TestUtils
import org.apache.commons.io.output.NullOutputStream
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.common.metadata.UserScramCredentialRecord
+import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue}
import org.junit.jupiter.api.{Test, Timeout}
@@ -115,9 +116,10 @@ Found problem:
val stream = new ByteArrayOutputStream()
val tempDir = TestUtils.tempDir()
try {
- Files.write(tempDir.toPath.resolve(KafkaServer.brokerMetaPropsFile),
+ Files.write(tempDir.toPath.resolve(MetaPropertiesEnsemble.META_PROPERTIES_NAME),
String.join("\n", util.Arrays.asList(
"version=1",
+ "node.id=1",
"cluster.id=XcZZOzUqS4yHOjhMQB6JLQ")).
getBytes(StandardCharsets.UTF_8))
assertEquals(1, StorageTool.
@@ -125,7 +127,7 @@ Found problem:
assertEquals(s"""Found log directory:
${tempDir.toString}
-Found metadata: {cluster.id=XcZZOzUqS4yHOjhMQB6JLQ, version=1}
+Found metadata: {cluster.id=XcZZOzUqS4yHOjhMQB6JLQ, node.id=1, version=1}
Found problem:
The kafka configuration file appears to be for a legacy cluster, but the directories are formatted for a cluster in KRaft mode.
@@ -139,7 +141,7 @@ Found problem:
val stream = new ByteArrayOutputStream()
val tempDir = TestUtils.tempDir()
try {
- Files.write(tempDir.toPath.resolve(KafkaServer.brokerMetaPropsFile),
+ Files.write(tempDir.toPath.resolve(MetaPropertiesEnsemble.META_PROPERTIES_NAME),
String.join("\n", util.Arrays.asList(
"version=0",
"broker.id=1",
@@ -163,8 +165,11 @@ Found problem:
def testFormatEmptyDirectory(): Unit = {
val tempDir = TestUtils.tempDir()
try {
- val metaProperties = MetaProperties(
- clusterId = "XcZZOzUqS4yHOjhMQB6JLQ", nodeId = 2)
+ val metaProperties = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("XcZZOzUqS4yHOjhMQB6JLQ").
+ setNodeId(2).
+ build()
val stream = new ByteArrayOutputStream()
val bootstrapMetadata = StorageTool.buildBootstrapMetadata(MetadataVersion.latest(), None, "test format command")
assertEquals(0, StorageTool.
@@ -367,18 +372,22 @@ Found problem:
def testDirUuidGeneration(): Unit = {
val tempDir = TestUtils.tempDir()
try {
- val metaProperties = MetaProperties(
- clusterId = "XcZZOzUqS4yHOjhMQB6JLQ", nodeId = 2)
- val bootstrapMetadata = StorageTool.buildBootstrapMetadata(MetadataVersion.latest(), None, "test format command")
+ val metaProperties = new MetaProperties.Builder().
+ setClusterId("XcZZOzUqS4yHOjhMQB6JLQ").
+ setNodeId(2).
+ build()
+ val bootstrapMetadata = StorageTool.
+ buildBootstrapMetadata(MetadataVersion.latest(), None, "test format command")
assertEquals(0, StorageTool.
formatCommand(new PrintStream(NullOutputStream.NULL_OUTPUT_STREAM), Seq(tempDir.toString), metaProperties, bootstrapMetadata, MetadataVersion.latest(), ignoreFormatted = false))
- val metaPropertiesFile = Paths.get(tempDir.toURI).resolve(KafkaServer.brokerMetaPropsFile).toFile
+ val metaPropertiesFile = Paths.get(tempDir.toURI).resolve(MetaPropertiesEnsemble.META_PROPERTIES_NAME).toFile
assertTrue(metaPropertiesFile.exists())
- val properties = new BrokerMetadataCheckpoint(metaPropertiesFile).read().get
- assertTrue(properties.containsKey("directory.id"))
- val directoryId = Uuid.fromString(properties.getProperty("directory.id"))
- assertFalse(Uuid.RESERVED.contains(directoryId))
+ val metaProps = new MetaProperties.Builder(
+ PropertiesUtils.readPropertiesFile(metaPropertiesFile.getAbsolutePath())).
+ build()
+ assertTrue(metaProps.directoryId().isPresent())
+ assertFalse(DirectoryId.reserved(metaProps.directoryId().get()))
} finally Utils.delete(tempDir)
}
}
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectory.java b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectory.java
index 863a583808ab0..a7cd382dc1677 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectory.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapDirectory.java
@@ -23,6 +23,7 @@
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
+import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
@@ -107,7 +108,7 @@ BootstrapMetadata readFromBinaryFile(String binaryPath) throws Exception {
"the binary bootstrap metadata file: " + binaryPath);
}
- public void writeBinaryFile(BootstrapMetadata bootstrapMetadata) throws Exception {
+ public void writeBinaryFile(BootstrapMetadata bootstrapMetadata) throws IOException {
if (!Files.isDirectory(Paths.get(directoryPath))) {
throw new RuntimeException("No such directory as " + directoryPath);
}
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaProperties.java b/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaProperties.java
new file mode 100644
index 0000000000000..7d1909a17b74b
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaProperties.java
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.properties;
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.Properties;
+
+/**
+ * An immutable class which contains the per-log-directory information stored in an individual
+ * meta.properties file.
+ */
+public final class MetaProperties {
+ /**
+ * The property that sets the version number.
+ */
+ static final String VERSION_PROP = "version";
+
+ /**
+ * The property that specifies the cluster id.
+ */
+ static final String CLUSTER_ID_PROP = "cluster.id";
+
+ /**
+ * The property that specifies the broker id. Only in V0.
+ */
+ static final String BROKER_ID_PROP = "broker.id";
+
+ /**
+ * The property that specifies the node id. Replaces broker.id in V1.
+ */
+ static final String NODE_ID_PROP = "node.id";
+
+ /**
+ * The property that specifies the directory id.
+ */
+ static final String DIRECTORY_ID_PROP = "directory.id";
+
+ /**
+ * The version of the MetaProperties file.
+ */
+ private final MetaPropertiesVersion version;
+
+ /**
+ * The cluster ID, which may be Optional.empty in V0.
+ */
+ private final Optional clusterId;
+
+ /**
+ * The node ID, which may be OptionalInt.empty in V0.
+ */
+ private final OptionalInt nodeId;
+
+ /**
+ * The directory ID, or Optional.empty if none is specified.
+ */
+ private final Optional directoryId;
+
+ /**
+ * Creates a new MetaProperties object.
+ */
+ public static class Builder {
+ private MetaPropertiesVersion version = MetaPropertiesVersion.V0;
+ private Optional clusterId = Optional.empty();
+ private OptionalInt nodeId = OptionalInt.empty();
+ private Optional directoryId = Optional.empty();
+
+ public Builder() {
+ }
+
+ public Builder(Optional metaProps) {
+ if (metaProps.isPresent()) {
+ this.version = metaProps.get().version();
+ this.clusterId = metaProps.get().clusterId();
+ this.nodeId = metaProps.get().nodeId();
+ this.directoryId = metaProps.get().directoryId();
+ }
+ }
+
+ public Builder(MetaProperties metaProps) {
+ this(Optional.of(metaProps));
+ }
+
+ public Builder(Properties props) {
+ this.version = MetaPropertiesVersion.fromNumberString(
+ props.getProperty(VERSION_PROP,
+ MetaPropertiesVersion.V0.numberString()));
+ if (version.hasBrokerId()) {
+ if (props.containsKey(BROKER_ID_PROP)) {
+ this.nodeId = OptionalInt.of(PropertiesUtils.loadRequiredIntProp(props, BROKER_ID_PROP));
+ }
+ } else {
+ this.nodeId = OptionalInt.of(PropertiesUtils.loadRequiredIntProp(props, NODE_ID_PROP));
+ }
+ this.clusterId = Optional.ofNullable(props.getProperty(CLUSTER_ID_PROP));
+ if (props.containsKey(DIRECTORY_ID_PROP)) {
+ try {
+ this.directoryId = Optional.of(Uuid.fromString(props.getProperty(DIRECTORY_ID_PROP)));
+ } catch (Exception e) {
+ throw new RuntimeException("Unable to read " + DIRECTORY_ID_PROP + " as a Uuid: " +
+ e.getMessage(), e);
+ }
+ } else {
+ this.directoryId = Optional.empty();
+ }
+ }
+
+ public MetaPropertiesVersion version() {
+ return version;
+ }
+
+ public Builder setVersion(MetaPropertiesVersion version) {
+ this.version = version;
+ return this;
+ }
+
+ public Optional clusterId() {
+ return clusterId;
+ }
+
+ public Builder setClusterId(String clusterId) {
+ return setClusterId(Optional.of(clusterId));
+ }
+
+ public Builder setClusterId(Optional clusterId) {
+ this.clusterId = clusterId;
+ return this;
+ }
+
+ public OptionalInt nodeId() {
+ return nodeId;
+ }
+
+ public Builder setNodeId(OptionalInt nodeId) {
+ this.nodeId = nodeId;
+ return this;
+ }
+
+ public Builder setNodeId(int nodeId) {
+ return setNodeId(OptionalInt.of(nodeId));
+ }
+
+ public Optional directoryId() {
+ return directoryId;
+ }
+
+ public Builder setDirectoryId(Optional directoryId) {
+ this.directoryId = directoryId;
+ return this;
+ }
+
+ public Builder setDirectoryId(Uuid directoryId) {
+ return setDirectoryId(Optional.of(directoryId));
+ }
+
+ public MetaProperties build() {
+ if (!version.equals(MetaPropertiesVersion.V0)) {
+ if (!clusterId.isPresent()) {
+ throw new RuntimeException("cluster.id was not found.");
+ }
+ if (!nodeId.isPresent()) {
+ throw new RuntimeException("node.id was not found.");
+ }
+ }
+ return new MetaProperties(version,
+ clusterId,
+ nodeId,
+ directoryId);
+ }
+ }
+
+ private MetaProperties(
+ MetaPropertiesVersion version,
+ Optional clusterId,
+ OptionalInt nodeId,
+ Optional directoryId
+ ) {
+ this.version = version;
+ this.clusterId = clusterId;
+ this.nodeId = nodeId;
+ this.directoryId = directoryId;
+ }
+
+ public MetaPropertiesVersion version() {
+ return version;
+ }
+
+ public Optional clusterId() {
+ return clusterId;
+ }
+
+ public OptionalInt nodeId() {
+ return nodeId;
+ }
+
+ public Optional directoryId() {
+ return directoryId;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(version,
+ clusterId,
+ nodeId,
+ directoryId);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == null || !(o.getClass().equals(MetaProperties.class))) return false;
+ MetaProperties other = (MetaProperties) o;
+ return version.equals(other.version) &&
+ clusterId.equals(other.clusterId) &&
+ nodeId.equals(other.nodeId) &&
+ directoryId.equals(other.directoryId);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder bld = new StringBuilder();
+ bld.append("MetaProperties");
+ bld.append("(version=").append(version.number());
+ if (clusterId.isPresent()) {
+ bld.append(", clusterId=").append(clusterId.get());
+ }
+ if (nodeId.isPresent()) {
+ bld.append(", nodeId=").append(nodeId.getAsInt());
+ }
+ if (directoryId.isPresent()) {
+ bld.append(", directoryId=").append(directoryId.get());
+ }
+ bld.append(")");
+ return bld.toString();
+ }
+
+ public Properties toProperties() {
+ Properties props = new Properties();
+ props.setProperty(VERSION_PROP, version.numberString());
+ if (clusterId.isPresent()) {
+ props.setProperty(CLUSTER_ID_PROP, clusterId.get());
+ }
+ if (version.hasBrokerId()) {
+ if (nodeId.isPresent()) {
+ props.setProperty(BROKER_ID_PROP, "" + nodeId.getAsInt());
+ }
+ } else {
+ props.setProperty(NODE_ID_PROP, "" + nodeId.getAsInt());
+ }
+ if (directoryId.isPresent()) {
+ props.setProperty(DIRECTORY_ID_PROP, directoryId.get().toString());
+ }
+ return props;
+ }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsemble.java b/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsemble.java
new file mode 100644
index 0000000000000..01726013e4895
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsemble.java
@@ -0,0 +1,619 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.properties;
+
+import org.apache.kafka.common.DirectoryId;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.function.BiConsumer;
+import java.util.stream.Collectors;
+
+/**
+ * A collection of meta.properties information for Kafka log directories.
+ *
+ * Directories are categorized into empty, error, and normal. Each directory must appear in only
+ * one category, corresponding to emptyLogDirs, errorLogDirs, and logDirProps.
+ *
+ * This class is immutable. Modified copies can be made with the Copier class.
+ */
+public final class MetaPropertiesEnsemble {
+ /**
+ * The log4j object for this class.
+ */
+ public static final Logger LOG = LoggerFactory.getLogger(MetaPropertiesEnsemble.class);
+
+ /**
+ * A completely empty MetaPropertiesEnsemble object.
+ */
+ public static final MetaPropertiesEnsemble EMPTY = new MetaPropertiesEnsemble(Collections.emptySet(),
+ Collections.emptySet(),
+ Collections.emptyMap(),
+ Optional.empty());
+
+ /**
+ * The name of the meta.properties file within each log directory.
+ */
+ public static final String META_PROPERTIES_NAME = "meta.properties";
+
+ /**
+ * The set of log dirs that were empty.
+ */
+ private final Set emptyLogDirs;
+
+ /**
+ * The set of log dirs that had errors.
+ */
+ private final Set errorLogDirs;
+
+ /**
+ * A map from log directories to the meta.properties information inside each one.
+ */
+ private final Map logDirProps;
+
+ /**
+ * The metadata log directory, or the empty string if there is none.
+ */
+ private final Optional metadataLogDir;
+
+ /**
+ * Utility class for loading a MetaPropertiesEnsemble from the disk.
+ */
+ public static class Loader {
+ private final TreeSet logDirs = new TreeSet<>();
+ private Optional metadataLogDir = Optional.empty();
+
+ public Loader addLogDirs(Collection logDirs) {
+ for (String logDir : logDirs) {
+ this.logDirs.add(logDir);
+ }
+ return this;
+ }
+
+ public Loader addLogDir(String logDir) {
+ this.logDirs.add(logDir);
+ return this;
+ }
+
+ public Loader addMetadataLogDir(String metadataLogDir) {
+ if (this.metadataLogDir.isPresent()) {
+ throw new RuntimeException("Cannot specify more than one metadata log directory. " +
+ "Already specified " + this.metadataLogDir.get());
+ }
+ this.metadataLogDir = Optional.of(metadataLogDir);
+ logDirs.add(metadataLogDir);
+ return this;
+ }
+
+ public MetaPropertiesEnsemble load() throws IOException {
+ if (logDirs.isEmpty()) {
+ throw new RuntimeException("You must specify at least one log directory.");
+ }
+ Set emptyLogDirs = new HashSet<>();
+ Set errorLogDirs = new HashSet<>();
+ Map logDirProps = new HashMap<>();
+ for (String logDir : logDirs) {
+ String metaPropsFile = new File(logDir, META_PROPERTIES_NAME).getAbsolutePath();
+ try {
+ Properties props = PropertiesUtils.readPropertiesFile(metaPropsFile);
+ MetaProperties meta = new MetaProperties.Builder(props).build();
+ logDirProps.put(logDir, meta);
+ } catch (NoSuchFileException | FileNotFoundException e) {
+ emptyLogDirs.add(logDir);
+ } catch (Exception e) {
+ LOG.error("Error while reading meta.properties file {}", metaPropsFile, e);
+ errorLogDirs.add(logDir);
+ }
+ }
+ return new MetaPropertiesEnsemble(emptyLogDirs, errorLogDirs, logDirProps, metadataLogDir);
+ }
+ }
+
+ public interface WriteErrorHandler {
+ void handle(
+ String logDir,
+ IOException e
+ ) throws IOException;
+ }
+
+ public interface PreWriteHandler {
+ void handle(
+ String logDir,
+ boolean isNew,
+ MetaProperties metaProperties
+ ) throws IOException;
+ }
+
+ /**
+ * Utility class for copying a MetaPropertiesEnsemble object, possibly with changes.
+ */
+ public static class Copier {
+ public static final BiConsumer LOGGING_ERROR_HANDLER = new BiConsumer() {
+ @Override
+ public void accept(String logDir, IOException e) {
+ MetaPropertiesEnsemble.LOG.error("Error while writing meta.properties to {}", logDir, e);
+ }
+ };
+
+ private final MetaPropertiesEnsemble prev;
+ private Random random = new Random();
+ private Set emptyLogDirs;
+ private Set errorLogDirs;
+ private Map logDirProps;
+ private Optional metaLogDir;
+
+ private PreWriteHandler preWriteHandler = (logDir, isNew, metaProperties) -> {
+ LOG.info("Writing out {} {}{}meta.properties file containing {}",
+ isNew ? "new" : "changed",
+ logDir,
+ File.separator,
+ metaProperties);
+ };
+
+ private WriteErrorHandler writeErrorHandler = (logDir, e) -> {
+ LOG.error("Error while writing meta.properties to {}", logDir, e);
+ throw e;
+ };
+
+ public Copier(MetaPropertiesEnsemble prev) {
+ this.prev = prev;
+ this.emptyLogDirs = new HashSet<>(prev.emptyLogDirs());
+ this.errorLogDirs = new HashSet<>(prev.errorLogDirs());
+ this.logDirProps = new HashMap<>(prev.logDirProps());
+ this.metaLogDir = prev.metadataLogDir;
+ }
+
+ /**
+ * Set the Random object to use for generating IDs.
+ *
+ * @param random The Random object to use for generating IDs.
+ * @return This copier
+ */
+ public Copier setRandom(Random random) {
+ this.random = random;
+ return this;
+ }
+
+ /**
+ * Access the mutable empty log directories set.
+ *
+ * @return The mutable empty log directories set.
+ */
+ public Set emptyLogDirs() {
+ return emptyLogDirs;
+ }
+
+ /**
+ * Access the mutable error log directories set.
+ *
+ * @return The mutable error log directories set.
+ */
+ public Set errorLogDirs() {
+ return errorLogDirs;
+ }
+
+ /**
+ * Access the mutable logDirProps map.
+ *
+ * @return The mutable error log directories map.
+ */
+ public Map logDirProps() {
+ return logDirProps;
+ }
+
+ /**
+ * Set the properties of a given log directory.
+ *
+ * @param logDir The log directory path.
+ * @param metaProps The properties to set.
+ *
+ * @return This copier.
+ */
+ public Copier setLogDirProps(String logDir, MetaProperties metaProps) {
+ emptyLogDirs.remove(logDir);
+ errorLogDirs.remove(logDir);
+ logDirProps.put(logDir, metaProps);
+ return this;
+ }
+
+ public Optional metaLogDir() {
+ return metaLogDir;
+ }
+
+ /**
+ * Set the the current metadata log directory.
+ *
+ * @param metaLogDir The metadata log directory, or Optional.empty if there is none.
+ *
+ * @return This copier.
+ */
+ public Copier setMetaLogDir(Optional metaLogDir) {
+ this.metaLogDir = metaLogDir;
+ return this;
+ }
+
+ /**
+ * Generate a random directory ID that is safe and not used by any other directory.
+ *
+ * @return A new random directory ID.
+ */
+ public Uuid generateValidDirectoryId() {
+ while (true) {
+ Uuid uuid = new Uuid(random.nextLong(), random.nextLong());
+ if (!(uuid.toString().startsWith("-") || DirectoryId.reserved(uuid))) {
+ boolean duplicate = false;
+ for (MetaProperties metaProps : logDirProps.values()) {
+ if (metaProps.directoryId().equals(Optional.of(uuid))) {
+ duplicate = true;
+ break;
+ }
+ }
+ if (!duplicate) {
+ return uuid;
+ }
+ }
+ }
+ }
+
+ /**
+ * Set the pre-write handler.
+ *
+ * @param preWriteHandler A handler that will be called before we try to write to a
+ * directory.
+ */
+ public Copier setPreWriteHandler(PreWriteHandler preWriteHandler) {
+ this.preWriteHandler = preWriteHandler;
+ return this;
+ }
+
+ /**
+ * Set the error handler.
+ *
+ * @param writeErrorHandler A handler that will be called any time we hit an exception
+ * writing out a meta.properties file.
+ */
+ public Copier setWriteErrorHandler(WriteErrorHandler writeErrorHandler) {
+ this.writeErrorHandler = writeErrorHandler;
+ return this;
+ }
+
+ /**
+ * Verify that we have set up the Copier correctly.
+ *
+ * @throws RuntimeException if a directory appears in more than one category.
+ */
+ public void verify() {
+ for (String logDir : emptyLogDirs) {
+ if (errorLogDirs.contains(logDir)) {
+ throw new RuntimeException("Error: log directory " + logDir +
+ " is in both emptyLogDirs and errorLogDirs.");
+ }
+ if (logDirProps.containsKey(logDir)) {
+ throw new RuntimeException("Error: log directory " + logDir +
+ " is in both emptyLogDirs and logDirProps.");
+ }
+ }
+ for (String logDir : errorLogDirs) {
+ if (logDirProps.containsKey(logDir)) {
+ throw new RuntimeException("Error: log directory " + logDir +
+ " is in both errorLogDirs and logDirProps.");
+ }
+ }
+ metaLogDir().ifPresent(m -> {
+ if (!(emptyLogDirs.contains(m) ||
+ logDirProps.containsKey(m) ||
+ errorLogDirs.contains(m))) {
+ throw new RuntimeException("Error: metaLogDir " + m + " does not appear " +
+ "in emptyLogDirs, errorLogDirs, or logDirProps.");
+ }
+ });
+ }
+
+ /**
+ * Write any changed log directories out to disk.
+ */
+ public void writeLogDirChanges() throws IOException {
+ Map newOrChanged = new HashMap<>();
+ HashSet newSet = new HashSet<>();
+ for (Entry entry : prev.logDirProps().entrySet()) {
+ MetaProperties prevMetaProps = entry.getValue();
+ MetaProperties metaProps = logDirProps.get(entry.getKey());
+ if (!prevMetaProps.equals(metaProps)) {
+ newOrChanged.put(entry.getKey(), metaProps);
+ }
+ }
+ for (Entry entry : logDirProps.entrySet()) {
+ if (!prev.logDirProps().containsKey(entry.getKey())) {
+ newOrChanged.put(entry.getKey(), entry.getValue());
+ newSet.add(entry.getKey());
+ }
+ }
+ for (Entry entry : newOrChanged.entrySet()) {
+ String logDir = entry.getKey();
+ MetaProperties metaProps = entry.getValue();
+ String metaPropsPath = new File(logDir, META_PROPERTIES_NAME).getAbsolutePath();
+ try {
+ preWriteHandler.handle(logDir, newSet.contains(logDir), metaProps);
+ PropertiesUtils.writePropertiesFile(metaProps.toProperties(),
+ metaPropsPath, true);
+ } catch (IOException e) {
+ errorLogDirs.add(logDir);
+ logDirProps.remove(logDir);
+ writeErrorHandler.handle(logDir, e);
+ }
+ }
+ }
+
+ /**
+ * Create a new immutable MetaPropertiesEnsemble file.
+ *
+ * @return A new MetaPropertiesEnsemble file containing the changes we made in this Copier.
+ */
+ public MetaPropertiesEnsemble copy() {
+ return new MetaPropertiesEnsemble(emptyLogDirs,
+ errorLogDirs,
+ logDirProps,
+ metaLogDir);
+ }
+ }
+
+ MetaPropertiesEnsemble(
+ Set emptyLogDirs,
+ Set errorLogDirs,
+ Map logDirProps,
+ Optional metadataLogDir
+ ) {
+ this.emptyLogDirs = Collections.unmodifiableSet(new TreeSet<>(emptyLogDirs));
+ this.errorLogDirs = Collections.unmodifiableSet(new TreeSet<>(errorLogDirs));
+ this.logDirProps = Collections.unmodifiableMap(new TreeMap<>(logDirProps));
+ this.metadataLogDir = metadataLogDir;
+ }
+
+ /**
+ * @return The empty log directories which did not contain a meta.properties file.
+ */
+ public Set emptyLogDirs() {
+ return emptyLogDirs;
+ }
+
+ /**
+ * @return The log directories that had I/O errors.
+ */
+ public Set errorLogDirs() {
+ return errorLogDirs;
+ }
+
+ /**
+ * @return A map from log directory paths to properties objects.
+ */
+ public Map logDirProps() {
+ return logDirProps;
+ }
+
+ /**
+ * @return An iterator that returns (logDir, metaProps) for all non-failed directories.
+ */
+ public Iterator>> nonFailedDirectoryProps() {
+ return new Iterator>>() {
+ private Iterator emptyLogDirsIterator = emptyLogDirs.iterator();
+ private Iterator> logDirsIterator =
+ logDirProps.entrySet().iterator();
+
+ @Override
+ public boolean hasNext() {
+ return emptyLogDirsIterator.hasNext() || logDirsIterator.hasNext();
+ }
+
+ @Override
+ public Entry> next() {
+ if (emptyLogDirsIterator.hasNext()) {
+ return new SimpleImmutableEntry<>(emptyLogDirsIterator.next(), Optional.empty());
+ }
+ Entry entry = logDirsIterator.next();
+ return new SimpleImmutableEntry<>(entry.getKey(), Optional.of(entry.getValue()));
+ }
+ };
+ }
+
+ /**
+ * @return The metadata log directory, or Optional.empty if there is none.
+ */
+ public Optional metadataLogDir() {
+ return metadataLogDir;
+ }
+
+ public enum VerificationFlag {
+ REQUIRE_V0,
+ REQUIRE_METADATA_LOG_DIR,
+ REQUIRE_AT_LEAST_ONE_VALID
+ }
+
+ /**
+ * Verify that the metadata properties ensemble is valid.
+ *
+ * We verify that v1 meta.properties files always have cluster.id set. v0 files may or may not
+ * have it set. If it is set, the cluster ID must be the same in all directories.
+ *
+ * We verify that v1 meta.properties files always have node.id set. v0 files may or may not have
+ * it set. If it is set in v0, it will be called broker.id rather than node.id. Node ID must be
+ * the same in call directories.
+ *
+ * directory.id may or may not be set, in both v0 and v1. If it is set, it must not be the same
+ * in multiple directories, and it must be safe.
+ *
+ * @param expectedClusterId The cluster ID to expect, or the empty string if we don't know yet.
+ * @param expectedNodeId The node ID to expect, or -1 if we don't know yet.
+ * @param verificationFlags The flags to use.
+ */
+ public void verify(
+ Optional expectedClusterId,
+ OptionalInt expectedNodeId,
+ EnumSet verificationFlags
+ ) {
+ Map seenUuids = new HashMap<>();
+ if (verificationFlags.contains(VerificationFlag.REQUIRE_AT_LEAST_ONE_VALID)) {
+ if (logDirProps.isEmpty()) {
+ throw new RuntimeException("No readable meta.properties files found.");
+ }
+ }
+ for (Entry entry : logDirProps.entrySet()) {
+ String logDir = entry.getKey();
+ String path = new File(logDir, META_PROPERTIES_NAME).toString();
+ MetaProperties metaProps = entry.getValue();
+ if (verificationFlags.contains(VerificationFlag.REQUIRE_V0)) {
+ if (!metaProps.version().equals(MetaPropertiesVersion.V0)) {
+ throw new RuntimeException("Found unexpected version in " + path + ". " +
+ "ZK-based brokers that are not migrating only support version 0 " +
+ "(which is implicit when the `version` field is missing).");
+ }
+ }
+ if (!metaProps.clusterId().isPresent()) {
+ if (metaProps.version().alwaysHasClusterId()) {
+ throw new RuntimeException("cluster.id was not specified in the v1 file: " +
+ path);
+ }
+ } else if (!expectedClusterId.isPresent()) {
+ expectedClusterId = metaProps.clusterId();
+ } else if (!metaProps.clusterId().get().equals(expectedClusterId.get())) {
+ throw new RuntimeException("Invalid cluster.id in: " + path + ". Expected " +
+ expectedClusterId.get() + ", but read " + metaProps.clusterId().get());
+ }
+ if (!metaProps.nodeId().isPresent()) {
+ if (metaProps.version().alwaysHasNodeId()) {
+ throw new RuntimeException("node.id was not specified in " + path);
+ }
+ } else if (!expectedNodeId.isPresent()) {
+ expectedNodeId = metaProps.nodeId();
+ } else if (metaProps.nodeId().getAsInt() != expectedNodeId.getAsInt()) {
+ throw new RuntimeException("Stored node id " + metaProps.nodeId().getAsInt() +
+ " doesn't match previous node id " + expectedNodeId.getAsInt() + " in " + path +
+ ". If you moved your data, make sure your configured node id matches. If you " +
+ "intend to create a new node, you should remove all data in your data " +
+ "directories.");
+ }
+ if (metaProps.directoryId().isPresent()) {
+ if (DirectoryId.reserved(metaProps.directoryId().get())) {
+ throw new RuntimeException("Invalid resrved directory ID " +
+ metaProps.directoryId().get() + " found in " + logDir);
+ }
+ String prevLogDir = seenUuids.put(metaProps.directoryId().get(), logDir);
+ if (prevLogDir != null) {
+ throw new RuntimeException("Duplicate directory ID " + metaProps.directoryId() +
+ " found. It was the ID of " + prevLogDir + ", " + "but also of " +
+ logDir);
+ }
+ }
+ }
+ if (verificationFlags.contains(VerificationFlag.REQUIRE_METADATA_LOG_DIR)) {
+ if (!metadataLogDir.isPresent()) {
+ throw new RuntimeException("No metadata log directory was specified.");
+ }
+ }
+ if (metadataLogDir.isPresent()) {
+ if (errorLogDirs.contains(metadataLogDir.get())) {
+ throw new RuntimeException("Encountered I/O error in metadata log directory " +
+ metadataLogDir.get() + ". Cannot continue.");
+ }
+ }
+ }
+
+ /**
+ * Find the node ID of this meta.properties ensemble.
+ *
+ * @return The node ID, or OptionalInt.empty if none could be found.
+ */
+ public OptionalInt nodeId() {
+ for (MetaProperties metaProps : logDirProps.values()) {
+ if (metaProps.nodeId().isPresent()) {
+ return metaProps.nodeId();
+ }
+ }
+ return OptionalInt.empty();
+ }
+
+ /**
+ * Find the cluster ID of this meta.properties ensemble.
+ *
+ * @return The cluster ID, or Optional.empty if none could be found.
+ */
+ public Optional clusterId() {
+ for (MetaProperties metaProps : logDirProps.values()) {
+ if (metaProps.clusterId().isPresent()) {
+ return metaProps.clusterId();
+ }
+ }
+ return Optional.empty();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == null || (!(o.getClass().equals(MetaPropertiesEnsemble.class)))) {
+ return false;
+ }
+ MetaPropertiesEnsemble other = (MetaPropertiesEnsemble) o;
+ return emptyLogDirs.equals(other.emptyLogDirs) &&
+ errorLogDirs.equals(other.errorLogDirs) &&
+ logDirProps.equals(other.logDirProps) &&
+ metadataLogDir.equals(other.metadataLogDir);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(emptyLogDirs,
+ errorLogDirs,
+ logDirProps,
+ metadataLogDir);
+ }
+
+ @Override
+ public String toString() {
+ TreeMap outputMap = new TreeMap<>();
+ emptyLogDirs.forEach(e -> outputMap.put(e, "EMPTY"));
+ errorLogDirs.forEach(e -> outputMap.put(e, "ERROR"));
+ logDirProps.entrySet().forEach(
+ e -> outputMap.put(e.getKey(), e.getValue().toString()));
+ return "MetaPropertiesEnsemble" +
+ "(metadataLogDir=" + metadataLogDir +
+ ", dirs={" + outputMap.entrySet().stream().
+ map(e -> e.getKey() + ": " + e.getValue()).
+ collect(Collectors.joining(", ")) +
+ "})";
+ }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesVersion.java b/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesVersion.java
new file mode 100644
index 0000000000000..abe5d3d1d1194
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesVersion.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.properties;
+
+/**
+ * The version of a meta.properties file.
+ */
+public enum MetaPropertiesVersion {
+ /**
+ * The original version of meta.properties. This is the version that gets used if there is no
+ * version field. No fields are guaranteed in v0.
+ */
+ V0(0),
+
+ /**
+ * The KRaft version of meta.properties. Not all KRaft clusters have this version; some which
+ * were migrated from ZK may have v0.
+ */
+ V1(1);
+
+ private final int number;
+
+ public static MetaPropertiesVersion fromNumberString(String numberString) {
+ int number;
+ try {
+ number = Integer.parseInt(numberString.trim());
+ } catch (NumberFormatException e) {
+ throw new RuntimeException("Invalid meta.properties version string '" +
+ numberString + "'");
+ }
+ return fromNumber(number);
+ }
+
+ public static MetaPropertiesVersion fromNumber(int number) {
+ switch (number) {
+ case 0: return V0;
+ case 1: return V1;
+ default: throw new RuntimeException("Unknown meta.properties version number " + number);
+ }
+ }
+
+ MetaPropertiesVersion(int number) {
+ this.number = number;
+ }
+
+ public int number() {
+ return number;
+ }
+
+ public String numberString() {
+ return Integer.toString(number);
+ }
+
+ public boolean hasBrokerId() {
+ return this == V0;
+ }
+
+ public boolean alwaysHasNodeId() {
+ return this != V0;
+ }
+
+ public boolean alwaysHasClusterId() {
+ return this != V0;
+ }
+}
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/properties/PropertiesUtils.java b/metadata/src/main/java/org/apache/kafka/metadata/properties/PropertiesUtils.java
new file mode 100644
index 0000000000000..6e36c51751b92
--- /dev/null
+++ b/metadata/src/main/java/org/apache/kafka/metadata/properties/PropertiesUtils.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.properties;
+
+import org.apache.kafka.common.utils.Utils;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Properties;
+
+public final class PropertiesUtils {
+ /**
+ * Writes a Java Properties object to a file.
+ *
+ * @param props The Properties object.
+ * @param path The file to write to.
+ * @throws IOException
+ */
+ public static void writePropertiesFile(
+ Properties props,
+ String path,
+ boolean fsync
+ ) throws IOException {
+ File tempFile = new File(path + ".tmp");
+ try (
+ FileOutputStream fos = new FileOutputStream(tempFile, false);
+ OutputStreamWriter osw = new OutputStreamWriter(fos, StandardCharsets.UTF_8);
+ PrintWriter pw = new PrintWriter(osw)
+ ) {
+ props.store(pw, "");
+ fos.flush();
+ if (fsync) {
+ fos.getFD().sync();
+ }
+ }
+ File targetFile = new File(path);
+ try {
+ Utils.atomicMoveWithFallback(tempFile.toPath(), targetFile.toPath(), fsync);
+ } catch (Throwable e) {
+ Utils.delete(tempFile);
+ throw e;
+ }
+ }
+
+ /**
+ * Reads a Java Properties object from a file.
+ *
+ * @param path The file to read from.
+ *
+ * @throws java.nio.file.NoSuchFileException If the file is not found.
+ * @throws IOException If there is another exception while reading.
+ */
+ public static Properties readPropertiesFile(String path) throws IOException {
+ Properties props = new Properties();
+ try (InputStream propStream = Files.newInputStream(Paths.get(path))) {
+ props.load(propStream);
+ }
+ return props;
+ }
+
+ /**
+ * Pull an integer from a Properties object.
+ *
+ * @param props The Properties object.
+ * @param keyName The key to look for.
+ *
+ * @return The integer.
+ */
+ static int loadRequiredIntProp(
+ Properties props,
+ String keyName
+ ) {
+ String value = props.getProperty(keyName);
+ if (value == null) {
+ throw new RuntimeException("Failed to find " + keyName);
+ }
+ try {
+ return Integer.parseInt(value);
+ } catch (NumberFormatException e) {
+ throw new RuntimeException("Unable to read " + keyName + " as a base-10 number.", e);
+ }
+ }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsembleTest.java b/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsembleTest.java
new file mode 100644
index 0000000000000..3c99b3b22287f
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsembleTest.java
@@ -0,0 +1,450 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.properties;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Test;
+
+import static org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.EMPTY;
+import static org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.META_PROPERTIES_NAME;
+import static org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag.REQUIRE_AT_LEAST_ONE_VALID;
+import static org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag.REQUIRE_METADATA_LOG_DIR;
+import static org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag.REQUIRE_V0;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+final public class MetaPropertiesEnsembleTest {
+ private static final MetaPropertiesEnsemble FOO =
+ new MetaPropertiesEnsemble(
+ new HashSet<>(Arrays.asList("/tmp/empty1", "/tmp/empty2")),
+ new HashSet<>(Arrays.asList("/tmp/error3")),
+ Arrays.asList(
+ new SimpleImmutableEntry<>("/tmp/dir4",
+ new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("fooClusterId").
+ setNodeId(2).
+ build()),
+ new SimpleImmutableEntry<>("/tmp/dir5",
+ new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("fooClusterId").
+ setNodeId(2).
+ build())).stream().collect(Collectors.
+ toMap(Entry::getKey, Entry::getValue)),
+ Optional.of("/tmp/dir4"));
+
+ private static String createLogDir(MetaProperties metaProps) throws IOException {
+ File logDir = TestUtils.tempDirectory();
+ PropertiesUtils.writePropertiesFile(metaProps.toProperties(),
+ new File(logDir, META_PROPERTIES_NAME).getAbsolutePath(), false);
+ return logDir.getAbsolutePath();
+ }
+
+ private static String createEmptyLogDir() throws IOException {
+ File logDir = TestUtils.tempDirectory();
+ return logDir.getAbsolutePath();
+ }
+
+ private static String createErrorLogDir() throws IOException {
+ File logDir = TestUtils.tempDirectory();
+ File metaPath = new File(logDir, META_PROPERTIES_NAME);
+ Files.write(metaPath.toPath(), new byte[] {(byte) 0});
+ metaPath.setReadable(false);
+ return logDir.getAbsolutePath();
+ }
+
+ @Test
+ public void testEmptyLogDirsForFoo() {
+ assertEquals(new HashSet<>(Arrays.asList("/tmp/empty1", "/tmp/empty2")),
+ FOO.emptyLogDirs());
+ }
+
+ @Test
+ public void testEmptyLogDirsForEmpty() {
+ assertEquals(new HashSet<>(), EMPTY.emptyLogDirs());
+ }
+
+ @Test
+ public void testErrorLogDirsForFoo() {
+ assertEquals(new HashSet<>(Arrays.asList("/tmp/error3")), FOO.errorLogDirs());
+ }
+
+ @Test
+ public void testErrorLogDirsForEmpty() {
+ assertEquals(new HashSet<>(), EMPTY.errorLogDirs());
+ }
+
+ @Test
+ public void testLogDirPropsForFoo() {
+ assertEquals(new HashSet<>(Arrays.asList("/tmp/dir4", "/tmp/dir5")),
+ FOO.logDirProps().keySet());
+ }
+
+ @Test
+ public void testLogDirPropsForEmpty() {
+ assertEquals(new HashSet<>(),
+ EMPTY.logDirProps().keySet());
+ }
+
+ @Test
+ public void testNonFailedDirectoryPropsForFoo() {
+ Map> results = new HashMap<>();
+ FOO.nonFailedDirectoryProps().forEachRemaining(entry -> {
+ results.put(entry.getKey(), entry.getValue());
+ });
+ assertEquals(Optional.empty(), results.get("/tmp/empty1"));
+ assertEquals(Optional.empty(), results.get("/tmp/empty2"));
+ assertNull(results.get("/tmp/error3"));
+ assertEquals(Optional.of(new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("fooClusterId").
+ setNodeId(2).
+ build()), results.get("/tmp/dir4"));
+ assertEquals(Optional.of(new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("fooClusterId").
+ setNodeId(2).
+ build()), results.get("/tmp/dir5"));
+ assertEquals(4, results.size());
+ }
+
+ @Test
+ public void testNonFailedDirectoryPropsForEmpty() {
+ assertFalse(EMPTY.nonFailedDirectoryProps().hasNext());
+ }
+
+ @Test
+ public void testMetadataLogDirForFoo() {
+ assertEquals(Optional.of("/tmp/dir4"), FOO.metadataLogDir());
+ }
+
+ @Test
+ public void testMetadataLogDirForEmpty() {
+ assertEquals(Optional.empty(), EMPTY.metadataLogDir());
+ }
+
+ @Test
+ public void testNodeIdForFoo() {
+ assertEquals(OptionalInt.of(2), FOO.nodeId());
+ }
+
+ @Test
+ public void testNodeIdForEmpty() {
+ assertEquals(OptionalInt.empty(), EMPTY.nodeId());
+ }
+
+ @Test
+ public void testClusterIdForFoo() {
+ assertEquals(Optional.of("fooClusterId"), FOO.clusterId());
+ }
+
+ @Test
+ public void testClusterIdForEmpty() {
+ assertEquals(Optional.empty(), EMPTY.clusterId());
+ }
+
+ @Test
+ public void testSuccessfulVerification() {
+ FOO.verify(Optional.empty(),
+ OptionalInt.empty(),
+ EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR));
+ }
+
+ @Test
+ public void testSuccessfulVerificationWithClusterId() {
+ FOO.verify(Optional.of("fooClusterId"),
+ OptionalInt.empty(),
+ EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR));
+ }
+
+ @Test
+ public void testSuccessfulVerificationWithClusterIdAndNodeId() {
+ FOO.verify(Optional.of("fooClusterId"),
+ OptionalInt.of(2),
+ EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR));
+ }
+
+ @Test
+ public void testVerificationFailureOnRequireV0() {
+ assertEquals("Found unexpected version in /tmp/dir4/meta.properties. ZK-based brokers " +
+ "that are not migrating only support version 0 (which is implicit when the " +
+ "`version` field is missing).",
+ assertThrows(RuntimeException.class, () ->
+ FOO.verify(Optional.empty(), OptionalInt.empty(), EnumSet.of(REQUIRE_V0))).
+ getMessage());
+ }
+
+ @Test
+ public void testVerificationFailureOnRequireAtLeastOneValid() {
+ assertEquals("No readable meta.properties files found.",
+ assertThrows(RuntimeException.class,
+ () -> EMPTY.verify(Optional.empty(),
+ OptionalInt.empty(),
+ EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID))).
+ getMessage());
+ }
+
+ @Test
+ public void testVerificationFailureOnLackOfMetadataLogDir() throws IOException {
+ MetaPropertiesEnsemble ensemble = new MetaPropertiesEnsemble(
+ Collections.singleton("/tmp/foo1"),
+ Collections.emptySet(),
+ Collections.emptyMap(),
+ Optional.empty());
+ assertEquals("No metadata log directory was specified.",
+ assertThrows(RuntimeException.class,
+ () -> ensemble.verify(Optional.empty(),
+ OptionalInt.empty(),
+ EnumSet.of(REQUIRE_METADATA_LOG_DIR))).
+ getMessage());
+ }
+
+ @Test
+ public void testVerificationFailureOnMetadataLogDirWithError() throws IOException {
+ MetaPropertiesEnsemble ensemble = new MetaPropertiesEnsemble(
+ Collections.emptySet(),
+ Collections.singleton("/tmp/foo1"),
+ Collections.emptyMap(),
+ Optional.of("/tmp/foo1"));
+ assertEquals("Encountered I/O error in metadata log directory /tmp/foo1. Cannot continue.",
+ assertThrows(RuntimeException.class,
+ () -> ensemble.verify(Optional.empty(),
+ OptionalInt.empty(),
+ EnumSet.of(REQUIRE_METADATA_LOG_DIR))).
+ getMessage());
+ }
+
+ @Test
+ public void testMetaPropertiesEnsembleLoad() throws IOException {
+ MetaPropertiesEnsemble.Loader loader = new MetaPropertiesEnsemble.Loader();
+ MetaProperties metaProps = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("AtgGav8yQjiaJ3rTXE7VCA").
+ setNodeId(1).
+ build();
+ loader.addMetadataLogDir(createLogDir(metaProps));
+ MetaPropertiesEnsemble metaPropertiesEnsemble = loader.load();
+ metaPropertiesEnsemble.verify(Optional.of("AtgGav8yQjiaJ3rTXE7VCA"),
+ OptionalInt.of(1),
+ EnumSet.of(REQUIRE_METADATA_LOG_DIR, REQUIRE_AT_LEAST_ONE_VALID));
+ assertEquals(1, metaPropertiesEnsemble.logDirProps().values().size());
+ assertEquals(metaProps, metaPropertiesEnsemble.logDirProps().values().iterator().next());
+ }
+
+ @Test
+ public void testMetaPropertiesEnsembleLoadEmpty() throws IOException {
+ MetaPropertiesEnsemble.Loader loader = new MetaPropertiesEnsemble.Loader();
+ loader.addMetadataLogDir(createEmptyLogDir());
+ MetaPropertiesEnsemble metaPropertiesEnsemble = loader.load();
+ metaPropertiesEnsemble.verify(Optional.of("AtgGav8yQjiaJ3rTXE7VCA"),
+ OptionalInt.of(1),
+ EnumSet.of(REQUIRE_METADATA_LOG_DIR));
+ assertEquals(1, metaPropertiesEnsemble.emptyLogDirs().size());
+ }
+
+ @Test
+ public void testMetaPropertiesEnsembleLoadError() throws IOException {
+ MetaPropertiesEnsemble.Loader loader = new MetaPropertiesEnsemble.Loader();
+ loader.addMetadataLogDir(createErrorLogDir());
+ loader.addLogDir(createLogDir(new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("AtgGav8yQjiaJ3rTXE7VCA").
+ setNodeId(1).
+ build()));
+ MetaPropertiesEnsemble metaPropertiesEnsemble = loader.load();
+ assertEquals(1, metaPropertiesEnsemble.errorLogDirs().size());
+ assertEquals(1, metaPropertiesEnsemble.logDirProps().size());
+ }
+
+ static private void verifyCopy(
+ MetaPropertiesEnsemble expected,
+ MetaPropertiesEnsemble.Copier copier
+ ) {
+ copier.verify();
+ MetaPropertiesEnsemble foo2 = copier.copy();
+ assertEquals(expected, foo2);
+ assertEquals(expected.hashCode(), foo2.hashCode());
+ assertEquals(expected.toString(), foo2.toString());
+ }
+
+ @Test
+ public void testCopierWithoutModifications() {
+ verifyCopy(FOO, new MetaPropertiesEnsemble.Copier(FOO));
+ }
+
+ @Test
+ public void testCopyFooItemByItem() {
+ MetaPropertiesEnsemble.Copier copier = new MetaPropertiesEnsemble.Copier(EMPTY);
+ copier.setMetaLogDir(FOO.metadataLogDir());
+ FOO.emptyLogDirs().forEach(e -> copier.emptyLogDirs().add(e));
+ FOO.logDirProps().entrySet().
+ forEach(e -> copier.logDirProps().put(e.getKey(), e.getValue()));
+ FOO.errorLogDirs().forEach(e -> copier.errorLogDirs().add(e));
+ verifyCopy(FOO, copier);
+ }
+
+ static class MetaPropertiesMockRandom extends Random {
+ private final AtomicInteger index = new AtomicInteger(0);
+
+ private List results = Arrays.asList(
+ 0L,
+ 0L,
+ 2336837413447398698L,
+ 1758400403264101670L,
+ 4341931186263415792L,
+ 6389410885970711333L,
+ 7265008559332826740L,
+ 3478747443029687715L
+ );
+
+ @Override
+ public long nextLong() {
+ int curIndex = index.getAndIncrement();
+ return results.get(curIndex % results.size());
+ }
+ }
+
+ @Test
+ public void testCopierGenerateValidDirectoryId() {
+ MetaPropertiesMockRandom random = new MetaPropertiesMockRandom();
+ MetaPropertiesEnsemble.Copier copier = new MetaPropertiesEnsemble.Copier(EMPTY);
+ copier.setRandom(random);
+ copier.logDirProps().put("/tmp/dir1",
+ new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("PpYMbsoRQV-589isZzNzEw").
+ setNodeId(0).
+ setDirectoryId(new Uuid(2336837413447398698L, 1758400403264101670L)).
+ build());
+ copier.logDirProps().put("/tmp/dir2",
+ new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("PpYMbsoRQV-589isZzNzEw").
+ setNodeId(0).
+ setDirectoryId(new Uuid(4341931186263415792L, 6389410885970711333L)).
+ build());
+ // Verify that we ignore the non-safe IDs, or the IDs that have already been used,
+ // when invoking generateValidDirectoryId.
+ assertEquals(new Uuid(7265008559332826740L, 3478747443029687715L),
+ copier.generateValidDirectoryId());
+ }
+
+ @Test
+ public void testCopierVerificationFailsOnEmptyAndErrorOverlap() {
+ MetaPropertiesEnsemble.Copier copier = new MetaPropertiesEnsemble.Copier(EMPTY);
+ copier.emptyLogDirs().add("/tmp/foo");
+ copier.errorLogDirs().add("/tmp/foo");
+ assertEquals("Error: log directory /tmp/foo is in both emptyLogDirs and errorLogDirs.",
+ assertThrows(RuntimeException.class, () -> copier.verify()).getMessage());
+ }
+
+ @Test
+ public void testCopierVerificationFailsOnEmptyAndLogDirsOverlap() {
+ MetaPropertiesEnsemble.Copier copier = new MetaPropertiesEnsemble.Copier(EMPTY);
+ copier.emptyLogDirs().add("/tmp/foo");
+ copier.logDirProps().put("/tmp/foo", new MetaProperties.Builder().build());
+ assertEquals("Error: log directory /tmp/foo is in both emptyLogDirs and logDirProps.",
+ assertThrows(RuntimeException.class, () -> copier.verify()).getMessage());
+ }
+
+ @Test
+ public void testCopierVerificationFailsOnErrorAndLogDirsOverlap() {
+ MetaPropertiesEnsemble.Copier copier = new MetaPropertiesEnsemble.Copier(EMPTY);
+ copier.errorLogDirs().add("/tmp/foo");
+ copier.logDirProps().put("/tmp/foo", new MetaProperties.Builder().build());
+ assertEquals("Error: log directory /tmp/foo is in both errorLogDirs and logDirProps.",
+ assertThrows(RuntimeException.class, () -> copier.verify()).getMessage());
+ }
+
+ private final static List SAMPLE_META_PROPS_LIST = Arrays.asList(
+ new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("AtgGav8yQjiaJ3rTXE7VCA").
+ setNodeId(1).
+ setDirectoryId(Uuid.fromString("s33AdXtkR8Gf_xRO-R_dpA")).
+ build(),
+ new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("AtgGav8yQjiaJ3rTXE7VCA").
+ setNodeId(1).
+ setDirectoryId(Uuid.fromString("oTM53yT_SbSfzlvkh_PfVA")).
+ build(),
+ new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("AtgGav8yQjiaJ3rTXE7VCA").
+ setNodeId(1).
+ setDirectoryId(Uuid.fromString("FcUhIv2mTzmLqGkVEabyag")).
+ build());
+
+ @Test
+ public void testCopierWriteLogDirChanges() throws Exception {
+ MetaPropertiesEnsemble.Loader loader = new MetaPropertiesEnsemble.Loader();
+ loader.addMetadataLogDir(createLogDir(SAMPLE_META_PROPS_LIST.get(0)));
+ MetaPropertiesEnsemble ensemble = loader.load();
+ MetaPropertiesEnsemble.Copier copier = new MetaPropertiesEnsemble.Copier(ensemble);
+ String newLogDir1 = createEmptyLogDir();
+ copier.logDirProps().put(newLogDir1, SAMPLE_META_PROPS_LIST.get(1));
+ String newLogDir2 = createEmptyLogDir();
+ copier.logDirProps().put(newLogDir2, SAMPLE_META_PROPS_LIST.get(2));
+ copier.writeLogDirChanges();
+ assertEquals(SAMPLE_META_PROPS_LIST.get(1).toProperties(), PropertiesUtils.readPropertiesFile(
+ new File(newLogDir1, META_PROPERTIES_NAME).getAbsolutePath()));
+ assertEquals(SAMPLE_META_PROPS_LIST.get(2).toProperties(), PropertiesUtils.readPropertiesFile(
+ new File(newLogDir2, META_PROPERTIES_NAME).getAbsolutePath()));
+ }
+
+ @Test
+ public void testCopierWriteChanged() throws Exception {
+ MetaPropertiesEnsemble.Loader loader = new MetaPropertiesEnsemble.Loader();
+ String dir0 = createLogDir(SAMPLE_META_PROPS_LIST.get(0));
+ loader.addMetadataLogDir(dir0);
+ loader.addLogDir(dir0);
+ String dir1 = createLogDir(SAMPLE_META_PROPS_LIST.get(1));
+ loader.addLogDir(dir1);
+ MetaPropertiesEnsemble ensemble = loader.load();
+ MetaPropertiesEnsemble.Copier copier = new MetaPropertiesEnsemble.Copier(ensemble);
+ copier.setLogDirProps(dir0, SAMPLE_META_PROPS_LIST.get(2));
+ copier.writeLogDirChanges();
+ assertEquals(SAMPLE_META_PROPS_LIST.get(2).toProperties(), PropertiesUtils.readPropertiesFile(
+ new File(dir0, META_PROPERTIES_NAME).getAbsolutePath()));
+ assertEquals(SAMPLE_META_PROPS_LIST.get(1).toProperties(), PropertiesUtils.readPropertiesFile(
+ new File(dir1, META_PROPERTIES_NAME).getAbsolutePath()));
+ }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesTest.java b/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesTest.java
new file mode 100644
index 0000000000000..c440965d4a5ab
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesTest.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.properties;
+
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.Properties;
+
+import org.apache.kafka.common.Uuid;
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+final public class MetaPropertiesTest {
+ @Test
+ public void testV0SerializationWithNothing() {
+ testV0Serialization(Optional.empty(),
+ OptionalInt.empty(),
+ Optional.empty(),
+ "MetaProperties(version=0)");
+ }
+
+ @Test
+ public void testV0SerializationWithJustClusterId() {
+ testV0Serialization(Optional.of("zd2vLVrZQlCLJj8-k7A10w"),
+ OptionalInt.empty(),
+ Optional.empty(),
+ "MetaProperties(version=0, clusterId=zd2vLVrZQlCLJj8-k7A10w)");
+ }
+
+ @Test
+ public void testV0SerializationWithJustNodeId() {
+ testV0Serialization(Optional.empty(),
+ OptionalInt.of(0),
+ Optional.empty(),
+ "MetaProperties(version=0, nodeId=0)");
+ }
+
+ @Test
+ public void testV0SerializationWithJustClusterIdAndNodeId() {
+ testV0Serialization(Optional.of("zd2vLVrZQlCLJj8-k7A10w"),
+ OptionalInt.of(0),
+ Optional.empty(),
+ "MetaProperties(version=0, clusterId=zd2vLVrZQlCLJj8-k7A10w, nodeId=0)");
+ }
+
+ @Test
+ public void testV0SerializationWithAll() {
+ testV0Serialization(Optional.of("zd2vLVrZQlCLJj8-k7A10w"),
+ OptionalInt.of(0),
+ Optional.of(Uuid.fromString("3Adc4FjfTeypRWROmQDNIQ")),
+ "MetaProperties(version=0, clusterId=zd2vLVrZQlCLJj8-k7A10w, nodeId=0, " +
+ "directoryId=3Adc4FjfTeypRWROmQDNIQ)");
+ }
+
+ private void testV0Serialization(
+ Optional clusterId,
+ OptionalInt nodeId,
+ Optional directoryId,
+ String expectedToStringOutput
+ ) {
+ MetaProperties metaProperties = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V0).
+ setClusterId(clusterId).
+ setNodeId(nodeId).
+ setDirectoryId(directoryId).
+ build();
+ assertEquals(MetaPropertiesVersion.V0, metaProperties.version());
+ assertEquals(clusterId, metaProperties.clusterId());
+ assertEquals(nodeId, metaProperties.nodeId());
+ assertEquals(directoryId, metaProperties.directoryId());
+ Properties props = new Properties();
+ props.setProperty("version", "0");
+ if (clusterId.isPresent()) {
+ props.setProperty("cluster.id", clusterId.get());
+ }
+ if (nodeId.isPresent()) {
+ props.setProperty("broker.id", "" + nodeId.getAsInt());
+ }
+ if (directoryId.isPresent()) {
+ props.setProperty("directory.id", directoryId.get().toString());
+ }
+ Properties props2 = metaProperties.toProperties();
+ assertEquals(props, props2);
+ MetaProperties metaProperties2 = new MetaProperties.Builder(props2).build();
+ System.out.println("metaProperties = " + metaProperties.toString());
+ System.out.println("metaProperties2 = " + metaProperties2.toString());
+ assertEquals(metaProperties, metaProperties2);
+ assertEquals(metaProperties.hashCode(), metaProperties2.hashCode());
+ assertEquals(metaProperties.toString(), metaProperties2.toString());
+ assertEquals(expectedToStringOutput, metaProperties.toString());
+ }
+
+ @Test
+ public void testV1SerializationWithoutDirectoryId() {
+ testV1Serialization("zd2vLVrZQlCLJj8-k7A10w",
+ 0,
+ Optional.empty(),
+ "MetaProperties(version=1, clusterId=zd2vLVrZQlCLJj8-k7A10w, nodeId=0)");
+ }
+
+ @Test
+ public void testV1SerializationWithDirectoryId() {
+ testV1Serialization("zd2vLVrZQlCLJj8-k7A10w",
+ 1,
+ Optional.of(Uuid.fromString("3Adc4FjfTeypRWROmQDNIQ")),
+ "MetaProperties(version=1, clusterId=zd2vLVrZQlCLJj8-k7A10w, nodeId=1, " +
+ "directoryId=3Adc4FjfTeypRWROmQDNIQ)");
+ }
+
+ @Test
+ public void testV1SerializationWithNonUuidClusterId() {
+ testV1Serialization("my@cluster@id",
+ 2,
+ Optional.empty(),
+ "MetaProperties(version=1, clusterId=my@cluster@id, nodeId=2)");
+ }
+
+ private void testV1Serialization(
+ String clusterId,
+ int nodeId,
+ Optional directoryId,
+ String expectedToStringOutput
+ ) {
+ MetaProperties metaProperties = new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId(clusterId).
+ setNodeId(nodeId).
+ setDirectoryId(directoryId).
+ build();
+ assertEquals(MetaPropertiesVersion.V1, metaProperties.version());
+ assertEquals(Optional.of(clusterId), metaProperties.clusterId());
+ assertEquals(OptionalInt.of(nodeId), metaProperties.nodeId());
+ assertEquals(directoryId, metaProperties.directoryId());
+ Properties props = new Properties();
+ props.setProperty("version", "1");
+ props.setProperty("cluster.id", clusterId);
+ props.setProperty("node.id", "" + nodeId);
+ if (directoryId.isPresent()) {
+ props.setProperty("directory.id", directoryId.get().toString());
+ }
+ Properties props2 = metaProperties.toProperties();
+ assertEquals(props, props2);
+ MetaProperties metaProperties2 = new MetaProperties.Builder(props2).build();
+ assertEquals(metaProperties, metaProperties2);
+ assertEquals(metaProperties.hashCode(), metaProperties2.hashCode());
+ assertEquals(metaProperties.toString(), metaProperties2.toString());
+ assertEquals(expectedToStringOutput, metaProperties.toString());
+ }
+
+ @Test
+ public void testClusterIdRequiredInV1() {
+ assertEquals("cluster.id was not found.", assertThrows(RuntimeException.class,
+ () -> new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setNodeId(1).
+ build()).getMessage());
+ }
+
+ @Test
+ public void testNodeIdRequiredInV1() {
+ assertEquals("node.id was not found.", assertThrows(RuntimeException.class,
+ () -> new MetaProperties.Builder().
+ setVersion(MetaPropertiesVersion.V1).
+ setClusterId("zd2vLVrZQlCLJj8-k7A10w").
+ build()).getMessage());
+ }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesVersionTest.java b/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesVersionTest.java
new file mode 100644
index 0000000000000..6fb017213a5da
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesVersionTest.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.properties;
+
+import org.junit.jupiter.api.Test;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+final public class MetaPropertiesVersionTest {
+ @Test
+ public void testV0ToNumber() {
+ assertEquals(0, MetaPropertiesVersion.V0.number());
+ }
+
+ @Test
+ public void testV0ToNumberString() {
+ assertEquals("0", MetaPropertiesVersion.V0.numberString());
+ }
+
+ @Test
+ public void testV0FromNumber() {
+ assertEquals(MetaPropertiesVersion.V0, MetaPropertiesVersion.fromNumber(0));
+ }
+
+ @Test
+ public void testV0FromNumberString() {
+ assertEquals(MetaPropertiesVersion.V0, MetaPropertiesVersion.fromNumberString("0"));
+ }
+
+ @Test
+ public void testV1ToNumber() {
+ assertEquals(1, MetaPropertiesVersion.V1.number());
+ }
+
+ @Test
+ public void testV1ToNumberString() {
+ assertEquals("1", MetaPropertiesVersion.V1.numberString());
+ }
+
+ @Test
+ public void testV1FromNumber() {
+ assertEquals(MetaPropertiesVersion.V1, MetaPropertiesVersion.fromNumber(1));
+ }
+
+ @Test
+ public void testV1FromNumberString() {
+ assertEquals(MetaPropertiesVersion.V1, MetaPropertiesVersion.fromNumberString("1"));
+ }
+
+ @Test
+ public void testFromInvalidNumber() {
+ assertEquals("Unknown meta.properties version number 2",
+ assertThrows(RuntimeException.class,
+ () -> MetaPropertiesVersion.fromNumber(2)).getMessage());
+ }
+
+ @Test
+ public void testFromInvalidString() {
+ assertEquals("Invalid meta.properties version string 'orange'",
+ assertThrows(RuntimeException.class,
+ () -> MetaPropertiesVersion.fromNumberString("orange")).getMessage());
+ }
+
+ @Test
+ public void testHasBrokerId() {
+ assertTrue(MetaPropertiesVersion.V0.hasBrokerId());
+ assertFalse(MetaPropertiesVersion.V1.hasBrokerId());
+ }
+
+ @Test
+ public void testAlwaysHasNodeId() {
+ assertFalse(MetaPropertiesVersion.V0.alwaysHasNodeId());
+ assertTrue(MetaPropertiesVersion.V1.alwaysHasNodeId());
+ }
+
+ @Test
+ public void testAlwaysHasClusterId() {
+ assertFalse(MetaPropertiesVersion.V0.alwaysHasClusterId());
+ assertTrue(MetaPropertiesVersion.V1.alwaysHasClusterId());
+ }
+}
diff --git a/metadata/src/test/java/org/apache/kafka/metadata/properties/PropertiesUtilsTest.java b/metadata/src/test/java/org/apache/kafka/metadata/properties/PropertiesUtilsTest.java
new file mode 100644
index 0000000000000..926767dcd58d4
--- /dev/null
+++ b/metadata/src/test/java/org/apache/kafka/metadata/properties/PropertiesUtilsTest.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.metadata.properties;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Properties;
+
+import org.apache.kafka.test.TestUtils;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+final public class PropertiesUtilsTest {
+ @Test
+ public void testReadPropertiesFile() throws IOException {
+ File tempFile = TestUtils.tempFile();
+ try {
+ String testContent = "a=1\nb=2\n#a comment\n\nc=3\nd=";
+ Files.write(tempFile.toPath(), testContent.getBytes());
+ Properties props = PropertiesUtils.readPropertiesFile(tempFile.getAbsolutePath());
+ assertEquals(4, props.size());
+ assertEquals("1", props.get("a"));
+ assertEquals("2", props.get("b"));
+ assertEquals("3", props.get("c"));
+ assertEquals("", props.get("d"));
+ } finally {
+ Files.deleteIfExists(tempFile.toPath());
+ }
+ }
+
+ @ParameterizedTest
+ @ValueSource(booleans = {false, true})
+ public void testWritePropertiesFile(boolean fsync) throws IOException {
+ File tempFile = TestUtils.tempFile();
+ try {
+ Properties props = new Properties();
+ props.setProperty("abc", "123");
+ props.setProperty("def", "456");
+ PropertiesUtils.writePropertiesFile(props, tempFile.getAbsolutePath(), fsync);
+ Properties props2 = PropertiesUtils.readPropertiesFile(tempFile.getAbsolutePath());
+ assertEquals(props, props2);
+ } finally {
+ Files.deleteIfExists(tempFile.toPath());
+ }
+ }
+
+ @Test
+ public void loadRequiredIntProp() throws IOException {
+ Properties props = new Properties();
+ props.setProperty("foo.bar", "123");
+ assertEquals(123, PropertiesUtils.loadRequiredIntProp(props, "foo.bar"));
+ }
+
+ @Test
+ public void loadMissingRequiredIntProp() throws IOException {
+ Properties props = new Properties();
+ assertEquals("Failed to find foo.bar",
+ assertThrows(RuntimeException.class,
+ () -> PropertiesUtils.loadRequiredIntProp(props, "foo.bar")).
+ getMessage());
+ }
+
+ @Test
+ public void loadNonIntegerRequiredIntProp() throws IOException {
+ Properties props = new Properties();
+ props.setProperty("foo.bar", "b");
+ assertEquals("Unable to read foo.bar as a base-10 number.",
+ assertThrows(RuntimeException.class,
+ () -> PropertiesUtils.loadRequiredIntProp(props, "foo.bar")).
+ getMessage());
+ }
+}