soarez commented on code in PR #14628:
URL: https://github.com/apache/kafka/pull/14628#discussion_r1371749943


##########
metadata/src/main/java/org/apache/kafka/metadata/properties/MetaProperties.java:
##########
@@ -0,0 +1,266 @@
+/*
+ * 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<String> clusterId;
+
+    /**
+     * The node ID, which may be OptionalInt.empty in V0.
+     */
+    private final OptionalInt nodeId;
+
+    /**
+     * The JBOD directory ID, or Optional.empty if none is specified.

Review Comment:
   ```suggestion
        * The directory ID, or Optional.empty if none is specified.
   ```



##########
metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsemble.java:
##########
@@ -0,0 +1,497 @@
+/*
+ * 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.KafkaException;
+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.Optional;
+import java.util.OptionalInt;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.function.BiConsumer;
+
+/**
+ * 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 class MetaPropertiesEnsemble {
+    /**
+     * The log4j object for this class.
+     */
+    private 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<String> emptyLogDirs;
+
+    /**
+     * The set of log dirs that had errors.
+     */
+    private final Set<String> errorLogDirs;
+
+    /**
+     * A map from log directories to the meta.properties information inside 
each one.
+     */
+    private final Map<String, MetaProperties> logDirProps;
+
+    /**
+     * The metadata log directory, or the empty string if there is none.
+     */
+    private final Optional<String> metadataLogDir;
+
+    /**
+     * Utility class for loading a MetaPropertiesEnsemble from the disk.
+     */
+    public static class Loader {
+        private final TreeSet<String> logDirs = new TreeSet<>();
+        private Optional<String> metadataLogDir = Optional.empty();
+
+        public Loader addLogDirs(Collection<String> 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<String> emptyLogDirs = new HashSet<>();
+            Set<String> errorLogDirs = new HashSet<>();
+            Map<String, MetaProperties> 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);
+        }
+    }
+
+    /**
+     * Utility class for copying a MetaPropertiesEnsemble object, possibly 
with changes.
+     */
+    public static class Copier {
+        private final MetaPropertiesEnsemble prev;
+        private Random random = new Random();
+        private Set<String> emptyLogDirs;
+        private Set<String> errorLogDirs;
+        private Map<String, MetaProperties> logDirProps;
+        private Optional<String> metaLogDir;
+
+        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<String> emptyLogDirs() {
+            return emptyLogDirs;
+        }
+
+        /**
+         * Access the mutable error log directories set.
+         *
+         * @return          The mutable error log directories set.
+         */
+        public Set<String> errorLogDirs() {
+            return errorLogDirs;
+        }
+
+        /**
+         * Access the mutable logDirProps map.
+         *
+         * @return          The mutable error log directories map.
+         */
+        public Map<String, MetaProperties> logDirProps() {
+            return logDirProps;
+        }
+
+        /**
+         * 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.isSafe()) {
+                    boolean duplicate = false;
+                    for (MetaProperties metaProps : logDirProps.values()) {
+                        if (metaProps.directoryId().equals(Optional.of(uuid))) 
{
+                            duplicate = true;
+                            break;
+                        }
+                    }
+                    if (!duplicate) {
+                        return uuid;
+                    }
+                }
+            }
+        }
+
+        /**
+         * 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.");
+                }
+            }
+        }
+
+        /**
+         * Write any changed log directories out to disk.
+         */
+        public void writeLogDirChanges() {
+            writeLogDirChanges((metaPropsPath, e) -> {
+                LOG.error("Error while writing meta.properties file {}", 
metaPropsPath, e);
+            });
+        }
+
+        /**
+         * Write any changed log directories out to disk.
+         *
+         * @param errorHandler  A handler that will be called with (logDir, 
exception) any time we
+         *                      hit an exception writing out a meta.properties 
file.
+         */
+        public void writeLogDirChanges(BiConsumer<String, Exception> 
errorHandler) {
+            Map<String, MetaProperties> newOrChanged = new HashMap<>();
+            HashSet<String> newSet = new HashSet<>();
+            for (Entry<String, MetaProperties> entry : 
prev.logDirProps().entrySet()) {
+                MetaProperties metaProps = entry.getValue();
+                MetaProperties prevMetaProps = logDirProps.get(entry.getKey());
+                if (!metaProps.equals(prevMetaProps)) {
+                    newOrChanged.put(entry.getKey(), entry.getValue());
+                }
+            }
+            for (Entry<String, MetaProperties> entry : logDirProps.entrySet()) 
{
+                if (!prev.logDirProps.containsKey(entry.getKey())) {
+                    newOrChanged.put(entry.getKey(), entry.getValue());
+                    newSet.add(entry.getKey());
+                }
+            }
+            for (Entry<String, MetaProperties> entry : 
newOrChanged.entrySet()) {
+                String logDir = entry.getKey();
+                String metaPropsPath = new File(logDir, 
META_PROPERTIES_NAME).getAbsolutePath();
+                try {
+                    
PropertiesUtils.writePropertiesFile(entry.getValue().toProperties(), 
metaPropsPath);
+                    LOG.info("Wrote out {} meta.properties file {} containing 
{}",
+                            newSet.contains(entry.getKey()) ? "new" : 
"changed",
+                            entry.getKey(), entry.getValue());
+                } catch (Exception e) {

Review Comment:
   `PropertiesUtils.writePropertiesFile` throws `IOException`, so can we 
restrict the catch exception type to that?



##########
core/src/main/scala/kafka/server/KafkaRaftServer.scala:
##########
@@ -135,39 +136,52 @@ 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): (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()
+    initialMetaPropsEnsemble.emptyLogDirs().forEach(logDir => {
+      throw new RuntimeException(s"No `meta.properties` found in $logDir (have 
you run `kafka-storage.sh` " +
+        "to format the directory?)")
+    })
+    val verificationFlags = if (config.migrationEnabled) {
+      util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR)
+    } else {
+      util.EnumSet.of(REQUIRE_V0, REQUIRE_AT_LEAST_ONE_VALID)
     }
+    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}).")
+        }
       }
