This is an automated email from the ASF dual-hosted git repository.
rndgstn pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new 631e6be3a02 KAFKA-14711: kafaka-metadata-quorum.sh does not honor
--command-confi… (#13241)
631e6be3a02 is described below
commit 631e6be3a02efe165611990e9f6721f7687d5457
Author: Ron Dagostino <[email protected]>
AuthorDate: Mon Feb 13 18:33:20 2023 -0500
KAFKA-14711: kafaka-metadata-quorum.sh does not honor --command-confi…
(#13241)
…g option
https://github.com/apache/kafka/pull/12951 accidentally changed the
behavior of the `kafaka-metadata-quorum.sh` CLI by making it silently ignore a
`--command-config <filename>` properties file that exists. This was an
undetected regression in the 3.4.0 release. This patch fixes the issue such
that any such specified file will be honored.
Reviewers: José Armando García Sancio <[email protected]>, Ismael Juma
<[email protected]>
---
checkstyle/import-control.xml | 1 +
.../apache/kafka/tools/MetadataQuorumCommand.java | 21 +++++++++++++--------
.../kafka/tools/MetadataQuorumCommandTest.java | 14 ++++++++++++++
3 files changed, 28 insertions(+), 8 deletions(-)
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 32f77c59d0c..71989a9daa4 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -411,6 +411,7 @@
<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.test" />
<allow pkg="com.fasterxml.jackson" />
<allow pkg="org.jose4j" />
<allow pkg="net.sourceforge.argparse4j" />
diff --git
a/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java
b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java
index 6cafea81b52..5f7c9fbd0a2 100644
--- a/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java
+++ b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java
@@ -31,6 +31,7 @@ import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.server.util.ToolsUtils;
import java.io.File;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.List;
@@ -86,14 +87,8 @@ public class MetadataQuorumCommand {
Namespace namespace = parser.parseArgsOrFail(args);
String command = namespace.getString("command");
- File commandConfig = namespace.get("command_config");
- Properties props = new Properties();
- if (commandConfig != null) {
- if (!commandConfig.exists())
- throw new TerseException("Properties file " +
commandConfig.getPath() + " does not exists!");
-
- Utils.loadProps(commandConfig.getPath());
- }
+ File optionalCommandConfig = namespace.get("command_config");
+ final Properties props = getProperties(optionalCommandConfig);
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG,
namespace.getString("bootstrap_server"));
admin = Admin.create(props);
@@ -116,6 +111,16 @@ public class MetadataQuorumCommand {
}
}
+ private static Properties getProperties(File optionalCommandConfig) throws
TerseException, IOException {
+ if (optionalCommandConfig == null) {
+ return new Properties();
+ } else {
+ if (!optionalCommandConfig.exists())
+ throw new TerseException("Properties file " +
optionalCommandConfig.getPath() + " does not exists!");
+ return Utils.loadProps(optionalCommandConfig.getPath());
+ }
+ }
+
private static void addDescribeParser(Subparsers subparsers) {
Subparser describeParser = subparsers
.addParser("describe")
diff --git
a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java
b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java
index bd4a7a0c07c..5d19b294333 100644
--- a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java
+++ b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java
@@ -22,10 +22,14 @@ import kafka.test.annotation.ClusterTestDefaults;
import kafka.test.annotation.ClusterTests;
import kafka.test.annotation.Type;
import kafka.test.junit.ClusterTestExtensions;
+import org.apache.kafka.clients.admin.AdminClientConfig;
import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.extension.ExtendWith;
+import java.io.File;
+import java.io.IOException;
import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.regex.Pattern;
@@ -141,6 +145,16 @@ class MetadataQuorumCommandTest {
assertEquals("0", replicationOutput.split("\n")[1].split("\\s+")[2]);
}
+ @ClusterTests({
+ @ClusterTest(clusterType = Type.CO_KRAFT, brokers = 1, controllers = 1)
+ })
+ public void testCommandConfig() throws IOException {
+ // specifying a --command-config containing properties that would
prevent login must fail
+ File tmpfile =
TestUtils.tempFile(AdminClientConfig.SECURITY_PROTOCOL_CONFIG +
"=SSL_PLAINTEXT");
+ assertEquals(1, MetadataQuorumCommand.mainNoExit("--bootstrap-server",
cluster.bootstrapServers(),
+ "--command-config", tmpfile.getAbsolutePath(),
"describe", "--status"));
+ }
+
@ClusterTest(clusterType = Type.ZK, brokers = 1)
public void testDescribeQuorumInZkMode() {
assertTrue(