fvaleri commented on code in PR #13417: URL: https://github.com/apache/kafka/pull/13417#discussion_r1270828901
########## checkstyle/import-control.xml: ########## @@ -302,6 +302,11 @@ <allow pkg="org.apache.kafka.clients.admin" /> <allow pkg="org.apache.kafka.clients.producer" /> <allow pkg="org.apache.kafka.clients.consumer" /> + <allow pkg="org.apache.kafka.metadata.bootstrap" /> + <allow pkg="org.apache.kafka.metadata.broker" /> + <allow pkg="org.apache.kafka.storage.internals.log" /> + <allow pkg="kafka.tools" /> + <allow pkg="kafka.utils" /> Review Comment: This entry is duplicated, and we don't need it if you get rid of kafka.utils.TestUtils dependency in StorageToolTest (it's only two simple methods). Then, we can get rid of both kafka.utils and kafka.tools. ########## tools/src/main/java/org/apache/kafka/tools/StorageTool.java: ########## @@ -0,0 +1,478 @@ +/* + * 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.tools; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; +import net.sourceforge.argparse4j.inf.Subparser; +import net.sourceforge.argparse4j.inf.Subparsers; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.metadata.FeatureLevelRecord; +import org.apache.kafka.common.metadata.UserScramCredentialRecord; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.security.scram.internals.ScramFormatter; +import org.apache.kafka.common.security.scram.internals.ScramMechanism; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.metadata.bootstrap.BootstrapDirectory; +import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; +import org.apache.kafka.metadata.broker.BrokerMetadataCheckpoint; +import org.apache.kafka.metadata.broker.MetaProperties; +import org.apache.kafka.metadata.broker.RawMetaProperties; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.storage.internals.log.LogConfig; + +import java.io.IOException; +import java.io.PrintStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.security.InvalidKeyException; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Base64; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.stream.Collectors; + +import static net.sourceforge.argparse4j.impl.Arguments.append; +import static net.sourceforge.argparse4j.impl.Arguments.store; +import static net.sourceforge.argparse4j.impl.Arguments.storeTrue; + +public class StorageTool { + public static void main(String... args) { + try { + Namespace namespace = parseArguments(args); + String command = namespace.getString("command"); + Optional<LogConfig> config = Optional.ofNullable(namespace.getString("config")).map(p -> { + try { + return new LogConfig(Utils.loadProps(p)); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + executeCommand(namespace, command, config); + } catch (Exception exception) { + System.err.println(exception.getMessage()); + Exit.exit(1, exception.getMessage()); + } + } + + private static void executeCommand(Namespace namespace, String command, Optional<LogConfig> config) throws Exception { + final String info = "info"; + final String format = "format"; + if ((command.equals(info) || command.equals(format)) && !config.isPresent()) { + return; // Do nothing if config is not present + } + + switch (command) { + case info: { + List<String> directories = configToLogDirectories(config.get()); + boolean selfManagedMode = configToSelfManagedMode(config.get()); + Exit.exit(infoCommand(System.out, selfManagedMode, directories)); + break; + } + case format: { + List<String> directories = configToLogDirectories(config.get()); + String clusterId = namespace.getString("cluster_id"); + MetadataVersion metadataVersion = getMetadataVersion(namespace, Optional.of(config.get().getInterBrokerProtocolVersionString())); + if (!metadataVersion.isKRaftSupported()) { + throw new TerseException("Must specify a valid KRaft metadata version of at least 3.0."); + } + MetaProperties metaProperties = buildMetadataProperties(clusterId, config.get()); + + List<ApiMessageAndVersion> metadataRecords = new ArrayList<>(); + Optional<List<UserScramCredentialRecord>> scramRecordsOptional = getUserScramCredentialRecords(namespace); + if (scramRecordsOptional.isPresent()) { + if (!metadataVersion.isScramSupported()) { + throw new TerseException("SCRAM is only supported in metadataVersion IBP_3_5_IV2 or later."); + } + for (ApiMessage record : scramRecordsOptional.get()) { + metadataRecords.add(new ApiMessageAndVersion(record, (short) 0)); + } + } + + BootstrapMetadata bootstrapMetadata = buildBootstrapMetadata(metadataVersion, Optional.of(metadataRecords), "format command"); + boolean ignoreFormatted = namespace.getBoolean("ignore_formatted"); + if (!configToSelfManagedMode(config.get())) { + throw new TerseException("The kafka configuration file appears to be for " + + "a legacy cluster. Formatting is only supported for clusters in KRaft mode."); + } + Exit.exit(formatCommand(System.out, directories, metaProperties, bootstrapMetadata, + metadataVersion, ignoreFormatted)); + break; + } + case "random-uuid": { + System.out.println(Uuid.randomUuid()); + Exit.exit(0); + break; + } + default: + throw new RuntimeException("Unknown command " + command); + } + } + + static int infoCommand(PrintStream stream, boolean selfManagedMode, List<String> directories) throws IOException { + List<String> problems = new ArrayList<>(); + List<String> foundDirectories = new ArrayList<>(); + Optional<RawMetaProperties> prevMetadata = Optional.empty(); + for (String directory : directories) { + Path directoryPath = Paths.get(directory); + if (!Files.isDirectory(directoryPath)) { + if (!Files.exists(directoryPath)) { + problems.add(directoryPath + " does not exist"); + } else { + problems.add(directoryPath + " is not a directory"); + } + } else { + foundDirectories.add(directoryPath.toString()); + Path metaPath = directoryPath.resolve("meta.properties"); + if (!Files.exists(metaPath)) { + problems.add(directoryPath + " is not formatted."); + } else { + Properties properties = Utils.loadProps(metaPath.toString()); + RawMetaProperties rawMetaProperties = new RawMetaProperties(properties); + Optional<RawMetaProperties> curMetadata; + + switch (rawMetaProperties.getVersion()) { + case 0: + case 1: + curMetadata = Optional.of(rawMetaProperties); + break; + default: + problems.add("Unsupported version for " + metaPath + ": " + rawMetaProperties.getVersion()); + curMetadata = Optional.empty(); + break; + } + + if (!prevMetadata.isPresent()) { + prevMetadata = curMetadata; + } else { + if (curMetadata.isPresent() && !prevMetadata.get().equals(curMetadata.get())) { + problems.add(String.format("Metadata for %s was %s, but other directories featured %s", metaPath, curMetadata.get(), prevMetadata.get())); + } + } + } + } + } + + if (prevMetadata.isPresent()) { + if (selfManagedMode) { + if (prevMetadata.get().getVersion() == 0) { + problems.add("The kafka configuration file appears to be for a cluster in KRaft mode, but " + "the directories are formatted for legacy mode."); + } + } else if (prevMetadata.get().getVersion() == 1) { + problems.add("The kafka configuration file appears to be for a legacy cluster, but " + "the directories are formatted for a cluster in KRaft mode."); + } + } + + return validateDirectories(stream, directories, problems, foundDirectories, prevMetadata); + } + + private static int validateDirectories(PrintStream stream, List<String> directories, List<String> problems, List<String> foundDirectories, Optional<RawMetaProperties> prevMetadata) { + if (directories.isEmpty()) { + stream.println("No directories specified."); + return 0; + } else { + if (!foundDirectories.isEmpty()) { + if (foundDirectories.size() == 1) { + stream.println("Found log directory:"); + } else { + stream.println("Found log directories:"); + } + foundDirectories.forEach(d -> stream.printf(d + "%n")); Review Comment: In the old implementation we have 2 spaces before the log dir path. It is important to keep the same output, because there may be some tool parsing it that we don't want to break. ########## tools/src/main/java/org/apache/kafka/tools/StorageTool.java: ########## @@ -0,0 +1,478 @@ +/* + * 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.tools; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.Namespace; +import net.sourceforge.argparse4j.inf.Subparser; +import net.sourceforge.argparse4j.inf.Subparsers; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.metadata.FeatureLevelRecord; +import org.apache.kafka.common.metadata.UserScramCredentialRecord; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.security.scram.internals.ScramFormatter; +import org.apache.kafka.common.security.scram.internals.ScramMechanism; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.metadata.bootstrap.BootstrapDirectory; +import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; +import org.apache.kafka.metadata.broker.BrokerMetadataCheckpoint; +import org.apache.kafka.metadata.broker.MetaProperties; +import org.apache.kafka.metadata.broker.RawMetaProperties; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.storage.internals.log.LogConfig; + +import java.io.IOException; +import java.io.PrintStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.security.InvalidKeyException; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Base64; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.stream.Collectors; + +import static net.sourceforge.argparse4j.impl.Arguments.append; +import static net.sourceforge.argparse4j.impl.Arguments.store; +import static net.sourceforge.argparse4j.impl.Arguments.storeTrue; + +public class StorageTool { + public static void main(String... args) { + try { + Namespace namespace = parseArguments(args); + String command = namespace.getString("command"); + Optional<LogConfig> config = Optional.ofNullable(namespace.getString("config")).map(p -> { + try { + return new LogConfig(Utils.loadProps(p)); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + executeCommand(namespace, command, config); + } catch (Exception exception) { + System.err.println(exception.getMessage()); + Exit.exit(1, exception.getMessage()); + } + } + + private static void executeCommand(Namespace namespace, String command, Optional<LogConfig> config) throws Exception { + final String info = "info"; + final String format = "format"; + if ((command.equals(info) || command.equals(format)) && !config.isPresent()) { + return; // Do nothing if config is not present + } + + switch (command) { + case info: { + List<String> directories = configToLogDirectories(config.get()); + boolean selfManagedMode = configToSelfManagedMode(config.get()); + Exit.exit(infoCommand(System.out, selfManagedMode, directories)); + break; + } + case format: { + List<String> directories = configToLogDirectories(config.get()); + String clusterId = namespace.getString("cluster_id"); + MetadataVersion metadataVersion = getMetadataVersion(namespace, Optional.of(config.get().getInterBrokerProtocolVersionString())); + if (!metadataVersion.isKRaftSupported()) { + throw new TerseException("Must specify a valid KRaft metadata version of at least 3.0."); + } + MetaProperties metaProperties = buildMetadataProperties(clusterId, config.get()); + + List<ApiMessageAndVersion> metadataRecords = new ArrayList<>(); + Optional<List<UserScramCredentialRecord>> scramRecordsOptional = getUserScramCredentialRecords(namespace); + if (scramRecordsOptional.isPresent()) { + if (!metadataVersion.isScramSupported()) { + throw new TerseException("SCRAM is only supported in metadataVersion IBP_3_5_IV2 or later."); + } + for (ApiMessage record : scramRecordsOptional.get()) { + metadataRecords.add(new ApiMessageAndVersion(record, (short) 0)); + } + } + + BootstrapMetadata bootstrapMetadata = buildBootstrapMetadata(metadataVersion, Optional.of(metadataRecords), "format command"); + boolean ignoreFormatted = namespace.getBoolean("ignore_formatted"); + if (!configToSelfManagedMode(config.get())) { + throw new TerseException("The kafka configuration file appears to be for " + + "a legacy cluster. Formatting is only supported for clusters in KRaft mode."); + } + Exit.exit(formatCommand(System.out, directories, metaProperties, bootstrapMetadata, + metadataVersion, ignoreFormatted)); + break; + } + case "random-uuid": { + System.out.println(Uuid.randomUuid()); + Exit.exit(0); + break; + } + default: + throw new RuntimeException("Unknown command " + command); + } + } + + static int infoCommand(PrintStream stream, boolean selfManagedMode, List<String> directories) throws IOException { + List<String> problems = new ArrayList<>(); + List<String> foundDirectories = new ArrayList<>(); + Optional<RawMetaProperties> prevMetadata = Optional.empty(); + for (String directory : directories) { + Path directoryPath = Paths.get(directory); + if (!Files.isDirectory(directoryPath)) { + if (!Files.exists(directoryPath)) { + problems.add(directoryPath + " does not exist"); + } else { + problems.add(directoryPath + " is not a directory"); + } + } else { + foundDirectories.add(directoryPath.toString()); + Path metaPath = directoryPath.resolve("meta.properties"); + if (!Files.exists(metaPath)) { + problems.add(directoryPath + " is not formatted."); + } else { + Properties properties = Utils.loadProps(metaPath.toString()); + RawMetaProperties rawMetaProperties = new RawMetaProperties(properties); + Optional<RawMetaProperties> curMetadata; + + switch (rawMetaProperties.getVersion()) { + case 0: + case 1: + curMetadata = Optional.of(rawMetaProperties); + break; + default: + problems.add("Unsupported version for " + metaPath + ": " + rawMetaProperties.getVersion()); + curMetadata = Optional.empty(); + break; + } + + if (!prevMetadata.isPresent()) { + prevMetadata = curMetadata; + } else { + if (curMetadata.isPresent() && !prevMetadata.get().equals(curMetadata.get())) { + problems.add(String.format("Metadata for %s was %s, but other directories featured %s", metaPath, curMetadata.get(), prevMetadata.get())); + } + } + } + } + } + + if (prevMetadata.isPresent()) { + if (selfManagedMode) { + if (prevMetadata.get().getVersion() == 0) { + problems.add("The kafka configuration file appears to be for a cluster in KRaft mode, but " + "the directories are formatted for legacy mode."); + } + } else if (prevMetadata.get().getVersion() == 1) { + problems.add("The kafka configuration file appears to be for a legacy cluster, but " + "the directories are formatted for a cluster in KRaft mode."); + } + } + + return validateDirectories(stream, directories, problems, foundDirectories, prevMetadata); + } + + private static int validateDirectories(PrintStream stream, List<String> directories, List<String> problems, List<String> foundDirectories, Optional<RawMetaProperties> prevMetadata) { + if (directories.isEmpty()) { + stream.println("No directories specified."); + return 0; + } else { + if (!foundDirectories.isEmpty()) { + if (foundDirectories.size() == 1) { + stream.println("Found log directory:"); + } else { + stream.println("Found log directories:"); + } + foundDirectories.forEach(d -> stream.printf(d + "%n")); + stream.println(""); + } + + if (prevMetadata.isPresent()) { + RawMetaProperties prev = prevMetadata.get(); + stream.println("Found metadata: " + prev); + stream.println(""); + } + + if (!problems.isEmpty()) { + if (problems.size() == 1) { + stream.println("Found problem:"); + } else { + stream.println("Found problems:"); + } + problems.forEach(d -> stream.printf(d + "%n")); Review Comment: In the old implementation we have 2 spaces before the problem desc. It is important to keep the same output, because there may be some tool parsing it that we don't want to break. -- 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