-    }
-
-    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).")
-    }
+    })
+
+    // Set directory IDs on all directories. Rewrite the files if needed.

Review Comment:
   This is a better place to generate missing directory IDs 👍 



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogDirFailureChannel.java:
##########
@@ -57,7 +56,7 @@ public boolean hasOfflineLogDir(String logDir) {
      * @param msg Error message.
      * @param e Exception instance.
      */
-    public void maybeAddOfflineLogDir(String logDir, String msg, IOException 
e) {
+    public void maybeAddOfflineLogDir(String logDir, String msg, Exception e) {

Review Comment:
   It seems odd to take a directory offline with an exception that's not 
related to IO. I think we don't need this change if 
`MetaPropertiesEnsemble.writeLogDirChanges()` restricts the catch type to 
`IOException`.



##########
core/src/main/scala/kafka/tools/StorageTool.scala:
##########
@@ -407,36 +414,41 @@ 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]))
+
+    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} directory 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 (copier.emptyLogDirs().isEmpty) {
       stream.println("All of the log directories are already formatted.")
+    } else {
+      copier.emptyLogDirs().forEach(logDir => {
+        val newMetaProperties = new MetaProperties.Builder(metaProperties).
+          setDirectoryId(copier.generateValidDirectoryId()).
+          build()
+        copier.logDirProps().put(logDir, newMetaProperties)

Review Comment:
   This interface is a bit strange – rather than returning a modifiable map, 
something like `copier.putLogDirProps(logDir, newMetaProperties)` would be more 
intuitive.



##########
core/src/main/scala/kafka/server/KafkaServer.scala:
##########
@@ -228,23 +226,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()
         }

Review Comment:
   Shouldn't we be generating missing directory IDs here? As we do in 
KafkaRaftServer, since LogManager no longer does that.



##########
metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesVersion.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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 {
+    V0(0),
+    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 "" + number;
+    }
+
+    public boolean hasBrokerId() {
+        return this == V0;
+    }
+
+    public boolean alwaysHasId() {

Review Comment:
   This one is a bit oddly named. Did you mean `alwaysHasNodeId` instead?



##########
core/src/main/scala/kafka/log/LogManager.scala:
##########
@@ -267,35 +267,27 @@ 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(dirs: Seq[File]): Map[String, Uuid] = {
+    val result = mutable.HashMap[String, Uuid]()
+    dirs.foreach(dir => {
       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 = Uuid.randomUuid()
-              rawMetaProperties.directoryId = uuid.toString
-              metadataCheckpoint.write(rawMetaProperties.props)
-              uuid
-          }
-          dir.getAbsolutePath -> uuid
-        }.toMap
+        val props = PropertiesUtils.readPropertiesFile(
+          new File(dir, 
MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath)
+        val metaProps = new MetaProperties.Builder(props).build()
+        metaProps.directoryId().ifPresent(directoryId => {
+          result += (dir.getAbsolutePath -> directoryId)
+        })

Review Comment:
   This seems off to me. I think we should expect every log directory to have 
an ID by this point.



##########
core/src/main/scala/kafka/tools/StorageTool.scala:
##########
@@ -407,36 +414,41 @@ 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]))
+
+    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} directory 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 (copier.emptyLogDirs().isEmpty) {
       stream.println("All of the log directories are already formatted.")
+    } else {
+      copier.emptyLogDirs().forEach(logDir => {
+        val newMetaProperties = new MetaProperties.Builder(metaProperties).
+          setDirectoryId(copier.generateValidDirectoryId()).
+          build()
+        copier.logDirProps().put(logDir, newMetaProperties)

Review Comment:
   This interface is a bit strange – rather than returning a modifiable map, 
something like `copier.putLogDirProps(logDir, newMetaProperties)` would be more 
intuitive.



##########
core/src/main/scala/kafka/server/KafkaRaftServer.scala:
##########
@@ -135,39 +136,52 @@ 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): (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()
+    initialMetaPropsEnsemble.emptyLogDirs().forEach(logDir => {
+      throw new RuntimeException(s"No `meta.properties` found in $logDir (have 
you run `kafka-storage.sh` " +
+        "to format the directory?)")
+    })
+    val verificationFlags = if (config.migrationEnabled) {
+      util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR)
+    } else {
+      util.EnumSet.of(REQUIRE_V0, REQUIRE_AT_LEAST_ONE_VALID)
     }
+    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}).")
+        }
       }
-    }
-
-    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).")
-    }
+    })
+
+    // Set directory IDs on all directories. Rewrite the files if needed.

Review Comment:
   This is a better place to generate missing directory IDs 👍 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to