From 21b96aec7df6d19bef2e1c4e7d2d55ca608455bb Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 21 Dec 2025 18:02:50 +0100 Subject: [PATCH 01/14] cassandra-19476 cql management api --- .../exceptions/CommandExecutionException.java | 32 +++ .../cassandra/management/api/Command.java | 43 ++++ .../management/api/CommandContext.java | 27 +++ .../management/api/CommandInvoker.java | 148 ++++++++++++ .../management/api/CommandMBean.java | 218 ++++++++++++++++++ .../management/api/CommandsRegistry.java | 29 +++ .../management/api/LocalCommand.java | 29 +++ .../tools/nodetool/ManagementContext.java | 31 +++ .../JmxManagementDistributedTest.java | 89 +++++++ 9 files changed, 646 insertions(+) create mode 100644 src/java/org/apache/cassandra/exceptions/CommandExecutionException.java create mode 100644 src/java/org/apache/cassandra/management/api/Command.java create mode 100644 src/java/org/apache/cassandra/management/api/CommandContext.java create mode 100644 src/java/org/apache/cassandra/management/api/CommandInvoker.java create mode 100644 src/java/org/apache/cassandra/management/api/CommandMBean.java create mode 100644 src/java/org/apache/cassandra/management/api/CommandsRegistry.java create mode 100644 src/java/org/apache/cassandra/management/api/LocalCommand.java create mode 100644 src/java/org/apache/cassandra/tools/nodetool/ManagementContext.java create mode 100644 test/distributed/org/apache/cassandra/distributed/test/management/JmxManagementDistributedTest.java diff --git a/src/java/org/apache/cassandra/exceptions/CommandExecutionException.java b/src/java/org/apache/cassandra/exceptions/CommandExecutionException.java new file mode 100644 index 000000000000..2d07df846d22 --- /dev/null +++ b/src/java/org/apache/cassandra/exceptions/CommandExecutionException.java @@ -0,0 +1,32 @@ +/* + * 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.cassandra.exceptions; + +public class CommandExecutionException extends CassandraException +{ + public CommandExecutionException(String msg) + { + super(ExceptionCode.INVALID, msg); + } + + public CommandExecutionException(String msg, Throwable cause) + { + super(ExceptionCode.INVALID, msg, cause); + } +} diff --git a/src/java/org/apache/cassandra/management/api/Command.java b/src/java/org/apache/cassandra/management/api/Command.java new file mode 100644 index 000000000000..d00f44b08b0f --- /dev/null +++ b/src/java/org/apache/cassandra/management/api/Command.java @@ -0,0 +1,43 @@ +/* + * 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.cassandra.management.api; + +import org.apache.cassandra.tools.nodetool.AbstractCommand; + +/** + * Command interface for all management commands. + *

+ * Name of the command that is expected from caller derived from actual command class name.
+ *

+ *

+ * Other protocols must expose command similarly. Rest API must expect {@code /api-root/my-usefull?param1=value1¶m2=value2} URI. + * + * @param Command user object type. + * @param Command result type. + */ +public interface Command +{ + String name(); + String description(); + Class commandUserObject(); +} diff --git a/src/java/org/apache/cassandra/management/api/CommandContext.java b/src/java/org/apache/cassandra/management/api/CommandContext.java new file mode 100644 index 000000000000..665060185feb --- /dev/null +++ b/src/java/org/apache/cassandra/management/api/CommandContext.java @@ -0,0 +1,27 @@ +/* + * 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.cassandra.management.api; + +import org.apache.cassandra.tools.Output; + +public interface CommandContext +{ + T findMBean(Class clazz); + Output output(); +} diff --git a/src/java/org/apache/cassandra/management/api/CommandInvoker.java b/src/java/org/apache/cassandra/management/api/CommandInvoker.java new file mode 100644 index 000000000000..165e4caf8d27 --- /dev/null +++ b/src/java/org/apache/cassandra/management/api/CommandInvoker.java @@ -0,0 +1,148 @@ +/* + * 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.cassandra.management.api; + +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.function.Consumer; + +import javax.inject.Inject; + +import org.apache.cassandra.exceptions.CommandExecutionException; +import picocli.CommandLine; + +public class CommandInvoker +{ + private final Command command; + private final CommandContext context; + private final String[] args; + + public CommandInvoker(Command command, CommandContext context, String... args) + { + this.command = command; + this.context = context; + this.args = args; + } + + /** + * Invoke the command and return the exit code. + * @param resultConsumer a consumer for the result. + * @return the exit code. + */ + public int invokeExitCode(Consumer resultConsumer) + { + return new CommandLine(instantiateCommand(command), new InjectContextFactory(context)).execute(args); + } + + /** + * Invoke the command and return the result. + * @param printer a consumer for the result. + * @return the result. + */ + public R invokeResult(Consumer printer) + { + new CommandLine(instantiateCommand(command), new InjectContextFactory(context)).execute(args); + return null; + } + + private Object instantiateCommand(Command command) + { + try + { + return new InjectContextFactory(context).create(command.commandUserObject()); + } + catch (Exception e) + { + throw new CommandExecutionException(String.format("Failed to inject resource [command=%s]", + command.commandUserObject()), e); + } + } + + private static class InjectContextFactory implements CommandLine.IFactory + { + private final CommandContext context; + + public InjectContextFactory(CommandContext context) + { + this.context = context; + } + + /** + * Create an instance of the specified class. This method is called by picocli to obtain instances of classes. + * @param clazz the class of the object to return. + * @return an instance of the specified class. + * @param the type of the object to return. + * @throws Exception an exception. + */ + public T create(Class clazz) throws Exception + { + if (clazz.isInterface()) + { + if (Collection.class.isAssignableFrom(clazz)) + { + if (List.class.isAssignableFrom(clazz)) + return clazz.cast(new ArrayList<>()); + else if (SortedSet.class.isAssignableFrom(clazz)) + return clazz.cast(new TreeSet<>()); + else if (Set.class.isAssignableFrom(clazz)) + return clazz.cast(new LinkedHashSet<>()); + else if (Queue.class.isAssignableFrom(clazz)) + return clazz.cast(new LinkedList<>()); + else + return clazz.cast(new ArrayList<>()); + } + if (SortedMap.class.isAssignableFrom(clazz)) + { + return clazz.cast(new TreeMap<>()); + } + if (Map.class.isAssignableFrom(clazz)) + { + return clazz.cast(new LinkedHashMap<>()); + } + } + + // Inject the context into the command object. + Object target = clazz.getDeclaredConstructor().newInstance(); + for (Field field : clazz.getDeclaredFields()) + { + if (field.isAnnotationPresent(Inject.class) && field.getType().isAssignableFrom(context.getClass())) + inject(field, target, context); + } + return clazz.cast(target); + } + + private static void inject(Field field, Object target, Object resource) throws Exception + { + field.setAccessible(true); + field.set(target, resource); + } + } +} diff --git a/src/java/org/apache/cassandra/management/api/CommandMBean.java b/src/java/org/apache/cassandra/management/api/CommandMBean.java new file mode 100644 index 000000000000..aab5f3f0ce39 --- /dev/null +++ b/src/java/org/apache/cassandra/management/api/CommandMBean.java @@ -0,0 +1,218 @@ +/* + * 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.cassandra.management.api; + +import java.lang.reflect.AnnotatedElement; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.List; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; +import javax.management.Attribute; +import javax.management.AttributeList; +import javax.management.AttributeNotFoundException; +import javax.management.DynamicMBean; +import javax.management.InvalidAttributeValueException; +import javax.management.MBeanException; +import javax.management.MBeanInfo; +import javax.management.MBeanOperationInfo; +import javax.management.MBeanParameterInfo; +import javax.management.ReflectionException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import picocli.CommandLine; + +import static javax.management.MBeanOperationInfo.ACTION; +import static org.apache.cassandra.management.CommandUtils.optionName; +import static org.apache.cassandra.management.CommandUtils.buildAttributeDescription; +import static org.apache.cassandra.management.CommandUtils.paramName; + +/** + * Command MBean exposes single mamagement command to the JMX interface. + * + * @see Command + * @see CommandsRegistry + */ +public class CommandMBean implements DynamicMBean +{ + /** Name of the JMX method to invoke command. */ + public static final String INVOKE = "invoke"; + public static final String OPTIONAL = "0..1"; + public static final String REPREATABLE = "0..*"; + + /** Used for tests. Name of the method to retrive last method result. */ + public static final String LAST_RES_METHOD = "lastResult"; + + private static final Logger logger = LoggerFactory.getLogger(CommandMBean.class); + private final Command command; + + /** + * @param command Management command to expose. + */ + public CommandMBean(Command command) + { + this.command = command; + } + + @Override + public Object invoke(String actionName, + Object[] params, + String[] signature) throws MBeanException, ReflectionException + { + // Default JMX invoker pass arguments in for params: Object[] = { "invoke", parameter_values_array, types_array} + // while JConsole pass params values directly in params array. This check supports both way of invocation. + if (params.length == 3 + && (params[0].equals(INVOKE) || params[0].equals(LAST_RES_METHOD)) + && params[1] instanceof Object[]) + return invoke((String) params[0], (Object[]) params[1], (String[]) params[2]); + + if (!INVOKE.equals(actionName)) + throw new UnsupportedOperationException(actionName); + + try + { + StringBuilder resStr = new StringBuilder(); + Consumer printer = str -> resStr.append(str).append('\n'); + +// CommandInvoker invoker = new CommandInvoker<>(command, new ParamsToArgument(params).argument(), ignite); +// res = invoker.invoke(printer, false); + + return resStr.toString(); + } + catch (Exception e) + { + logger.error("Invoke error:", e); + throw e; + } + } + + /** + * All options are required within the ArgGroup, while the group itself is optional: + *

+     * public class DependentOptions {
+     *     @ArgGroup(exclusive = false, multiplicity = "0..1")
+     *     Dependent group;
+     *
+     *     static class Dependent {
+     *         @Option(names = "-a", required = true) int a;
+     *         @Option(names = "-b", required = true) int b;
+     *         @Option(names = "-c", required = true) int c;
+     *     }
+     * }
+ * + * @return MBean info. + */ + @Override + public MBeanInfo getMBeanInfo() + { + List args = new ArrayList<>(); + visitCommandSpec(command, (name, desc) -> args.add(new MBeanParameterInfo(name, String.class.getName(), desc))); + return new MBeanInfo(CommandMBean.class.getName(), + command.getClass().getSimpleName(), + null, + null, + new MBeanOperationInfo[]{ + new MBeanOperationInfo(INVOKE, + command.description(), + args.toArray(MBeanParameterInfo[]::new), + String.class.getName(), + ACTION) + }, + null); + } + + public static void visitCommandSpec(Command command, BiConsumer visitor) + { + CommandLine.Model.CommandSpec spec = CommandLine.Model.CommandSpec.forAnnotatedObject(command.commandUserObject()); + // Options are required within the ArgGroup, while the group itself is optional. + for (CommandLine.Model.OptionSpec option : spec.options()) + { + if (!option.isOption()) + continue; + + visitor.accept(optionName(option.longestName()), buildAttributeDescription(option.description())); + } + + // Positional parameters are required. + for (CommandLine.Model.PositionalParamSpec positional : spec.positionalParameters()) + { + if (!positional.isPositional()) + continue; + + visitor.accept(paramName(positional.paramLabel()), buildAttributeDescription(positional.description())); + } + } + + @Override + public AttributeList getAttributes(String[] attributes) + { + throw new UnsupportedOperationException("getAttributes"); + } + + @Override + public AttributeList setAttributes(AttributeList attributes) + { + throw new UnsupportedOperationException("setAttributes"); + } + + @Override + public Object getAttribute(String attribute) throws AttributeNotFoundException, MBeanException, ReflectionException + { + throw new UnsupportedOperationException("getAttribute"); + } + + @Override + public void setAttribute(Attribute attribute) throws AttributeNotFoundException, InvalidAttributeValueException, + MBeanException, ReflectionException + { + throw new UnsupportedOperationException("setAttribute"); + } + + /** */ + private class ParamsToArgument implements Function + { + private int cntr; + private final Object[] vals; + + private ParamsToArgument(Object[] vals) + { + this.vals = vals; + } + + public Object argument() + { + // This will map vals to argument fields. +// return CommandUtils.argument(command.argClass(), (fld, pos) -> apply(fld), this); + return null; + } + + @Override + public Object apply(Field field) + { + String val = (String) vals[cntr]; + + cntr++; + +// return !F.isEmpty(val) ? CommandUtils.parseVal(val, field.getType()) : null; + return null; + } + } +} diff --git a/src/java/org/apache/cassandra/management/api/CommandsRegistry.java b/src/java/org/apache/cassandra/management/api/CommandsRegistry.java new file mode 100644 index 000000000000..858fb39bdedf --- /dev/null +++ b/src/java/org/apache/cassandra/management/api/CommandsRegistry.java @@ -0,0 +1,29 @@ +/* + * 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.cassandra.management.api; + +import java.util.Map; + +/** + * Registry that knows all of its subcommands. + */ +public interface CommandsRegistry extends Iterable>> +{ + Command command(String name); +} diff --git a/src/java/org/apache/cassandra/management/api/LocalCommand.java b/src/java/org/apache/cassandra/management/api/LocalCommand.java new file mode 100644 index 000000000000..da724289a2fb --- /dev/null +++ b/src/java/org/apache/cassandra/management/api/LocalCommand.java @@ -0,0 +1,29 @@ +/* + * 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.cassandra.management.api; + +import java.util.function.Consumer; + +/** + * A command that can be executed locally on the node, without the need to know the remote node. + */ +public interface LocalCommand extends Command +{ + R execute(CommandContext context, Consumer printer); +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/ManagementContext.java b/src/java/org/apache/cassandra/tools/nodetool/ManagementContext.java new file mode 100644 index 000000000000..44e9e9c7cbd1 --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/ManagementContext.java @@ -0,0 +1,31 @@ +/* + * 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.cassandra.tools.nodetool; + +import java.net.UnknownHostException; + +import org.apache.cassandra.service.StorageServiceMBean; + +/** + * Management context for nodetool commands to access management services like StorageServiceMBean etc. + */ +public interface ManagementContext +{ + T getManagementService(Class serviceClass); +} diff --git a/test/distributed/org/apache/cassandra/distributed/test/management/JmxManagementDistributedTest.java b/test/distributed/org/apache/cassandra/distributed/test/management/JmxManagementDistributedTest.java new file mode 100644 index 000000000000..02a9da943b2b --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/management/JmxManagementDistributedTest.java @@ -0,0 +1,89 @@ +/* + * 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.cassandra.distributed.test.management; + +import java.util.HashSet; +import java.util.Set; +import javax.management.MBeanServerConnection; +import javax.management.remote.JMXConnector; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInstanceConfig; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.impl.INodeProvisionStrategy; +import org.apache.cassandra.distributed.shared.JMXUtil; +import org.apache.cassandra.distributed.test.TestBaseImpl; + +import static org.apache.cassandra.distributed.test.jmx.JMXGetterCheckTest.testAllValidGetters; +import static org.hamcrest.Matchers.startsWith; + +public class JmxManagementDistributedTest extends TestBaseImpl +{ + @Test + public void testOneNetworkInterfaceProvisioning() throws Exception + { + Set allInstances = new HashSet<>(); + int iterations = 2; // Make sure the JMX infrastructure all cleans up properly by running this multiple times. + + try (Cluster cluster = Cluster.build(1) + .withDynamicPortAllocation(true) + .withNodeProvisionStrategy(INodeProvisionStrategy.Strategy.OneNetworkInterface) + .withConfig(c -> c.with(Feature.values())).start()) + { + Set instancesContacted = new HashSet<>(); + for (IInvokableInstance instance : cluster) + { + testInstance(instancesContacted, instance); + } + + while (!Thread.currentThread().isInterrupted()) + { + Thread.sleep(1000); + System.out.println(">>>>> sleeping"); + } + allInstances.addAll(instancesContacted); + // Make sure we actually exercise the mbeans by testing a bunch of getters. + // Without this it's possible for the test to pass as we don't touch any mBeans that we register. + testAllValidGetters(cluster); + } + Assert.assertEquals("Each instance from each cluster should have been unique", iterations * 2, allInstances.size()); + } + + private void testInstance(Set instancesContacted, IInvokableInstance instance) + { + IInstanceConfig config = instance.config(); + try (JMXConnector jmxc = JMXUtil.getJmxConnector(config)) + { + MBeanServerConnection mbsc = jmxc.getMBeanServerConnection(); + // instances get their default domain set to their IP address, so us it + // to check that we are actually connecting to the correct instance + String defaultDomain = mbsc.getDefaultDomain(); + instancesContacted.add(defaultDomain); + Assert.assertThat(defaultDomain, startsWith(JMXUtil.getJmxHost(config) + ':' + config.jmxPort())); + } + catch (Throwable t) + { + throw new RuntimeException("Could not connect to JMX", t); + } + } +} From 8431f8a2e9f53a33230ad33a1a0a2fcb9f220d75 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 30 Dec 2025 22:25:29 +0100 Subject: [PATCH 02/14] Add CQL Mapagement API support, see desing doc CEP-38 --- src/antlr/Lexer.g | 1 + src/antlr/Parser.g | 42 +- .../cassandra/audit/AuditLogEntryType.java | 3 +- .../auth/AbstractCIDRAuthorizer.java | 4 +- .../org/apache/cassandra/auth/AuthCache.java | 31 + .../cassandra/auth/AuthCacheService.java | 6 + .../auth/NetworkPermissionsCache.java | 6 + .../cassandra/auth/PasswordAuthenticator.java | 6 + .../cassandra/auth/PermissionsCache.java | 6 + .../org/apache/cassandra/auth/RolesCache.java | 6 + .../auth/jmx/AuthorizationProxy.java | 6 + .../config/CassandraRelevantEnv.java | 6 + .../config/CassandraRelevantProperties.java | 6 + .../statements/ExecuteCommandStatement.java | 220 +++++ .../db/virtual/CIDRFilteringMetricsTable.java | 2 +- .../management/CassandraCommandRegistry.java | 123 +++ .../management/CommandExecutionArgsSerde.java | 260 +++++ .../cassandra/management/CommandInvoker.java | 107 ++ .../cassandra/management/CommandMBean.java | 344 +++++++ .../cassandra/management/CommandService.java | 363 +++++++ .../management/CommandServiceMBean.java | 48 + .../management/InternalNodeMBeanAccessor.java | 476 +++++++++ .../cassandra/management/MBeanAccessor.java | 160 +++ .../cassandra/management/ManagementUtils.java | 66 ++ .../SimpleCommandExecutionArgs.java | 63 ++ .../api/ArgumentMetadata.java} | 21 +- .../cassandra/management/api/Command.java | 59 +- .../management/api/CommandExecutionArgs.java | 32 + ...text.java => CommandExecutionContext.java} | 5 +- .../management/api/CommandInvoker.java | 148 --- .../management/api/CommandMBean.java | 218 ----- .../management/api/CommandMetadata.java | 35 + .../management/api/CommandRegistry.java | 45 + ...ocalCommand.java => CommandsProvider.java} | 9 +- ...mandsRegistry.java => OptionMetadata.java} | 8 +- .../management/api/ParameterMetadata.java | 25 + .../picocli/PicocliCommandAdapter.java | 125 +++ .../picocli/PicocliCommandArgsConverter.java | 201 ++++ .../picocli/PicocliCommandMetadata.java | 123 +++ .../PicocliCommandRegistryAdapter.java | 137 +++ .../picocli/PicocliCommandsProvider.java | 63 ++ .../picocli/PicocliMetadataExtractor.java | 53 + .../picocli/PicocliOptionMetadata.java | 141 +++ .../picocli/PicocliParameterMetadata.java | 132 +++ .../management/picocli/TypeConverter.java | 38 + .../picocli/TypeConverterRegistry.java | 150 +++ .../cassandra/service/CassandraDaemon.java | 5 + .../cassandra/tools/INodeProbeFactory.java | 4 +- .../org/apache/cassandra/tools/NodeProbe.java | 913 ++++++------------ .../org/apache/cassandra/tools/NodeTool.java | 5 +- .../tools/RemoteJmxMBeanAccessor.java | 439 +++++++++ .../apache/cassandra/tools/RepairRunner.java | 10 +- .../tools/nodetool/AbstractCommand.java | 2 +- .../cassandra/tools/nodetool/CqlConnect.java | 107 ++ .../tools/nodetool/ForceCompact.java | 1 + .../cassandra/tools/nodetool/History.java | 2 +- .../cassandra/tools/nodetool/JmxConnect.java | 95 +- .../apache/cassandra/tools/nodetool/Sjk.java | 8 +- .../WithPortDisplayAbstractCommand.java | 2 +- .../strategy/CommandExecutionStraregy.java | 42 + .../CommandMBeanExecutionStrategy.java | 162 ++++ .../strategy/CqlCommandExecutionStrategy.java | 230 +++++ .../ProtocolAwareExecutionStrategy.java | 135 +++ .../StaticMBeanExecutionStrategy.java | 77 ++ .../org/apache/cassandra/utils/JsonUtils.java | 47 + ....cassandra.management.api.CommandsProvider | 20 + .../cassandra/distributed/impl/Instance.java | 2 + .../mock/nodetool/InternalNodeProbe.java | 116 ++- .../JmxManagementDistributedTest.java | 89 -- .../cql3/CQLNodetoolProtocolTester.java | 64 ++ .../org/apache/cassandra/cql3/CQLTester.java | 17 + .../management/CommandServiceTest.java | 107 ++ .../management/CqlExecuteCommandTest.java | 77 ++ .../apache/cassandra/tools/NodeProbeTest.java | 6 +- .../apache/cassandra/tools/ToolRunner.java | 14 +- .../tools/nodetool/ClearSnapshotTest.java | 6 +- .../cassandra/tools/nodetool/CompactTest.java | 10 +- .../tools/nodetool/ForceCompactionTest.java | 13 +- .../cassandra/stress/util/JmxCollector.java | 17 +- 79 files changed, 5703 insertions(+), 1270 deletions(-) create mode 100644 src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java create mode 100644 src/java/org/apache/cassandra/management/CassandraCommandRegistry.java create mode 100644 src/java/org/apache/cassandra/management/CommandExecutionArgsSerde.java create mode 100644 src/java/org/apache/cassandra/management/CommandInvoker.java create mode 100644 src/java/org/apache/cassandra/management/CommandMBean.java create mode 100644 src/java/org/apache/cassandra/management/CommandService.java create mode 100644 src/java/org/apache/cassandra/management/CommandServiceMBean.java create mode 100644 src/java/org/apache/cassandra/management/InternalNodeMBeanAccessor.java create mode 100644 src/java/org/apache/cassandra/management/MBeanAccessor.java create mode 100644 src/java/org/apache/cassandra/management/ManagementUtils.java create mode 100644 src/java/org/apache/cassandra/management/SimpleCommandExecutionArgs.java rename src/java/org/apache/cassandra/{tools/nodetool/ManagementContext.java => management/api/ArgumentMetadata.java} (70%) create mode 100644 src/java/org/apache/cassandra/management/api/CommandExecutionArgs.java rename src/java/org/apache/cassandra/management/api/{CommandContext.java => CommandExecutionContext.java} (89%) delete mode 100644 src/java/org/apache/cassandra/management/api/CommandInvoker.java delete mode 100644 src/java/org/apache/cassandra/management/api/CommandMBean.java create mode 100644 src/java/org/apache/cassandra/management/api/CommandMetadata.java create mode 100644 src/java/org/apache/cassandra/management/api/CommandRegistry.java rename src/java/org/apache/cassandra/management/api/{LocalCommand.java => CommandsProvider.java} (77%) rename src/java/org/apache/cassandra/management/api/{CommandsRegistry.java => OptionMetadata.java} (80%) create mode 100644 src/java/org/apache/cassandra/management/api/ParameterMetadata.java create mode 100644 src/java/org/apache/cassandra/management/picocli/PicocliCommandAdapter.java create mode 100644 src/java/org/apache/cassandra/management/picocli/PicocliCommandArgsConverter.java create mode 100644 src/java/org/apache/cassandra/management/picocli/PicocliCommandMetadata.java create mode 100644 src/java/org/apache/cassandra/management/picocli/PicocliCommandRegistryAdapter.java create mode 100644 src/java/org/apache/cassandra/management/picocli/PicocliCommandsProvider.java create mode 100644 src/java/org/apache/cassandra/management/picocli/PicocliMetadataExtractor.java create mode 100644 src/java/org/apache/cassandra/management/picocli/PicocliOptionMetadata.java create mode 100644 src/java/org/apache/cassandra/management/picocli/PicocliParameterMetadata.java create mode 100644 src/java/org/apache/cassandra/management/picocli/TypeConverter.java create mode 100644 src/java/org/apache/cassandra/management/picocli/TypeConverterRegistry.java create mode 100644 src/java/org/apache/cassandra/tools/RemoteJmxMBeanAccessor.java create mode 100644 src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java create mode 100644 src/java/org/apache/cassandra/tools/nodetool/strategy/CommandExecutionStraregy.java create mode 100644 src/java/org/apache/cassandra/tools/nodetool/strategy/CommandMBeanExecutionStrategy.java create mode 100644 src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java create mode 100644 src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java create mode 100644 src/java/org/apache/cassandra/tools/nodetool/strategy/StaticMBeanExecutionStrategy.java create mode 100644 src/resources/META-INF/services/org.apache.cassandra.management.api.CommandsProvider delete mode 100644 test/distributed/org/apache/cassandra/distributed/test/management/JmxManagementDistributedTest.java create mode 100644 test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java create mode 100644 test/unit/org/apache/cassandra/management/CommandServiceTest.java create mode 100644 test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java diff --git a/src/antlr/Lexer.g b/src/antlr/Lexer.g index c7065f7351b5..4c62e0c4b450 100644 --- a/src/antlr/Lexer.g +++ b/src/antlr/Lexer.g @@ -148,6 +148,7 @@ K_MODIFY: M O D I F Y; K_AUTHORIZE: A U T H O R I Z E; K_DESCRIBE: D E S C R I B E; K_EXECUTE: E X E C U T E; +K_COMMAND: C O M M A N D; K_NORECURSIVE: N O R E C U R S I V E; K_MBEAN: M B E A N; K_MBEANS: M B E A N S; diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g index 8794d00d02a9..b380da2d46f9 100644 --- a/src/antlr/Parser.g +++ b/src/antlr/Parser.g @@ -294,6 +294,7 @@ cqlStatement returns [CQLStatement.Raw stmt] | st55=securityLabelOnUserTypeStatement { $stmt = st55; } | st56=commentOnUserTypeFieldStatement { $stmt = st56; } | st57=securityLabelOnUserTypeFieldStatement { $stmt = st57; } + | st58=executeCommandStatement { $stmt = st58; } ; /* @@ -1370,6 +1371,44 @@ securityLabelOnUserTypeFieldStatement returns [SecurityLabelOnUserTypeFieldState { $stmt = new SecurityLabelOnUserTypeFieldStatement.Raw($typeFieldRef.typeName, $typeFieldRef.field, label != null ? $label.text : null, provider); } ; +/** + * COMMAND WITH key1 = value1 AND key2 = value2; + */ +executeCommandStatement returns [ExecuteCommandStatement.Raw stmt] + @init { + stmtBegins(); + java.util.Map args = new java.util.LinkedHashMap<>(); + } + : K_COMMAND cmdName=noncol_ident + K_WITH commandProperties[args] + { + $stmt = new ExecuteCommandStatement.Raw(cmdName.toString(), args); + } + ; + +commandProperties[java.util.Map args] + : commandProperty[args] (K_AND commandProperty[args])* + ; + +commandProperty[java.util.Map args] + : k=noncol_ident '=' v=commandPropertyValue + { + String key = k.toString(); + Object value = v; + if (args.put(key, value) != null) + { + addRecognitionError("Duplicate argument: " + key); + } + } + ; + +commandPropertyValue returns [Object value] + : s=STRING_LITERAL { $value = $s.text; } + | i=INTEGER { $value = $i.text; } + | b=BOOLEAN { $value = $b.text; } + | l=listLiteral { $value = $l.value; } + ; + /** * DROP TABLE [IF EXISTS] ; */ @@ -1980,7 +2019,7 @@ collectionLiteral returns [Term.Raw value] listLiteral returns [Term.Raw value] @init {List l = new ArrayList();} - @after {$value = new ArrayLiteral(l);} +@after {$value = new ArrayLiteral(l);} : '[' ( t1=term { l.add(t1); } ( ',' tn=term { l.add(tn); } )* )? ']' { $value = new ArrayLiteral(l); } ; @@ -2481,5 +2520,6 @@ basic_unreserved_keyword returns [String str] | K_LABELS | K_FIELD | K_COLUMN + | K_COMMAND ) { $str = $k.text; } ; diff --git a/src/java/org/apache/cassandra/audit/AuditLogEntryType.java b/src/java/org/apache/cassandra/audit/AuditLogEntryType.java index ff61f2f2a757..68a6d65f2b89 100644 --- a/src/java/org/apache/cassandra/audit/AuditLogEntryType.java +++ b/src/java/org/apache/cassandra/audit/AuditLogEntryType.java @@ -82,7 +82,8 @@ public enum AuditLogEntryType UNAUTHORIZED_ATTEMPT(AuditLogEntryCategory.AUTH), LOGIN_SUCCESS(AuditLogEntryCategory.AUTH), LIST_SUPERUSERS(AuditLogEntryCategory.DCL), - JMX(AuditLogEntryCategory.JMX); + JMX(AuditLogEntryCategory.JMX), + EXECUTE_COMMAND(AuditLogEntryCategory.OTHER); private final AuditLogEntryCategory category; diff --git a/src/java/org/apache/cassandra/auth/AbstractCIDRAuthorizer.java b/src/java/org/apache/cassandra/auth/AbstractCIDRAuthorizer.java index a2d591318edc..d3de6bd29134 100644 --- a/src/java/org/apache/cassandra/auth/AbstractCIDRAuthorizer.java +++ b/src/java/org/apache/cassandra/auth/AbstractCIDRAuthorizer.java @@ -28,8 +28,8 @@ */ public abstract class AbstractCIDRAuthorizer implements ICIDRAuthorizer { - protected static CIDRPermissionsManager cidrPermissionsManager; - protected static CIDRGroupsMappingManager cidrGroupsMappingManager; + public static CIDRPermissionsManager cidrPermissionsManager; + public static CIDRGroupsMappingManager cidrGroupsMappingManager; protected static CIDRAuthorizerMetrics cidrAuthorizerMetrics; diff --git a/src/java/org/apache/cassandra/auth/AuthCache.java b/src/java/org/apache/cassandra/auth/AuthCache.java index 8c9df378fcb6..0375ef91d923 100644 --- a/src/java/org/apache/cassandra/auth/AuthCache.java +++ b/src/java/org/apache/cassandra/auth/AuthCache.java @@ -45,6 +45,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.auth.jmx.AuthorizationProxy; import org.apache.cassandra.cache.UnweightedCacheSize; import org.apache.cassandra.concurrent.ExecutorPlus; import org.apache.cassandra.concurrent.ScheduledExecutors; @@ -468,6 +469,36 @@ public int entries() return Ints.checkedCast(getEstimatedSize()); } + /** + * Accepts a visitor for this cache instance. Subclasses should override this method to dispatch + * to the appropriate visitor method based on the specific MBean interface they implement. + * @param visitor the visitor to accept + */ + public void accept(MBeanVisitor visitor) + { + visitor.visit(this); + } + + /** + * Visitor interface for processing auth cache MBeans. + * Allows type-safe iteration over different cache types without instanceof checks. + */ + public interface MBeanVisitor + { + /** Visits a credentials cache MBean. */ + default void visitCredentials(PasswordAuthenticator.CredentialsCacheMBean cache) {} + /** Visits a JMX permissions cache MBean. */ + default void visitJmxPermissions(AuthorizationProxy.JmxPermissionsCacheMBean cache) {} + /** Visits a permissions cache MBean. */ + default void visitPermissions(PermissionsCacheMBean cache) {} + /** Visits a network permissions cache MBean. */ + default void visitNetwork(NetworkPermissionsCacheMBean cache) {} + /** Visits a roles cache MBean. */ + default void visitRoles(RolesCacheMBean cache) {} + /** Visits a generic auth cache (fallback for caches that don't implement specific MBean interfaces).*/ + default void visit(AuthCacheMBean cache) {} + } + private class MetricsUpdater implements StatsCounter { @Override diff --git a/src/java/org/apache/cassandra/auth/AuthCacheService.java b/src/java/org/apache/cassandra/auth/AuthCacheService.java index 8fddd1e0e922..13c08e3ba26d 100644 --- a/src/java/org/apache/cassandra/auth/AuthCacheService.java +++ b/src/java/org/apache/cassandra/auth/AuthCacheService.java @@ -26,6 +26,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,6 +61,11 @@ public synchronized void warmCaches() } } + public synchronized Set> getCaches() + { + return Sets.newHashSet(caches); + } + /** * NOTE: Can only be called once per instance run. * diff --git a/src/java/org/apache/cassandra/auth/NetworkPermissionsCache.java b/src/java/org/apache/cassandra/auth/NetworkPermissionsCache.java index 1c18fed7a74f..73a134e7cdee 100644 --- a/src/java/org/apache/cassandra/auth/NetworkPermissionsCache.java +++ b/src/java/org/apache/cassandra/auth/NetworkPermissionsCache.java @@ -52,4 +52,10 @@ protected void unregisterMBean() super.unregisterMBean(); MBeanWrapper.instance.unregisterMBean(MBEAN_NAME_BASE + DEPRECATED_CACHE_NAME, MBeanWrapper.OnException.LOG); } + + @Override + public void accept(MBeanVisitor visitor) + { + visitor.visitNetwork(this); + } } diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java index 82c8f24c6ffc..fae5421a9912 100644 --- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java +++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java @@ -351,6 +351,12 @@ public void invalidateCredentials(String roleName) { invalidate(roleName); } + + @Override + public void accept(MBeanVisitor visitor) + { + visitor.visitCredentials(this); + } } public static interface CredentialsCacheMBean extends AuthCacheMBean diff --git a/src/java/org/apache/cassandra/auth/PermissionsCache.java b/src/java/org/apache/cassandra/auth/PermissionsCache.java index 0757b5e3bfbe..099b1f0dbb66 100644 --- a/src/java/org/apache/cassandra/auth/PermissionsCache.java +++ b/src/java/org/apache/cassandra/auth/PermissionsCache.java @@ -50,4 +50,10 @@ public void invalidatePermissions(String roleName, String resourceName) { invalidate(Pair.create(new AuthenticatedUser(roleName), Resources.fromName(resourceName))); } + + @Override + public void accept(MBeanVisitor visitor) + { + visitor.visitPermissions(this); + } } diff --git a/src/java/org/apache/cassandra/auth/RolesCache.java b/src/java/org/apache/cassandra/auth/RolesCache.java index d34de1928c9d..8ade811b9878 100644 --- a/src/java/org/apache/cassandra/auth/RolesCache.java +++ b/src/java/org/apache/cassandra/auth/RolesCache.java @@ -82,4 +82,10 @@ public void invalidateRoles(String roleName) { invalidate(RoleResource.role(roleName)); } + + @Override + public void accept(MBeanVisitor visitor) + { + visitor.visitRoles(this); + } } diff --git a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java index 183050b4c467..9d28e1a84d39 100644 --- a/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java +++ b/src/java/org/apache/cassandra/auth/jmx/AuthorizationProxy.java @@ -609,6 +609,12 @@ protected void unregisterMBean() super.unregisterMBean(); MBeanWrapper.instance.unregisterMBean(MBEAN_NAME_BASE + DEPRECATED_CACHE_NAME, MBeanWrapper.OnException.LOG); } + + @Override + public void accept(MBeanVisitor visitor) + { + visitor.visitJmxPermissions(this); + } } public static interface JmxPermissionsCacheMBean extends AuthCacheMBean diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantEnv.java b/src/java/org/apache/cassandra/config/CassandraRelevantEnv.java index e563c5917640..f85961921ce6 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantEnv.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantEnv.java @@ -36,6 +36,12 @@ public enum CassandraRelevantEnv JAVA_HOME ("JAVA_HOME"), CIRCLECI("CIRCLECI"), CASSANDRA_SKIP_SYNC("CASSANDRA_SKIP_SYNC"), + /** + * Defines the protocol used by the Cassandra CLI to connect to Cassandra nodes. + * Possible values are {@code "static_mbeans"}, {@code "command_mbeans"} or {@code cql}. + * By default, the Cassandra CLI uses the JMX protocol via static MBeans. + */ + CASSANDRA_CLI_EXECUTION_PROTOCOL("CASSANDRA_CLI_EXECUTION_PROTOCOL"), /** By default, the standard Cassandra CLI layout is used for backward compatibility, however, * the new Picocli layout can be enabled by setting this property to the {@code "picocli"}. */ CASSANDRA_CLI_LAYOUT("CASSANDRA_CLI_LAYOUT"), diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index ca79ff372772..790c44f2c607 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -90,6 +90,12 @@ public enum CassandraRelevantProperties CACHEABLE_MUTATION_SIZE_LIMIT("cassandra.cacheable_mutation_size_limit_bytes", convertToString(1_000_000)), CASSANDRA_ALLOW_SIMPLE_STRATEGY("cassandra.allow_simplestrategy"), CASSANDRA_AVAILABLE_PROCESSORS("cassandra.available_processors"), + /** + * Defines the protocol used by the Cassandra CLI to connect to Cassandra nodes. + * Possible values are {@code "static_mbeans"}, {@code "command_mbeans"} or {@code cql}. + * By default, the Cassandra CLI uses the JMX protocol via static MBeans. + */ + CASSANDRA_CLI_EXECUTION_PROTOCOL("cassandra.cli.execution.protocol", "static_mbean"), /** By default, the standard Cassandra CLI layout is used for backward compatibility, however, * the new Picocli layout can be enabled by setting this property to the {@code "picocli"}. */ CASSANDRA_CLI_LAYOUT("cassandra.cli.layout", "airline"), diff --git a/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java b/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java new file mode 100644 index 000000000000..28c009ca0b83 --- /dev/null +++ b/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java @@ -0,0 +1,220 @@ +/* + * 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.cassandra.cql3.statements; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import org.apache.cassandra.audit.AuditLogContext; +import org.apache.cassandra.audit.AuditLogEntryType; +import org.apache.cassandra.cql3.CQLStatement; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.ColumnSpecification; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.ResultSet; +import org.apache.cassandra.cql3.terms.ArrayLiteral; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.exceptions.CommandExecutionException; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.management.CommandExecutionArgsSerde; +import org.apache.cassandra.management.CommandService; +import org.apache.cassandra.management.api.Command; +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.transport.Dispatcher; +import org.apache.cassandra.transport.messages.ResultMessage; + +import static org.apache.cassandra.utils.ByteBufferUtil.bytes; + +public class ExecuteCommandStatement +{ + public static class Raw extends CQLStatement.Raw implements CQLStatement + { + private final String commandName; + private final Map args; + + public Raw(String commandName, Map args) + { + this.commandName = commandName; + this.args = args; + } + + public CQLStatement prepare(ClientState state) + { + return this; + } + + public void authorize(ClientState state) throws UnauthorizedException + { + state.validateLogin(); + } + + @Override + public void validate(ClientState state) throws InvalidRequestException + { + Command command = CommandService.instance.getRegistry().command(commandName); + if (command == null) + throw new InvalidRequestException("Command not found: " + commandName); + } + + @Override + public ResultMessage execute(QueryState state, QueryOptions options, Dispatcher.RequestTime requestTime) throws InvalidRequestException + { + try + { + Command command = CommandService.instance.getRegistry().command(commandName); + if (command == null) + throw new InvalidRequestException("Command not found: " + commandName); + + Map paramsMap = convertArgsMap(args); + + CommandExecutionArgs commandArgs = CommandExecutionArgsSerde.fromMap(paramsMap, command.metadata()); + + String output = CommandService.instance.executeCommand(commandName, commandArgs); + ColumnSpecification outputColumn = new ColumnSpecification("system", + "command_output", + new ColumnIdentifier("output", true), + UTF8Type.instance); + + ResultSet resultSet = new ResultSet(new ResultSet.ResultMetadata(List.of(outputColumn))); + resultSet.addColumnValue(bytes(output)); + return new ResultMessage.Rows(resultSet); + } + catch (CommandExecutionException e) + { + throw new InvalidRequestException("Failed to execute command '" + commandName + "': " + e.getMessage(), e); + } + catch (MarshalException e) + { + throw new InvalidRequestException("Invalid JSON format for command arguments: " + e.getMessage(), e); + } + catch (IllegalArgumentException e) + { + throw new InvalidRequestException("Invalid command arguments: " + e.getMessage(), e); + } + } + + /** Parse a CQL list literal string like "['k1', 'k2']" into a List of strings, and unquote CQL string literals. */ + private static Map convertArgsMap(Map args) + { + Map result = new LinkedHashMap<>(); + for (Map.Entry entry : args.entrySet()) + { + Object value = entry.getValue(); + + if (value instanceof ArrayLiteral) + result.put(entry.getKey(), parseCqlListLiteral(((ArrayLiteral) value).getText())); + else if (value instanceof String) + result.put(entry.getKey(), unquoteCqlString((String) value)); + else + throw new InvalidRequestException("Invalid argument type: " + value.getClass()); + } + return result; + } + + /** Unquote a CQL string literal, handling escaped quotes: 'It''s' -> It's. */ + private static String unquoteCqlString(String str) + { + String trimmed = str.trim(); + if (trimmed.length() >= 2 && trimmed.startsWith("'") && trimmed.endsWith("'")) + { + String unquoted = trimmed.substring(1, trimmed.length() - 1); + return unquoted.replace("''", "'"); + } + return trimmed; + } + + private static List parseCqlListLiteral(String listLiteral) + { + List list = new ArrayList<>(); + String content = listLiteral.substring(1, listLiteral.length() - 1).trim(); + if (content.isEmpty()) + return list; + + for (int i = 0; i < content.length(); ) + { + i = skipWhitespace(content, i); + if (i >= content.length()) + break; + + // Parse quoted string (CQL list elements should always be quoted) + if (content.charAt(i) != '\'') + throw new IllegalArgumentException("Expected quoted string in list literal: " + listLiteral); + + int end = findClosingQuote(content, i, listLiteral); + String quoted = content.substring(i, end + 1); + list.add(unquoteCqlString(quoted)); + i = end + 1; + + i = skipCommaAndWhitespace(content, i); + } + + return list; + } + + private static int skipWhitespace(String content, int start) + { + while (start < content.length() && Character.isWhitespace(content.charAt(start))) + start++; + return start; + } + + private static int skipCommaAndWhitespace(String content, int start) + { + while (start < content.length() && (content.charAt(start) == ',' || Character.isWhitespace(content.charAt(start)))) + start++; + return start; + } + + private static int findClosingQuote(String content, int start, String listLiteral) + { + // Skip the opening quote. + for (int i = start + 1; i < content.length(); ) + { + if (content.charAt(i) == '\'') + { + if (i + 1 < content.length() && content.charAt(i + 1) == '\'') + i += 2; + else + return i; + } + else + { + i++; + } + } + throw new IllegalArgumentException("Unclosed string in list literal: " + listLiteral); + } + + public ResultMessage executeLocally(QueryState state, QueryOptions options) throws InvalidRequestException + { + return execute(state, options, Dispatcher.RequestTime.forImmediateExecution()); + } + + @Override + public AuditLogContext getAuditLogContext() + { + return new AuditLogContext(AuditLogEntryType.EXECUTE_COMMAND, commandName); + } + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/CIDRFilteringMetricsTable.java b/src/java/org/apache/cassandra/db/virtual/CIDRFilteringMetricsTable.java index 795b44ca060e..8203320fd9d3 100644 --- a/src/java/org/apache/cassandra/db/virtual/CIDRFilteringMetricsTable.java +++ b/src/java/org/apache/cassandra/db/virtual/CIDRFilteringMetricsTable.java @@ -56,7 +56,7 @@ public class CIDRFilteringMetricsTable implements CIDRFilteringMetricsTableMBean { public static final String MBEAN_NAME = "org.apache.cassandra.db:type=CIDRFilteringMetricsTable"; - private static final CIDRFilteringMetricsTable instance = new CIDRFilteringMetricsTable(); + public static final CIDRFilteringMetricsTable instance = new CIDRFilteringMetricsTable(); CIDRFilteringMetricsTable() { diff --git a/src/java/org/apache/cassandra/management/CassandraCommandRegistry.java b/src/java/org/apache/cassandra/management/CassandraCommandRegistry.java new file mode 100644 index 000000000000..23b2a6febcd9 --- /dev/null +++ b/src/java/org/apache/cassandra/management/CassandraCommandRegistry.java @@ -0,0 +1,123 @@ +/* + * 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.cassandra.management; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import org.apache.cassandra.management.api.Command; +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.api.CommandExecutionContext; +import org.apache.cassandra.management.api.CommandMetadata; +import org.apache.cassandra.management.api.CommandRegistry; +import org.apache.cassandra.management.api.CommandsProvider; +import org.apache.cassandra.management.api.OptionMetadata; +import org.apache.cassandra.management.api.ParameterMetadata; + +import static org.apache.cassandra.management.ManagementUtils.loadService; + +public class CassandraCommandRegistry implements CommandRegistry +{ + private final Map> commandMap = new ConcurrentHashMap<>(); + + public CassandraCommandRegistry() + { + for (CommandsProvider provider : loadService(CommandsProvider.class)) + provider.commands().forEach(this::register); + } + + public void register(Command command) + { + commandMap.putIfAbsent(command.name(), command); + } + + @Override + public Command command(String name) + { + return commandMap.get(name); + } + + @Override + public Iterable>> commands() + { + return commandMap.entrySet(); + } + + @Override + public CommandMetadata metadata() + { + // Return metadata that reflects this is a registry, not a leaf command + return new RootCommandMetadata(); + } + + @Override + public String execute(CommandExecutionArgs arguments, CommandExecutionContext context) + { + // CommandRegistry is not directly executable: routing happens at the invoker/CQL layer. + throw new UnsupportedOperationException( + String.format("CommandRegistry '%s' is not directly executable. " + + "Specify a subcommand. Available commands: %s", + name(), + String.join(", ", commandMap.keySet()))); + } + + private class RootCommandMetadata implements CommandMetadata + { + @Override + public String name() + { + return "root"; + } + + @Override + public String description() + { + return "Root command registry - use a specific subcommand"; + } + + @Override + public List aliases() + { + return Collections.emptyList(); + } + + @Override + public List options() + { + return Collections.emptyList(); + } + + @Override + public List parameters() + { + return Collections.emptyList(); + } + + @Override + public List subcommands() + { + return commandMap.values().stream() + .map(Command::metadata) + .collect(Collectors.toList()); + } + } +} diff --git a/src/java/org/apache/cassandra/management/CommandExecutionArgsSerde.java b/src/java/org/apache/cassandra/management/CommandExecutionArgsSerde.java new file mode 100644 index 000000000000..bfb54a42fc9c --- /dev/null +++ b/src/java/org/apache/cassandra/management/CommandExecutionArgsSerde.java @@ -0,0 +1,260 @@ +/* + * 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.cassandra.management; + +import java.lang.reflect.Array; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.stream.Collectors; + +import org.apache.cassandra.management.api.ArgumentMetadata; +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.api.CommandMetadata; +import org.apache.cassandra.management.api.OptionMetadata; +import org.apache.cassandra.management.api.ParameterMetadata; +import org.apache.cassandra.management.picocli.PicocliCommandMetadata; +import org.apache.cassandra.management.picocli.PicocliOptionMetadata; +import org.apache.cassandra.management.picocli.PicocliParameterMetadata; +import org.apache.cassandra.management.picocli.TypeConverterRegistry; +import org.apache.cassandra.utils.JsonUtils; + +import static org.apache.cassandra.management.api.ParameterMetadata.COMMAND_POSITIONAL_PARAM_PREFIX; + +public class CommandExecutionArgsSerde +{ + public static String toJson(CommandExecutionArgs args) + { + Map params = new LinkedHashMap<>(); + + for (Map.Entry entry : args.options().entrySet()) + { + OptionMetadata optionMetadata = entry.getKey(); + Object value = entry.getValue(); + + if (value == null) + continue; + + String primaryName = normalizeOptionName(optionMetadata.paramLabel()); + params.put(primaryName, normalizeJsonValue(value)); + } + + for (Map.Entry entry : args.parameters().entrySet()) + { + ParameterMetadata paramMetadata = entry.getKey(); + Object value = entry.getValue(); + + if (value == null) + continue; + + int index = paramMetadata.index(); + String indexKey = COMMAND_POSITIONAL_PARAM_PREFIX + index; + params.put(indexKey, normalizeJsonValue(value)); + } + + return JsonUtils.writeAsJsonString(params); + } + + public static CommandExecutionArgs fromJson(String json, CommandMetadata metadata) + { + if (!(metadata instanceof PicocliCommandMetadata)) + throw new IllegalArgumentException("CommandMetadata must be PicocliCommandMetadata for picocli commands"); + + Map jsonMap = JsonUtils.fromJsonMap(json); + Map options = new LinkedHashMap<>(); + Map parameters = new LinkedHashMap<>(); + + convertFromMap(jsonMap, metadata, options, parameters); + return new SimpleCommandExecutionArgs(options, parameters); + } + + public static CommandExecutionArgs fromMap(Map format, CommandMetadata metadata) + { + if (!(metadata instanceof PicocliCommandMetadata)) + throw new IllegalArgumentException("CommandMetadata must be PicocliCommandMetadata for picocli commands"); + + Map options = new LinkedHashMap<>(); + Map parameters = new LinkedHashMap<>(); + + convertFromMap(format, metadata, options, parameters); + return new SimpleCommandExecutionArgs(options, parameters); + } + + private static void convertFromMap(Map source, + CommandMetadata metadata, + Map options, + Map parameters) + { + for (Map.Entry entry : source.entrySet()) + { + String paramName = entry.getKey(); + Object value = entry.getValue(); + + if (value == null) + continue; + + OptionMetadata option = findOptionByNameIgnoreCase(metadata, paramName); + if (option != null) + { + Object convertedValue = convertValue(value, option); + options.put(option, convertedValue); + continue; + } + + ParameterMetadata param = findParameterByName(metadata, paramName); + if (param != null) + { + Object convertedValue = convertValue(value, param); + parameters.put(param, convertedValue); + } + else + { + throw new IllegalArgumentException("Unknown parameter: " + paramName); + } + } + } + + /** Convert value using custom converter if provided, otherwise use basic conversion. */ + private static Object convertValue(Object value, ArgumentMetadata argSpec) + { + try + { + // Custom type converters are only supported for picocli-based metadata. + if (argSpec instanceof PicocliParameterMetadata) + return ((PicocliParameterMetadata) argSpec).convertValue(value); + else if (argSpec instanceof PicocliOptionMetadata) + return ((PicocliOptionMetadata) argSpec).convertValue(value); + + return TypeConverterRegistry.convertValueBasic(value, argSpec.type()); + } + catch (IllegalArgumentException e) + { + throw e; + } + catch (Exception e) + { + throw new IllegalArgumentException(String.format("Failed to convert value '%s' to type %s: %s", + value, + argSpec.type().getName(), + e.getMessage()), e); + } + } + + /** + * This method converts values from CommandExecutionArgs into JSON-serializable formats before + * they're put into a Map(String, Object) that will be serialized to JSON. Different implementations + * (e.g., Set, LinkedHashSet) should be normalized to List for consistent JSON output. + *

+ * The conversion rules are: + * - Converts primitive arrays (String[], int[], etc.) -> Object[] (JSON-serializable) + * - Converts any Collection -> List (JSON-serializable) + * - Leaves other types unchanged + * + * @param value the value to convert. + * @return the converted value. + */ + private static Object normalizeJsonValue(Object value) + { + if (value == null) + return null; + + if (value.getClass().isArray()) + { + int length = Array.getLength(value); + Object[] array = new Object[length]; + for (int i = 0; i < length; i++) + array[i] = normalizeJsonValue(Array.get(value, i)); + return array; + } + + if (value instanceof java.util.Collection) + { + return ((java.util.Collection) value).stream() + .map(CommandExecutionArgsSerde::normalizeJsonValue) + .collect(Collectors.toList()); + } + + return value; + } + + /** + * Find ParameterMetadata by name (supports both an index format and paramLabel). + */ + private static ParameterMetadata findParameterByName(CommandMetadata metadata, String name) + { + if (name.startsWith(COMMAND_POSITIONAL_PARAM_PREFIX)) + { + try + { + int index = Integer.parseInt(name.substring(COMMAND_POSITIONAL_PARAM_PREFIX.length())); + for (ParameterMetadata param : metadata.parameters()) + { + if (param.index() == index) + return param; + } + } + catch (NumberFormatException e) + { + // Not a valid index format, continue to check paramLabel + } + } + + for (ParameterMetadata param : metadata.parameters()) + { + String paramLabel = param.paramLabel(); + if (paramLabel != null && paramLabel.equalsIgnoreCase(name)) + return param; + } + + return null; + } + + /** + * Find OptionMetadata by name (case-insensitive, normalized). + * Matches against paramLabel and all names/aliases. + */ + private static OptionMetadata findOptionByNameIgnoreCase(CommandMetadata metadata, String name) + { + String normalizedName = normalizeOptionName(name); + + for (OptionMetadata option : metadata.options()) + { + String paramLabel = normalizeOptionName(option.paramLabel()); + if (paramLabel.equalsIgnoreCase(normalizedName)) + return option; + + for (String alias : option.names()) + { + String normalizedAlias = normalizeOptionName(alias); + if (normalizedAlias.equalsIgnoreCase(normalizedName)) + return option; + } + } + + return null; + } + + private static String normalizeOptionName(String name) + { + if (name.startsWith("--")) + return name.substring(2); + else if (name.startsWith("-")) + return name.substring(1); + return name; + } +} diff --git a/src/java/org/apache/cassandra/management/CommandInvoker.java b/src/java/org/apache/cassandra/management/CommandInvoker.java new file mode 100644 index 000000000000..c522f2c84665 --- /dev/null +++ b/src/java/org/apache/cassandra/management/CommandInvoker.java @@ -0,0 +1,107 @@ +/* + * 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.cassandra.management; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.exceptions.CommandExecutionException; +import org.apache.cassandra.management.api.Command; +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.api.CommandExecutionContext; +import org.apache.cassandra.management.api.CommandMetadata; +import org.apache.cassandra.management.api.OptionMetadata; +import org.apache.cassandra.management.api.ParameterMetadata; + +/** + * Default implementation of CommandInvoker. + * + *

This invoker: + *

    + *
  • Validates command arguments against metadata
  • + *
  • Executes the command via Command.execute()
  • + *
  • Handles exceptions and provides error context
  • + *
  • Can be extended for cross-cutting concerns (logging, metrics, etc.)
  • + *
+ * + *

The invoker is stateless and can be reused for multiple commands. + */ +public class CommandInvoker +{ + private static final Logger logger = LoggerFactory.getLogger(CommandInvoker.class); + + private final Command command; + private final CommandExecutionArgs arguments; + private final CommandExecutionContext executionContext; + + public CommandInvoker(Command command, CommandExecutionArgs arguments, CommandExecutionContext executionContext) + { + if (command == null) + throw new IllegalArgumentException("Command cannot be null"); + if (arguments == null) + throw new IllegalArgumentException("CommandExecutionArgs cannot be null"); + if (executionContext == null) + throw new IllegalArgumentException("CommandExecutionContext cannot be null"); + + this.command = command; + this.arguments = arguments; + this.executionContext = executionContext; + } + + public R invoke() + { + CommandMetadata metadata = command.metadata(); + String commandName = metadata.name(); + + try + { + validateArguments(arguments, metadata); + return command.execute(arguments, executionContext); + } + catch (IllegalArgumentException e) + { + logger.warn("Invalid arguments for command '{}': {}", commandName, e.getMessage()); + throw new CommandExecutionException( + String.format("Invalid arguments for command '%s': %s", commandName, e.getMessage()), e); + } + catch (Exception e) + { + logger.error("Error executing command '{}'", commandName, e); + throw new CommandExecutionException( + String.format("Failed to execute command '%s': %s", commandName, e.getMessage()), e); + } + } + + /** Validate command arguments against metadata. */ + protected void validateArguments(CommandExecutionArgs arguments, CommandMetadata metadata) + { + for (OptionMetadata option : metadata.options()) + { + if (option.required() && !arguments.hasOption(option)) + throw new IllegalArgumentException(String.format("Required option '%s' is missing", option.paramLabel())); + } + + for (ParameterMetadata param : metadata.parameters()) + { + if (param.required() && !arguments.hasParameter(param)) + throw new IllegalArgumentException(String.format("Required parameter at index %d ('%s') is missing", + param.index(), param.paramLabel())); + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/management/CommandMBean.java b/src/java/org/apache/cassandra/management/CommandMBean.java new file mode 100644 index 000000000000..acf7101eda29 --- /dev/null +++ b/src/java/org/apache/cassandra/management/CommandMBean.java @@ -0,0 +1,344 @@ +/* + * 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.cassandra.management; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.stream.Collectors; + +import javax.management.Attribute; +import javax.management.AttributeList; +import javax.management.AttributeNotFoundException; +import javax.management.DynamicMBean; +import javax.management.InvalidAttributeValueException; +import javax.management.MBeanException; +import javax.management.MBeanInfo; +import javax.management.MBeanOperationInfo; +import javax.management.MBeanParameterInfo; +import javax.management.ReflectionException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.management.api.ArgumentMetadata; +import org.apache.cassandra.management.api.Command; +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.api.CommandMetadata; +import org.apache.cassandra.management.api.OptionMetadata; +import org.apache.cassandra.management.api.ParameterMetadata; +import org.apache.cassandra.serializers.MarshalException; +import org.apache.cassandra.utils.JsonUtils; + +import static javax.management.MBeanOperationInfo.ACTION; +import static javax.management.MBeanOperationInfo.INFO; +import static org.apache.cassandra.management.api.ParameterMetadata.COMMAND_POSITIONAL_PARAM_PREFIX; +import static org.apache.cassandra.utils.JsonUtils.convertDefaultValue; +import static org.apache.cassandra.utils.JsonUtils.getJsonType; + +/** + * Command MBean exposes a single management command to the JMX interface. + * + *

Uses JSON-based parameter format: + *

    + *
  • Single "invoke" operation with JSON string parameter
  • + *
  • JSON format: {"optionName": "value", "param0": "value", ...}
  • + *
  • Option names: use option name or any alias (e.g., "concurrent-compactors", "--concurrent-compactors")
  • + *
  • Positional parameters: use "param0", "param1", etc. or parameter name
  • + *
  • Returns command output as String
  • + *
+ * + *

+ * Invocation: + *

+ * // Command: setconcurrentcompactors --concurrent-compactors 4
+ * mbean.invoke("invoke",
+ *     new Object[]{"{\"concurrent-compactors\": \"4\"}"},
+ *     new String[]{ "String" });
+ *
+ * // Command: getendpoints keyspace table
+ * mbean.invoke("invoke",
+ *     new Object[]{"{\"param0\": \"mykeyspace\", \"param1\": \"mytable\"}"},
+ *     new String[]{ "String" });
+ * 
+ * + *

+ * Alternatively, the parameters can be passed as name-value pairs: + *

+ * // Command: setconcurrentcompactors --concurrent-compactors 4
+ * mbean.invoke("invoke",
+ *     new Object[]{"concurrent-compactors", "4"},
+ *     new String[]{"String", "String"});
+ *
+ * // Command: getendpoints keyspace table
+ * mbean.invoke("invoke",
+ *     new Object[]{"param0", "mykeyspace", "param1", "mytable"},
+ *     new String[]{"String", "String", "String", "String"});
+ * 
+ */ +public class CommandMBean implements DynamicMBean +{ + public static final String INVOKE_METHOD = "invoke"; + public static final String GET_JSON_SCHEMA_METHOD = "getJsonSchema"; + public static final String GET_PARAMETER_INFO_METHOD = "getParameterInfo"; + + private static final Logger logger = LoggerFactory.getLogger(CommandMBean.class); + + private final Command command; + private final CommandMetadata metadata; + private final BiFunction executor; + + public CommandMBean(Command command, BiFunction executor) + { + this.command = command; + this.metadata = command.metadata(); + this.executor = executor; + } + + @Override + public MBeanInfo getMBeanInfo() + { + List operations = new ArrayList<>(); + + operations.add(new MBeanOperationInfo( + INVOKE_METHOD, + "Execute command with JSON parameters. Format: {\"optionName\": \"value\", \"param0\": \"value\", ...}", + new MBeanParameterInfo[]{ + new MBeanParameterInfo("jsonParameters", + String.class.getName(), + "JSON object with command parameters. Use getJsonSchema() to see available parameters and types.") + }, + String.class.getName(), + ACTION)); + + operations.add(new MBeanOperationInfo( + GET_JSON_SCHEMA_METHOD, + "Get JSON schema describing all available parameters (name -> type mapping)", + new MBeanParameterInfo[0], + String.class.getName(), + INFO)); + + operations.add(new MBeanOperationInfo( + GET_PARAMETER_INFO_METHOD, + "Get information about all available parameters (human-readable format)", + new MBeanParameterInfo[0], + String.class.getName(), + INFO)); + + return new MBeanInfo(CommandMBean.class.getName(), + metadata.description() != null ? metadata.description() : "Command: " + metadata.name(), + null, + null, + operations.toArray(new MBeanOperationInfo[0]), + null); + } + + @Override + public Object invoke(String actionName, Object[] params, String[] signature) throws MBeanException, ReflectionException + { + if (INVOKE_METHOD.equals(actionName)) + { + if (params == null || params.length != 1) + throw new IllegalArgumentException("invoke requires exactly one parameter (JSON string)"); + + String jsonParams = (String) params[0]; + try + { + CommandExecutionArgs args = CommandExecutionArgsSerde.fromJson(jsonParams, metadata); + return executor.apply(command.name(), args); + } + catch (MarshalException e) + { + logger.error("Error parsing JSON parameters for command: {}", metadata.name(), e); + throw new MBeanException(e, "Invalid JSON format: " + e.getMessage()); + } + catch (Exception e) + { + logger.error("Error executing command: {}", metadata.name(), e); + throw new MBeanException(e, "Failed to execute command: " + metadata.name() + ": " + e.getMessage()); + } + } + + if (GET_JSON_SCHEMA_METHOD.equals(actionName)) + return getJsonSchema(); + + if (GET_PARAMETER_INFO_METHOD.equals(actionName)) + return getParameterInfo(); + + throw new UnsupportedOperationException("Unknown operation: " + actionName); + } + + public String getJsonSchema() + { + try + { + Map schema = new LinkedHashMap<>(); + schema.put("$schema", "http://json-schema.org/draft-07/schema#"); + schema.put("type", "object"); + schema.put("title", metadata.name()); + schema.put("description", metadata.description()); + + Map properties = new LinkedHashMap<>(); + List required = new ArrayList<>(); + + for (OptionMetadata option : metadata.options()) + { + String primaryName = option.paramLabel(); + properties.put(primaryName, buildJsonSchemaProperty(option)); + + if (option.required()) + required.add(primaryName); + } + + List sortedParams = new ArrayList<>(metadata.parameters()); + sortedParams.sort(Comparator.comparingInt(ParameterMetadata::index)); + + for (ParameterMetadata param : sortedParams) + { + String paramName = COMMAND_POSITIONAL_PARAM_PREFIX + param.index(); + properties.put(paramName, buildJsonSchemaProperty(param)); + + if (param.required()) + required.add(paramName); + } + + schema.put("properties", properties); + + if (!required.isEmpty()) + schema.put("required", required); + + return JsonUtils.writeAsPrettyJsonString(schema); + } + catch (Exception e) + { + logger.error("Error generating JSON schema for command: {}", metadata.name(), e); + throw new RuntimeException("Failed to generate JSON schema: " + e.getMessage(), e); + } + } + + private Map buildJsonSchemaProperty(ArgumentMetadata arg) + { + Map prop = new LinkedHashMap<>(); + prop.put("type", getJsonType(arg.type())); + + if (arg.names() != null && arg.names().length > 0) + prop.put("aliases", Arrays.stream(arg.names()) + .filter(name -> !name.equals(arg.paramLabel())) + .collect(Collectors.toList())); + + if (arg.description() != null && !arg.description().isEmpty()) + prop.put("description", arg.description()); + + String defaultValue = arg.defaultValue(); + if (defaultValue != null && !defaultValue.isEmpty()) + prop.put("default", convertDefaultValue(defaultValue, arg.type())); + + if (arg.type().isArray() || List.class.isAssignableFrom(arg.type())) + { + prop.put("type", "array"); + Map items = new LinkedHashMap<>(); + items.put("type", "string"); + prop.put("items", items); + } + + if (arg.type().isEnum()) + { + prop.put("enum", Arrays.stream(arg.type().getEnumConstants()) + .map(Object::toString) + .collect(Collectors.toList())); + } + + return prop; + } + + private static String getTypeName(Class type) + { + return type.getCanonicalName(); + } + + private String getParameterInfo() + { + StringBuilder info = new StringBuilder(); + info.append("Command: ").append(metadata.name()).append('\n'); + info.append("Description: ").append(metadata.description()).append("\n\n"); + + info.append("Options:\n"); + for (OptionMetadata option : metadata.options()) + { + info.append(" - ").append(option.paramLabel()); + if (option.names().length > 0) + info.append(" (aliases: ").append(String.join(", ", option.names())).append(')'); + appendRequiredClause(info, getTypeName(option.type()), option.required(), option.description()); + } + + info.append("\nPositional Parameters:\n"); + List sortedParams = new ArrayList<>(metadata.parameters()); + sortedParams.sort((a, b) -> Integer.compare(a.index(), b.index())); + + for (ParameterMetadata param : sortedParams) + { + info.append(" - param").append(param.index()); + if (param.paramLabel() != null && !param.paramLabel().isEmpty()) + info.append(" (").append(param.paramLabel()).append(')'); + appendRequiredClause(info, getTypeName(param.type()), param.required(), param.description()); + } + + return info.toString(); + } + + private static void appendRequiredClause(StringBuilder info, + String typeName, + boolean required, + String description) + { + info.append(" [").append(typeName).append(']'); + if (required) + info.append(" [REQUIRED]"); + if (description != null && !description.isEmpty()) + info.append("\n ").append(description); + info.append('\n'); + } + + @Override + public Object getAttribute(String attribute) throws AttributeNotFoundException, MBeanException, ReflectionException + { + throw new UnsupportedOperationException("Not supported yet."); + } + + @Override + public void setAttribute(Attribute attribute) throws AttributeNotFoundException, InvalidAttributeValueException, MBeanException, ReflectionException + { + throw new UnsupportedOperationException("Not supported yet."); + } + + @Override + public AttributeList getAttributes(String[] attributes) + { + throw new UnsupportedOperationException("Not supported yet."); + } + + @Override + public AttributeList setAttributes(AttributeList attributes) + { + throw new UnsupportedOperationException("Not supported yet."); + } +} diff --git a/src/java/org/apache/cassandra/management/CommandService.java b/src/java/org/apache/cassandra/management/CommandService.java new file mode 100644 index 000000000000..283483762570 --- /dev/null +++ b/src/java/org/apache/cassandra/management/CommandService.java @@ -0,0 +1,363 @@ +/* + * 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.cassandra.management; + + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Deque; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.atomic.AtomicInteger; + +import javax.management.ObjectName; + +import com.google.common.base.Strings; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.exceptions.CommandExecutionException; +import org.apache.cassandra.management.api.Command; +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.api.CommandExecutionContext; +import org.apache.cassandra.management.api.CommandRegistry; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.Output; +import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.MBeanWrapper; + +import static org.apache.cassandra.management.ManagementUtils.countCommands; + +/** + * Service that manages the command registry lifecycle and execution. + *

+ * Similar to StorageService, SnapshotManager the service coordinates various aspects of command management: + * - Manages lifecycle (initialization, shutdown) + * - Coordinates MBean registration + * - Provides command execution coordination + * - Integrates with daemon startup + * - Manages state (registry, MBean instances) + */ +public class CommandService implements CommandServiceMBean +{ + private static final String MBEAN_DOMAIN = "org.apache.cassandra.management"; + private static final String MBEAN_TYPE_COMMAND = "Command"; + private static final int MAX_EXECUTION_HISTORY = 100; + private static final String COMMAND_NAME_DELIMITER = "."; + private static final Logger logger = LoggerFactory.getLogger(CommandService.class); + + public static final CommandService instance = new CommandService(); + + private final BoundedExecutionHistory executionHistory = new BoundedExecutionHistory(MAX_EXECUTION_HISTORY); + private final Map commandMBeanNames = new ConcurrentHashMap<>(); + private final MBeanAccessor accessor = new InternalNodeMBeanAccessor(); + private final CommandRegistry registry; + + private volatile boolean started = false; + + private CommandService() + { + this.registry = new CassandraCommandRegistry(); + } + + public synchronized void start() + { + if (started) + return; + + logger.info("Starting command service"); + + registerCommandMBeansRecursively(registry, ""); + MBeanWrapper.instance.registerMBean(this, MBEAN_NAME); + + started = true; + logger.info("Command service started with '{}' commands", countCommands(registry)); + } + + public static void shutdown() + { + instance.stop(); + } + + public synchronized void stop() + { + if (!started) + return; + + logger.info("Stopping command service"); + + unregisterCommandMBeans(); + + try + { + MBeanWrapper.instance.unregisterMBean(MBEAN_NAME); + } + catch (Exception e) + { + logger.warn("Failed to unregister CommandService MBean", e); + } + + started = false; + logger.info("Command service stopped"); + } + + public CommandRegistry getRegistry() + { + return registry; + } + + /** + * Execute a command by name with the given arguments. + * + * @param commandName name of the command to execute. + * @param arguments arguments for the command execution. + * @return captured output of the command execution. + */ + public String executeCommand(String commandName, CommandExecutionArgs arguments) + { + // TODO do input arguments validation and conversion to the CommandExecutionArgs class. + // TODO this should be called via invoker that handles permissions, logging, etc. + if (!started) + throw new IllegalStateException("CommandService is not started"); + + Command command = registry.command(commandName); + if (command == null) + throw new IllegalArgumentException("Command not found: " + commandName); + + UUID executionId = UUID.randomUUID(); + CapturingOutput captured = new CapturingOutput(); + ExecutionHistory record = new ExecutionHistory(commandName, Clock.Global.currentTimeMillis()); + CommandExecutionContext executionContext = new ServerCommandExecutionContext(new NodeProbe(accessor, captured.createOutput())); + + try + { + logger.info("Executing command '{}' with execution ID: {}", commandName, executionId); + executionHistory.add(record); + + CommandInvoker invoker = new CommandInvoker<>(command, arguments, executionContext); + // Currently, for picocli-based commands in C*, which have no structured result, + // the output is written to the Output in the context. + Object ignore = invoker.invoke(); + + record.completed(Clock.Global.currentTimeMillis()); + logger.info("Command '{}' (execution ID: {}) completed successfully", commandName, executionId); + + return captured.getCapturedOutput(); + } + catch (Exception e) + { + record.failed(Clock.Global.currentTimeMillis(), e); + logger.error("Unexpected error executing command '{}' (execution ID: {})", commandName, executionId, e); + throw new CommandExecutionException(String.format("Failed to execute command '%s': %s", commandName, e.getMessage()), e); + } + } + + @Override + public String[] getCommandNames() + { + List commandNames = new ArrayList<>(); + collectCommandNamesRecursively(registry, "", commandNames); + return commandNames.toArray(new String[0]); + } + + private void collectCommandNamesRecursively(CommandRegistry registry, + String parentCommandName, + List result) + { + for (Map.Entry> entry : registry.commands()) + { + String commandName = entry.getKey(); + Command command = entry.getValue(); + + String fullCommandName = Strings.isNullOrEmpty(parentCommandName) ? + commandName : + String.join(COMMAND_NAME_DELIMITER, parentCommandName, commandName); + + if (command instanceof CommandRegistry) + collectCommandNamesRecursively((CommandRegistry) command, fullCommandName, result); + else + result.add(fullCommandName); + } + } + + @Override + public int getCommandCount() + { + return countCommands(registry); + } + + @Override + public String getCommandMBeanName(String fullCommandName) + { + ObjectName objectName = commandMBeanNames.get(fullCommandName); + if (objectName == null) + throw new IllegalArgumentException("Command not found: " + fullCommandName); + return objectName.toString(); + } + + public boolean isStarted() + { + return started; + } + + private void registerCommandMBeansRecursively(CommandRegistry registry, String parentCommandName) + { + for (Map.Entry> e : registry.commands()) + { + // TODO we might want to support aliases for commands donw the line + String commandName = e.getKey(); + Command command = e.getValue(); + String fullCommandName = Strings.isNullOrEmpty(parentCommandName) ? + commandName : + String.join(COMMAND_NAME_DELIMITER, parentCommandName, commandName); + + if (command instanceof CommandRegistry) + { + registerCommandMBeansRecursively((CommandRegistry) command, fullCommandName); + } + else + { + try + { + String escapedName = ObjectName.quote(fullCommandName); + ObjectName objectName = new ObjectName(String.format("%s:type=%s,name=%s", + MBEAN_DOMAIN, MBEAN_TYPE_COMMAND, escapedName)); + CommandMBean commandMBean = new CommandMBean(command, this::executeCommand); + MBeanWrapper.instance.registerMBean(commandMBean, objectName, MBeanWrapper.OnException.LOG); + commandMBeanNames.put(fullCommandName, objectName); + + logger.debug("Registered command MBean: {} -> {}", fullCommandName, objectName); + } + catch (Exception ex) + { + logger.warn("Failed to register MBean for command: {}", fullCommandName, ex); + } + } + } + } + + private void unregisterCommandMBeans() + { + for (ObjectName objectName : commandMBeanNames.values()) + MBeanWrapper.instance.unregisterMBean(objectName, MBeanWrapper.OnException.LOG); + commandMBeanNames.clear(); + } + + private static class CapturingOutput + { + private final ByteArrayOutputStream buffer = new ByteArrayOutputStream(); + private final PrintStream output = new PrintStream(buffer, true); + private final PrintStream error = new PrintStream(buffer, true); + + public String getCapturedOutput() + { + output.flush(); + error.flush(); + return buffer.toString(); + } + + Output createOutput() + { + return new Output(output, error); + } + } + + private static class ServerCommandExecutionContext implements CommandExecutionContext + { + private final NodeProbe probe; + private final Output output; + + public ServerCommandExecutionContext(NodeProbe probe) + { + this.probe = probe; + this.output = probe.output(); + } + + @Override + public NodeProbe nodeProbe() + { + return probe; + } + + @Override + public Output output() + { + return output; + } + } + + private static class ExecutionHistory + { + final UUID executionId; + final String commandName; + final long startTime; + volatile long endTime; + volatile boolean success; + volatile Throwable error; + + ExecutionHistory(String commandName, long startTime) + { + this.executionId = UUID.randomUUID(); + this.commandName = commandName; + this.startTime = startTime; + } + + void completed(long endTime) + { + this.endTime = endTime; + this.success = true; + } + + void failed(long endTime, Throwable error) + { + this.endTime = endTime; + this.success = false; + this.error = error; + } + } + + private static class BoundedExecutionHistory + { + private final Deque dq = new ConcurrentLinkedDeque<>(); + private final AtomicInteger size = new AtomicInteger(0); + private final int maxSize; + + public BoundedExecutionHistory(int maxSize) + { + this.maxSize = maxSize; + } + + public void add(ExecutionHistory info) + { + dq.offer(info); + + if (size.incrementAndGet() > maxSize) + { + ExecutionHistory removed = dq.pollFirst(); + if (removed != null) + size.decrementAndGet(); + } + } + } +} diff --git a/src/java/org/apache/cassandra/management/CommandServiceMBean.java b/src/java/org/apache/cassandra/management/CommandServiceMBean.java new file mode 100644 index 000000000000..18572e7f780a --- /dev/null +++ b/src/java/org/apache/cassandra/management/CommandServiceMBean.java @@ -0,0 +1,48 @@ +/* + * 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.cassandra.management; + +/** + * MBean interface for CommandService. + * Exposes registry-level operations (not individual commands). + */ +public interface CommandServiceMBean +{ + String MBEAN_NAME = "org.apache.cassandra.management:type=CommandService"; + + /** + * Get a list of all command names in the registry. + * @return array of command names + */ + String[] getCommandNames(); + + /** + * Get the total number of commands in the registry. + * @return number of commands + */ + int getCommandCount(); + + /** + * Get ObjectName string for a specific command MBean. + * @param fullCommandName name of the command + * @return ObjectName string for the command MBean + * @throws IllegalArgumentException if command not found + */ + String getCommandMBeanName(String fullCommandName); +} diff --git a/src/java/org/apache/cassandra/management/InternalNodeMBeanAccessor.java b/src/java/org/apache/cassandra/management/InternalNodeMBeanAccessor.java new file mode 100644 index 000000000000..adecf7147ecf --- /dev/null +++ b/src/java/org/apache/cassandra/management/InternalNodeMBeanAccessor.java @@ -0,0 +1,476 @@ +/* + * 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.cassandra.management; + +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; +import java.lang.management.RuntimeMXBean; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Hashtable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import javax.management.JMX; +import javax.management.MBeanServer; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; + +import org.apache.cassandra.audit.AuditLogManager; +import org.apache.cassandra.audit.AuditLogManagerMBean; +import org.apache.cassandra.auth.AbstractCIDRAuthorizer; +import org.apache.cassandra.auth.AuthCache; +import org.apache.cassandra.auth.AuthCacheMBean; +import org.apache.cassandra.auth.AuthCacheService; +import org.apache.cassandra.auth.CIDRGroupsMappingManagerMBean; +import org.apache.cassandra.auth.CIDRPermissionsManagerMBean; +import org.apache.cassandra.auth.NetworkPermissionsCacheMBean; +import org.apache.cassandra.auth.PasswordAuthenticator; +import org.apache.cassandra.auth.PermissionsCacheMBean; +import org.apache.cassandra.auth.RolesCacheMBean; +import org.apache.cassandra.auth.jmx.AuthorizationProxy; +import org.apache.cassandra.batchlog.BatchlogManager; +import org.apache.cassandra.batchlog.BatchlogManagerMBean; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.ColumnFamilyStoreMBean; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.db.compaction.CompactionManagerMBean; +import org.apache.cassandra.db.compression.CompressionDictionaryManagerMBean; +import org.apache.cassandra.db.guardrails.Guardrails; +import org.apache.cassandra.db.guardrails.GuardrailsMBean; +import org.apache.cassandra.db.virtual.CIDRFilteringMetricsTable; +import org.apache.cassandra.db.virtual.CIDRFilteringMetricsTableMBean; +import org.apache.cassandra.gms.FailureDetector; +import org.apache.cassandra.gms.FailureDetectorMBean; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.gms.GossiperMBean; +import org.apache.cassandra.hints.HintsService; +import org.apache.cassandra.hints.HintsServiceMBean; +import org.apache.cassandra.locator.DynamicEndpointSnitch; +import org.apache.cassandra.locator.DynamicEndpointSnitchMBean; +import org.apache.cassandra.locator.EndpointSnitchInfo; +import org.apache.cassandra.locator.EndpointSnitchInfoMBean; +import org.apache.cassandra.locator.LocationInfo; +import org.apache.cassandra.locator.LocationInfoMBean; +import org.apache.cassandra.locator.NodeProximity; +import org.apache.cassandra.metrics.ThreadPoolMetrics; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.MessagingServiceMBean; +import org.apache.cassandra.service.ActiveRepairService; +import org.apache.cassandra.service.ActiveRepairServiceMBean; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.AutoRepairServiceMBean; +import org.apache.cassandra.service.CacheService; +import org.apache.cassandra.service.CacheServiceMBean; +import org.apache.cassandra.service.GCInspector; +import org.apache.cassandra.service.GCInspectorMXBean; +import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.service.StorageProxyMBean; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.StorageServiceMBean; +import org.apache.cassandra.service.accord.AccordOperations; +import org.apache.cassandra.service.accord.AccordOperationsMBean; +import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.SnapshotManagerMBean; +import org.apache.cassandra.streaming.StreamManager; +import org.apache.cassandra.streaming.StreamManagerMBean; +import org.apache.cassandra.tcm.CMSOperations; +import org.apache.cassandra.tcm.CMSOperationsMBean; +import org.apache.cassandra.tools.RemoteJmxMBeanAccessor; +import org.apache.cassandra.utils.MBeanWrapper; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; +import static org.apache.cassandra.service.CassandraDaemon.SKIP_GC_INSPECTOR; + +/** + * Server-side implementation of {@link MBeanAccessor} for in-process execution. + * + *

+ * This implementation provides direct access to MBean instances without using JMX, + * eliminating the need for remote connections or JMX proxies. It is designed for + * server-side command execution as part of CEP-38 Management API, where commands run + * in the same JVM as the Cassandra daemon. + * + *

+ * Unlike {@link RemoteJmxMBeanAccessor}, this implementation: + *

    + *
  • Directly accesses singleton instances (e.g., {@code StorageService.instance})
  • + *
  • Does not require network connections or JMX connectors
  • + *
  • Provides better performance by avoiding serialization/deserialization
  • + *
  • Has no connection state to manage
  • + *
  • Works directly with {@link Keyspace} and {@link ColumnFamilyStore} instances
  • + *
+ * + *

+ * Lazy Initialization: MBean providers are initialized registered for known MBeans. + * + * @see MBeanAccessor + * @see RemoteJmxMBeanAccessor + * @since 5.1 + */ +public class InternalNodeMBeanAccessor implements MBeanAccessor +{ + private final Map, MBeanProvider> mBeanProviders = new ConcurrentHashMap<>(); + private final Map, Object> mBeanCache = new ConcurrentHashMap<>(); + private final Map metricCache = new ConcurrentHashMap<>(); + + /** + * Creates a new InternalNodeMBeanAccessor using direct instance access. + */ + public InternalNodeMBeanAccessor() + { + initializeMBeanProviders(); + } + + /** + * Initializes all statically known MBean instances. + */ + private void initializeMBeanProviders() + { + registerMBeanProvider(AccordOperationsMBean.class, () -> AccordOperations.instance); + registerMBeanProvider(ActiveRepairServiceMBean.class, ActiveRepairService::instance); + registerMBeanProvider(AuditLogManagerMBean.class, () -> AuditLogManager.instance); + registerMBeanProvider(AutoRepairServiceMBean.class, () -> AutoRepairService.instance); + registerMBeanProvider(BatchlogManagerMBean.class, () -> BatchlogManager.instance); + registerMBeanProvider(CMSOperationsMBean.class, () -> CMSOperations.instance); + registerMBeanProvider(CacheServiceMBean.class, () -> CacheService.instance); + registerMBeanProvider(CompactionManagerMBean.class, () -> CompactionManager.instance); + registerMBeanProvider(DynamicEndpointSnitchMBean.class, this::resolveDynamicEndpointSnitch); + registerMBeanProvider(FailureDetectorMBean.class, () -> (FailureDetectorMBean) FailureDetector.instance); + registerMBeanProvider(GCInspectorMXBean.class, this::resolveGCInspector); + registerMBeanProvider(GossiperMBean.class, () -> Gossiper.instance); + registerMBeanProvider(GuardrailsMBean.class, () -> Guardrails.instance); + registerMBeanProvider(HintsServiceMBean.class, () -> HintsService.instance); + registerMBeanProvider(MemoryMXBean.class, ManagementFactory::getMemoryMXBean); + registerMBeanProvider(MessagingServiceMBean.class, MessagingService::instance); + registerMBeanProvider(RuntimeMXBean.class, ManagementFactory::getRuntimeMXBean); + registerMBeanProvider(SnapshotManagerMBean.class, () -> SnapshotManager.instance); + registerMBeanProvider(StorageProxyMBean.class, () -> StorageProxy.instance); + registerMBeanProvider(StorageServiceMBean.class, () -> StorageService.instance); + registerMBeanProvider(StreamManagerMBean.class, () -> StreamManager.instance); + + // Utility MBeans are stateless and can be created on demand. + // They query DatabaseDescriptor for the current state, so new instances are fine + registerMBeanProvider(EndpointSnitchInfoMBean.class, EndpointSnitchInfo::new); + registerMBeanProvider(LocationInfoMBean.class, LocationInfo::new); + + // AuthCache MBeans + registerMBeanProvider(AuthorizationProxy.JmxPermissionsCacheMBean.class, + () -> findAuthCache(AuthorizationProxy.JmxPermissionsCacheMBean.class)); + registerMBeanProvider(NetworkPermissionsCacheMBean.class, + () -> findAuthCache(NetworkPermissionsCacheMBean.class)); + registerMBeanProvider(PasswordAuthenticator.CredentialsCacheMBean.class, + () -> findAuthCache(PasswordAuthenticator.CredentialsCacheMBean.class)); + registerMBeanProvider(PermissionsCacheMBean.class, + () -> findAuthCache(PermissionsCacheMBean.class)); + registerMBeanProvider(RolesCacheMBean.class, + () -> findAuthCache(RolesCacheMBean.class)); + + // CIDR Auth MBeans + registerMBeanProvider(CIDRFilteringMetricsTableMBean.class, () -> CIDRFilteringMetricsTable.instance); + registerMBeanProvider(CIDRGroupsMappingManagerMBean.class, () -> AbstractCIDRAuthorizer.cidrGroupsMappingManager); + registerMBeanProvider(CIDRPermissionsManagerMBean.class, () -> AbstractCIDRAuthorizer.cidrPermissionsManager); + } + + /** + * Gets DynamicEndpointSnitch from DatabaseDescriptor if it's a DynamicEndpointSnitch, + * otherwise returns null. + */ + private DynamicEndpointSnitchMBean resolveDynamicEndpointSnitch() + { + if (!DatabaseDescriptor.isDynamicEndpointSnitch()) + throw new IllegalStateException("DynamicEndpointSnitch has been requested but is not enabled"); + + NodeProximity proximity = DatabaseDescriptor.getNodeProximity(); + assert proximity instanceof DynamicEndpointSnitch; + + return (DynamicEndpointSnitchMBean) proximity; + } + + private GCInspectorMXBean resolveGCInspector() + { + if (SKIP_GC_INSPECTOR) + throw new IllegalStateException("GCInspector has been requested but is disabled via SKIP_GC_INSPECTOR flag"); + + try + { + MBeanServer mbs = MBeanWrapper.instance.getMBeanServer(); + if (mbs == null) + return null; + + ObjectName name = new ObjectName(GCInspector.MBEAN_NAME); + if (mbs.isRegistered(name)) + return JMX.newMBeanProxy(mbs, name, GCInspectorMXBean.class); + } + catch (Exception e) + { + // Fall through to create a new instance + } + + return null; + } + + /** Finds an auth cache MBean instance from AuthCacheService. */ + private T findAuthCache(Class clazz) + { + Set> caches = AuthCacheService.instance.getCaches(); + if (caches.isEmpty()) + return null; + + AuthCacheFinder visitor = new AuthCacheFinder(clazz); + for (AuthCache cache : caches) + { + cache.accept(visitor); + Object found = visitor.getCache(); + if (found == null) + continue; + return clazz.cast(found); + } + return null; + } + + private void registerMBeanProvider(Class clazz, MBeanProvider locator) + { + Object prev = mBeanProviders.putIfAbsent(clazz, locator); + assert prev == null : "MBean locator for " + clazz.getName() + " is already registered"; + } + + @Override + public T findMBean(Class clazz) + { + Object cached = mBeanCache.get(clazz); + if (cached != null) + return clazz.cast(cached); + + Object prev = mBeanCache.computeIfAbsent(clazz, k -> { + MBeanProvider provider = mBeanProviders.get(k); + return provider == null ? null : provider.provide(); + }); + if (prev == null) + throw new RuntimeException("MBean of type " + clazz.getName() + " is not registered"); + return clazz.cast(prev); + } + + @Override + public T findMBeanMetric(Class clazz, Props props) + { + // Use the internal MBean server to look up MBean by ObjectName. This leverages existing + // JMX infrastructure and avoids the complexity of constructing metric names from Props. + // + // Alternatively, we could use CassandraMetricsRegistry to look up metrics by metric name + // (e.g., "org.apache.cassandra.metrics.Keyspace.ReadLatency.mykeyspace"), but this requires + // constructing the full metric name from Props, which is problematic. + // + // The reconstructing the scope problem: Different MetricNameFactory implementations + // construct scopes differently: + // - KeyspaceMetrics: scope = keyspace property + // - TableMetrics: scope = keyspace + '.' + scope property + // - DefaultNameFactory: scope = scope property + // - SAI AbstractMetrics: scope = keyspace.table.index.scope (all combined) + // + // To construct metric names from Props, we would need to duplicate scope construction logic + // from each factory or refactor to share it. Using ObjectName, in turn, avoids this. We query + // MBeanServer using the ObjectName pattern already constructed by factories during registration. + // + // However, it will be beneficial to revisit this down the line for performance optimizations and + // to avoid JMX entanglement, so we could switch it off for in-process access. + + try + { + ObjectName objectName = buildObjectNameFromProps(props); + String cacheKey = objectName.getCanonicalName(); + + @SuppressWarnings("unchecked") + T cached = (T) metricCache.get(cacheKey); + if (cached != null) + return cached; + + MBeanServer mbs = MBeanWrapper.instance.getMBeanServer(); + if (!mbs.isRegistered(objectName)) + return null; + + assert clazz.isInterface() && clazz.getName().endsWith("MBean"); + return clazz.cast(metricCache.computeIfAbsent(cacheKey, k -> JMX.newMBeanProxy(mbs, objectName, clazz))); + } + catch (Exception e) + { + throw new RuntimeException("Error accessing metric MBean: " + e.getMessage(), e); + } + } + + private static ObjectName buildObjectNameFromProps(Props props) throws MalformedObjectNameException + { + return new ObjectName("org.apache.cassandra.metrics", new Hashtable<>(props.toMap())); + } + + @Override + public ColumnFamilyStoreMBean findColumnFamily(String type, String keyspace, String columnFamily) + { + try + { + // TODO not sure that we have to open keyspace again here? + Keyspace ks = Keyspace.open(keyspace); + return ks.getColumnFamilyStore(columnFamily); + } + catch (Exception e) + { + throw new RuntimeException("Error accessing column family: " + keyspace + "." + columnFamily, e); + } + } + + @Override + public CompressionDictionaryManagerMBean findCompressionDictionary(String keyspace, String table) + { + try + { + // TODO not sure that we have to open keyspace again here? + Keyspace ks = Keyspace.open(keyspace); + ColumnFamilyStore cfs = ks.getColumnFamilyStore(table); + return cfs.compressionDictionaryManager(); + } + catch (Exception e) + { + throw new RuntimeException("Error accessing compression dictionary: " + keyspace + "." + table, e); + } + } + + @Override + public List threadPoolInfos() + { + List infos = new ArrayList<>(); + for (ThreadPoolMetrics metrics : Metrics.allThreadPoolMetrics()) + infos.add(new ThreadPoolInfo(metrics.path, metrics.poolName)); + return infos; + } + + @Override + public List> findColumnFamilies(String type) + { + try + { + assert type.equals("IndexColumnFamilies") || type.equals("ColumnFamilies"); + + List> mbeans = new ArrayList<>(); + + for (Keyspace keyspace : Keyspace.all()) + { + for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores()) + { + if (type.equals("IndexColumnFamilies") && !cfs.isIndex()) + continue; + if (type.equals("ColumnFamilies") && cfs.isIndex()) + continue; + + mbeans.add(new AbstractMap.SimpleImmutableEntry<>(keyspace.getName(), cfs)); + } + } + + return mbeans; + } + catch (Exception e) + { + throw new RuntimeException("Error accessing column families", e); + } + } + + @Override + public void close() + { + metricCache.clear(); + mBeanCache.clear(); + } + + /** + * Functional interface for providing MBean instances lazily. + * Used to defer MBean initialization until the MBean is actually accessed. + * + * @param the MBean interface type + */ + @FunctionalInterface + public interface MBeanProvider + { + /** + * @return the MBean instance, or {@code null} if the MBean is not available + * @throws RuntimeException if the MBean cannot be provided (e.g., not initialized yet) + */ + T provide(); + } + + /** Visitor that finds a specific auth cache MBean type from AuthCacheService. */ + private static class AuthCacheFinder implements AuthCache.MBeanVisitor + { + private final Class targetType; + private Object foundCache; + + AuthCacheFinder(Class targetType) + { + this.targetType = targetType; + } + + @Override + public void visitCredentials(PasswordAuthenticator.CredentialsCacheMBean cache) + { + if (targetType.equals(PasswordAuthenticator.CredentialsCacheMBean.class)) + foundCache = cache; + } + + @Override + public void visitJmxPermissions(AuthorizationProxy.JmxPermissionsCacheMBean cache) + { + if (targetType.equals(AuthorizationProxy.JmxPermissionsCacheMBean.class)) + foundCache = cache; + } + + @Override + public void visitPermissions(PermissionsCacheMBean cache) + { + if (targetType.equals(PermissionsCacheMBean.class)) + foundCache = cache; + } + + @Override + public void visitNetwork(NetworkPermissionsCacheMBean cache) + { + if (targetType.equals(NetworkPermissionsCacheMBean.class)) + foundCache = cache; + } + + @Override + public void visitRoles(RolesCacheMBean cache) + { + if (targetType.equals(RolesCacheMBean.class)) + foundCache = cache; + } + + @Override + public void visit(AuthCacheMBean cache) + { + // No-op. Used for caches without specific MBean types. + } + + Object getCache() + { + return foundCache; + } + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/management/MBeanAccessor.java b/src/java/org/apache/cassandra/management/MBeanAccessor.java new file mode 100644 index 000000000000..38032ad3f034 --- /dev/null +++ b/src/java/org/apache/cassandra/management/MBeanAccessor.java @@ -0,0 +1,160 @@ +/* + * 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.cassandra.management; + + +import java.util.List; +import java.util.Map; + +import javax.annotation.Nullable; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; + +import org.apache.cassandra.db.ColumnFamilyStoreMBean; +import org.apache.cassandra.db.compression.CompressionDictionaryManagerMBean; +import org.apache.cassandra.metrics.CassandraMetricsRegistry; + +public interface MBeanAccessor extends AutoCloseable +{ + /** + * Finds statically known MBeans by MBean class name. + * @param clazz the MBean class. + * @return MBean class or {@code null} if not found. + * @param the MBean type. + */ + @Nullable + T findMBean(Class clazz); + T findMBeanMetric(Class clazz, Props props); + + ColumnFamilyStoreMBean findColumnFamily(String type, String keyspace, String columnFamily); + CompressionDictionaryManagerMBean findCompressionDictionary(String keyspace, String table); + + /** List of thread pool MBeans with associated path and pool name. */ + List threadPoolInfos(); + /** List of column family MBeans with associated keyspace name. */ + List> findColumnFamilies(String type); + + /** {@inheritDoc} */ + @Override + default void close() { } + + default CassandraMetricsRegistry.JmxCounterMBean findMBeanCounter(Props props) + { + return findMBeanMetric(CassandraMetricsRegistry.JmxCounterMBean.class, props); + } + + default CassandraMetricsRegistry.JmxGaugeMBean findMBeanGauge(Props props) + { + return findMBeanMetric(CassandraMetricsRegistry.JmxGaugeMBean.class, props); + } + + default CassandraMetricsRegistry.JmxMeterMBean findMBeanMeter(Props props) + { + return findMBeanMetric(CassandraMetricsRegistry.JmxMeterMBean.class, props); + } + + default CassandraMetricsRegistry.JmxTimerMBean findMBeanTimer(Props props) + { + return findMBeanMetric(CassandraMetricsRegistry.JmxTimerMBean.class, props); + } + + default CassandraMetricsRegistry.JmxHistogramMBean findMBeanHistogram(Props props) + { + return findMBeanMetric(CassandraMetricsRegistry.JmxHistogramMBean.class, props); + } + + class Props + { + private final Map values = Maps.newLinkedHashMap(); + + public Props(String type, String path, String keyspace, String table, String scope, String name) + { + if (type == null || type.isEmpty()) + throw new IllegalArgumentException("type is required"); + values.put("type", type); + if (path != null) + values.put("path", path); + if (keyspace != null) + values.put("keyspace", keyspace); + if (table != null) + values.put("table", table); + if (scope != null) + values.put("scope", scope); + if (name != null) + values.put("name", name); + } + + public static Props metric(String type, String name) + { + return new Props(type, null, null, null, null, name); + } + + public static Props scoped(String type, String scope, String name) + { + return new Props(type, null, null, null, scope, name); + } + + public static Props threadPool(String type, String path, String scope, String name) + { + return new Props(type, path, null, null, scope, name); + } + + public static Props sai(String type, String keyspace, String table, String scope, String name) + { + return new Props(type, null, keyspace, table, scope, name); + } + + public static Props columnFamily(String type, String keyspace, String scope, String name) + { + return new Props(type, null, keyspace, null, scope, name); + } + + public static Props keyspace(String type, String keyspace, String name) + { + return new Props(type, null, keyspace, null, null, name); + } + + public Map toMap() + { + return ImmutableMap.copyOf(values); + } + } + + class ThreadPoolInfo + { + private final String path; + private final String poolName; + + public ThreadPoolInfo(String path, String poolName) + { + this.path = path; + this.poolName = poolName; + } + + public String path() + { + return path; + } + public String scope() + { + return path + '.' + poolName; + } + } +} diff --git a/src/java/org/apache/cassandra/management/ManagementUtils.java b/src/java/org/apache/cassandra/management/ManagementUtils.java new file mode 100644 index 000000000000..45d347a1a8e0 --- /dev/null +++ b/src/java/org/apache/cassandra/management/ManagementUtils.java @@ -0,0 +1,66 @@ +/* + * 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.cassandra.management; + +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.util.Map; +import java.util.ServiceLoader; + +import org.apache.cassandra.management.api.Command; +import org.apache.cassandra.management.api.CommandRegistry; + +public final class ManagementUtils +{ + public static Iterable loadService(Class serviceClz) + { + return AccessController.doPrivileged((PrivilegedAction>) () -> ServiceLoader.load(serviceClz)); + } + + public static int countCommands(CommandRegistry registry) + { + int count = 0; + for (Map.Entry> entry : registry.commands()) + { + Command cmd = entry.getValue(); + if (cmd instanceof CommandRegistry) + count += countCommands((CommandRegistry) cmd); + else + count++; + } + return count; + } + + public static String stripAngleBrackets(String name) + { + if (name == null || name.isEmpty()) + return name; + + String trimmed = name.trim(); + + if (trimmed.length() >= 2 && + trimmed.charAt(0) == '<' && + trimmed.charAt(trimmed.length() - 1) == '>') + { + return trimmed.substring(1, trimmed.length() - 1).trim(); + } + + return name; + } +} diff --git a/src/java/org/apache/cassandra/management/SimpleCommandExecutionArgs.java b/src/java/org/apache/cassandra/management/SimpleCommandExecutionArgs.java new file mode 100644 index 000000000000..a3f2a462f1f0 --- /dev/null +++ b/src/java/org/apache/cassandra/management/SimpleCommandExecutionArgs.java @@ -0,0 +1,63 @@ +/* + * 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.cassandra.management; + +import java.util.LinkedHashMap; +import java.util.Map; + +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.api.OptionMetadata; +import org.apache.cassandra.management.api.ParameterMetadata; + +public class SimpleCommandExecutionArgs implements CommandExecutionArgs +{ + private final Map options; + private final Map parameters; + + public SimpleCommandExecutionArgs(Map options, + Map parameters) + { + this.options = new LinkedHashMap<>(options); + this.parameters = new LinkedHashMap<>(parameters); + } + + @Override + public Map parameters() + { + return parameters; + } + + @Override + public Map options() + { + return options; + } + + @Override + public boolean hasParameter(ParameterMetadata param) + { + return parameters.containsKey(param); + } + + @Override + public boolean hasOption(OptionMetadata option) + { + return options.containsKey(option); + } +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/ManagementContext.java b/src/java/org/apache/cassandra/management/api/ArgumentMetadata.java similarity index 70% rename from src/java/org/apache/cassandra/tools/nodetool/ManagementContext.java rename to src/java/org/apache/cassandra/management/api/ArgumentMetadata.java index 44e9e9c7cbd1..da7c6a8bdfb4 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/ManagementContext.java +++ b/src/java/org/apache/cassandra/management/api/ArgumentMetadata.java @@ -16,16 +16,17 @@ * limitations under the License. */ -package org.apache.cassandra.tools.nodetool; +package org.apache.cassandra.management.api; -import java.net.UnknownHostException; - -import org.apache.cassandra.service.StorageServiceMBean; - -/** - * Management context for nodetool commands to access management services like StorageServiceMBean etc. - */ -public interface ManagementContext +public interface ArgumentMetadata { - T getManagementService(Class serviceClass); + String[] names(); + String paramLabel(); + + String description(); + Class type(); + String defaultValue(); + boolean required(); + /** "0" (flag), "1", "0..1", "0..*" */ + String arity(); } diff --git a/src/java/org/apache/cassandra/management/api/Command.java b/src/java/org/apache/cassandra/management/api/Command.java index d00f44b08b0f..c23ed0acdde6 100644 --- a/src/java/org/apache/cassandra/management/api/Command.java +++ b/src/java/org/apache/cassandra/management/api/Command.java @@ -18,26 +18,51 @@ package org.apache.cassandra.management.api; -import org.apache.cassandra.tools.nodetool.AbstractCommand; - /** * Command interface for all management commands. *

- * Name of the command that is expected from caller derived from actual command class name.
- *

    - *
  • Name format: All words divided by capital letters except "Command" suffix will form hierarchical command name.
  • - *
  • Example: {@code MyUsefullCommand} is name of command so {@code nodetool myusefull param1 param2} - * expected from user.
  • - *
- *

- * Other protocols must expose command similarly. Rest API must expect {@code /api-root/my-usefull?param1=value1¶m2=value2} URI. - * - * @param Command user object type. - * @param Command result type. + * Follows the Command design pattern: + *

+ * ┌─────────────┐
+ * │   Client    │ Creates commands
+ * └──────┬──────┘
+ *        │
+ *        ▼
+ * ┌─────────────┐       ┌──────────────┐
+ * │  Invoker    │──────▶│   Command    │ Interface
+ * │ (executes)  │       │  execute()   │
+ * └─────────────┘       └──────┬───────┘
+ *                              │
+ *                              ▼
+ *                     ┌─────────────────┐
+ *                     │ ConcreteCommand │ Implements Command
+ *                     │  execute() {    │
+ *                     │   receiver.do() │
+ *                     │  }              │
+ *                     └────────┬────────┘
+ *                              │
+ *                              ▼
+ *                     ┌─────────────────┐
+ *                     │    Receiver     │ Does actual work
+ *                     │   (NodeProbe)   │
+ *                     └─────────────────┘
+ * 
*/ -public interface Command +public interface Command { - String name(); - String description(); - Class commandUserObject(); + /** Get command metadata - replaces argClass() from the cep-38 doc with richer information. */ + CommandMetadata metadata(); + + /** + * Execute the command. + * + * @param arguments command arguments. + * @param context execution context (provides NodeProbe and Output). + * @return structured data result, or {@code null} if only output streams are used and also + * used for backwards compatibility with all existing picocli-based commands. + */ + R execute(CommandExecutionArgs arguments, CommandExecutionContext context); + + default String name() { return metadata().name(); } + default String description() { return metadata().description(); } } diff --git a/src/java/org/apache/cassandra/management/api/CommandExecutionArgs.java b/src/java/org/apache/cassandra/management/api/CommandExecutionArgs.java new file mode 100644 index 000000000000..af11036640b0 --- /dev/null +++ b/src/java/org/apache/cassandra/management/api/CommandExecutionArgs.java @@ -0,0 +1,32 @@ +/* + * 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.cassandra.management.api; + +import java.util.Map; + +public interface CommandExecutionArgs +{ + /** @return Map of parameter metadata to their values. */ + Map parameters(); + /** @return Map of option metadata to their values. */ + Map options(); + + boolean hasParameter(ParameterMetadata param); + boolean hasOption(OptionMetadata option); +} diff --git a/src/java/org/apache/cassandra/management/api/CommandContext.java b/src/java/org/apache/cassandra/management/api/CommandExecutionContext.java similarity index 89% rename from src/java/org/apache/cassandra/management/api/CommandContext.java rename to src/java/org/apache/cassandra/management/api/CommandExecutionContext.java index 665060185feb..312722887e54 100644 --- a/src/java/org/apache/cassandra/management/api/CommandContext.java +++ b/src/java/org/apache/cassandra/management/api/CommandExecutionContext.java @@ -18,10 +18,11 @@ package org.apache.cassandra.management.api; +import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.Output; -public interface CommandContext +public interface CommandExecutionContext { - T findMBean(Class clazz); + NodeProbe nodeProbe(); Output output(); } diff --git a/src/java/org/apache/cassandra/management/api/CommandInvoker.java b/src/java/org/apache/cassandra/management/api/CommandInvoker.java deleted file mode 100644 index 165e4caf8d27..000000000000 --- a/src/java/org/apache/cassandra/management/api/CommandInvoker.java +++ /dev/null @@ -1,148 +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 org.apache.cassandra.management.api; - -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.Collection; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.Set; -import java.util.SortedMap; -import java.util.SortedSet; -import java.util.TreeMap; -import java.util.TreeSet; -import java.util.function.Consumer; - -import javax.inject.Inject; - -import org.apache.cassandra.exceptions.CommandExecutionException; -import picocli.CommandLine; - -public class CommandInvoker -{ - private final Command command; - private final CommandContext context; - private final String[] args; - - public CommandInvoker(Command command, CommandContext context, String... args) - { - this.command = command; - this.context = context; - this.args = args; - } - - /** - * Invoke the command and return the exit code. - * @param resultConsumer a consumer for the result. - * @return the exit code. - */ - public int invokeExitCode(Consumer resultConsumer) - { - return new CommandLine(instantiateCommand(command), new InjectContextFactory(context)).execute(args); - } - - /** - * Invoke the command and return the result. - * @param printer a consumer for the result. - * @return the result. - */ - public R invokeResult(Consumer printer) - { - new CommandLine(instantiateCommand(command), new InjectContextFactory(context)).execute(args); - return null; - } - - private Object instantiateCommand(Command command) - { - try - { - return new InjectContextFactory(context).create(command.commandUserObject()); - } - catch (Exception e) - { - throw new CommandExecutionException(String.format("Failed to inject resource [command=%s]", - command.commandUserObject()), e); - } - } - - private static class InjectContextFactory implements CommandLine.IFactory - { - private final CommandContext context; - - public InjectContextFactory(CommandContext context) - { - this.context = context; - } - - /** - * Create an instance of the specified class. This method is called by picocli to obtain instances of classes. - * @param clazz the class of the object to return. - * @return an instance of the specified class. - * @param the type of the object to return. - * @throws Exception an exception. - */ - public T create(Class clazz) throws Exception - { - if (clazz.isInterface()) - { - if (Collection.class.isAssignableFrom(clazz)) - { - if (List.class.isAssignableFrom(clazz)) - return clazz.cast(new ArrayList<>()); - else if (SortedSet.class.isAssignableFrom(clazz)) - return clazz.cast(new TreeSet<>()); - else if (Set.class.isAssignableFrom(clazz)) - return clazz.cast(new LinkedHashSet<>()); - else if (Queue.class.isAssignableFrom(clazz)) - return clazz.cast(new LinkedList<>()); - else - return clazz.cast(new ArrayList<>()); - } - if (SortedMap.class.isAssignableFrom(clazz)) - { - return clazz.cast(new TreeMap<>()); - } - if (Map.class.isAssignableFrom(clazz)) - { - return clazz.cast(new LinkedHashMap<>()); - } - } - - // Inject the context into the command object. - Object target = clazz.getDeclaredConstructor().newInstance(); - for (Field field : clazz.getDeclaredFields()) - { - if (field.isAnnotationPresent(Inject.class) && field.getType().isAssignableFrom(context.getClass())) - inject(field, target, context); - } - return clazz.cast(target); - } - - private static void inject(Field field, Object target, Object resource) throws Exception - { - field.setAccessible(true); - field.set(target, resource); - } - } -} diff --git a/src/java/org/apache/cassandra/management/api/CommandMBean.java b/src/java/org/apache/cassandra/management/api/CommandMBean.java deleted file mode 100644 index aab5f3f0ce39..000000000000 --- a/src/java/org/apache/cassandra/management/api/CommandMBean.java +++ /dev/null @@ -1,218 +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 org.apache.cassandra.management.api; - -import java.lang.reflect.AnnotatedElement; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.List; -import java.util.function.BiConsumer; -import java.util.function.Consumer; -import java.util.function.Function; -import javax.management.Attribute; -import javax.management.AttributeList; -import javax.management.AttributeNotFoundException; -import javax.management.DynamicMBean; -import javax.management.InvalidAttributeValueException; -import javax.management.MBeanException; -import javax.management.MBeanInfo; -import javax.management.MBeanOperationInfo; -import javax.management.MBeanParameterInfo; -import javax.management.ReflectionException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import picocli.CommandLine; - -import static javax.management.MBeanOperationInfo.ACTION; -import static org.apache.cassandra.management.CommandUtils.optionName; -import static org.apache.cassandra.management.CommandUtils.buildAttributeDescription; -import static org.apache.cassandra.management.CommandUtils.paramName; - -/** - * Command MBean exposes single mamagement command to the JMX interface. - * - * @see Command - * @see CommandsRegistry - */ -public class CommandMBean implements DynamicMBean -{ - /** Name of the JMX method to invoke command. */ - public static final String INVOKE = "invoke"; - public static final String OPTIONAL = "0..1"; - public static final String REPREATABLE = "0..*"; - - /** Used for tests. Name of the method to retrive last method result. */ - public static final String LAST_RES_METHOD = "lastResult"; - - private static final Logger logger = LoggerFactory.getLogger(CommandMBean.class); - private final Command command; - - /** - * @param command Management command to expose. - */ - public CommandMBean(Command command) - { - this.command = command; - } - - @Override - public Object invoke(String actionName, - Object[] params, - String[] signature) throws MBeanException, ReflectionException - { - // Default JMX invoker pass arguments in for params: Object[] = { "invoke", parameter_values_array, types_array} - // while JConsole pass params values directly in params array. This check supports both way of invocation. - if (params.length == 3 - && (params[0].equals(INVOKE) || params[0].equals(LAST_RES_METHOD)) - && params[1] instanceof Object[]) - return invoke((String) params[0], (Object[]) params[1], (String[]) params[2]); - - if (!INVOKE.equals(actionName)) - throw new UnsupportedOperationException(actionName); - - try - { - StringBuilder resStr = new StringBuilder(); - Consumer printer = str -> resStr.append(str).append('\n'); - -// CommandInvoker invoker = new CommandInvoker<>(command, new ParamsToArgument(params).argument(), ignite); -// res = invoker.invoke(printer, false); - - return resStr.toString(); - } - catch (Exception e) - { - logger.error("Invoke error:", e); - throw e; - } - } - - /** - * All options are required within the ArgGroup, while the group itself is optional: - *
-     * public class DependentOptions {
-     *     @ArgGroup(exclusive = false, multiplicity = "0..1")
-     *     Dependent group;
-     *
-     *     static class Dependent {
-     *         @Option(names = "-a", required = true) int a;
-     *         @Option(names = "-b", required = true) int b;
-     *         @Option(names = "-c", required = true) int c;
-     *     }
-     * }
- * - * @return MBean info. - */ - @Override - public MBeanInfo getMBeanInfo() - { - List args = new ArrayList<>(); - visitCommandSpec(command, (name, desc) -> args.add(new MBeanParameterInfo(name, String.class.getName(), desc))); - return new MBeanInfo(CommandMBean.class.getName(), - command.getClass().getSimpleName(), - null, - null, - new MBeanOperationInfo[]{ - new MBeanOperationInfo(INVOKE, - command.description(), - args.toArray(MBeanParameterInfo[]::new), - String.class.getName(), - ACTION) - }, - null); - } - - public static void visitCommandSpec(Command command, BiConsumer visitor) - { - CommandLine.Model.CommandSpec spec = CommandLine.Model.CommandSpec.forAnnotatedObject(command.commandUserObject()); - // Options are required within the ArgGroup, while the group itself is optional. - for (CommandLine.Model.OptionSpec option : spec.options()) - { - if (!option.isOption()) - continue; - - visitor.accept(optionName(option.longestName()), buildAttributeDescription(option.description())); - } - - // Positional parameters are required. - for (CommandLine.Model.PositionalParamSpec positional : spec.positionalParameters()) - { - if (!positional.isPositional()) - continue; - - visitor.accept(paramName(positional.paramLabel()), buildAttributeDescription(positional.description())); - } - } - - @Override - public AttributeList getAttributes(String[] attributes) - { - throw new UnsupportedOperationException("getAttributes"); - } - - @Override - public AttributeList setAttributes(AttributeList attributes) - { - throw new UnsupportedOperationException("setAttributes"); - } - - @Override - public Object getAttribute(String attribute) throws AttributeNotFoundException, MBeanException, ReflectionException - { - throw new UnsupportedOperationException("getAttribute"); - } - - @Override - public void setAttribute(Attribute attribute) throws AttributeNotFoundException, InvalidAttributeValueException, - MBeanException, ReflectionException - { - throw new UnsupportedOperationException("setAttribute"); - } - - /** */ - private class ParamsToArgument implements Function - { - private int cntr; - private final Object[] vals; - - private ParamsToArgument(Object[] vals) - { - this.vals = vals; - } - - public Object argument() - { - // This will map vals to argument fields. -// return CommandUtils.argument(command.argClass(), (fld, pos) -> apply(fld), this); - return null; - } - - @Override - public Object apply(Field field) - { - String val = (String) vals[cntr]; - - cntr++; - -// return !F.isEmpty(val) ? CommandUtils.parseVal(val, field.getType()) : null; - return null; - } - } -} diff --git a/src/java/org/apache/cassandra/management/api/CommandMetadata.java b/src/java/org/apache/cassandra/management/api/CommandMetadata.java new file mode 100644 index 000000000000..1622a3f72390 --- /dev/null +++ b/src/java/org/apache/cassandra/management/api/CommandMetadata.java @@ -0,0 +1,35 @@ +/* + * 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.cassandra.management.api; + +import java.util.List; + +/** + * TODO: Which information abstraction from cli commands do we want to expose via this interface? + */ +public interface CommandMetadata +{ + String name(); + String description(); + List aliases(); + + List options(); + List parameters(); + List subcommands(); +} diff --git a/src/java/org/apache/cassandra/management/api/CommandRegistry.java b/src/java/org/apache/cassandra/management/api/CommandRegistry.java new file mode 100644 index 000000000000..ab151ca4e6ae --- /dev/null +++ b/src/java/org/apache/cassandra/management/api/CommandRegistry.java @@ -0,0 +1,45 @@ +/* + * 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.cassandra.management.api; + +import java.util.Map; + +/** + * Registry that knows all of its subcommands and acts as a composite command for hierarchy. + * Allows registering and retrieving commands by name. + *

+ * For example, nodetool commands structure is like: + *

+ * NodetoolCommand (parent)
+ *   ├── setconcurrentcompactors (leaf command, implemented via Command interface)
+ *   ├── compressiondictionary (parent command, implemented via CommandRegistry interface)
+ *   │   ├── train (leaf)
+ *   │   ├── list (leaf)
+ *   │   ├── export (leaf)
+ *   │   └── import (leaf)
+ *   ├── decommission (parent)
+ *   │   └── abort (leaf)
+ *   └── ... 100+ more commands
+ * 
+ */ +public interface CommandRegistry extends Command +{ + Command command(String name); + Iterable>> commands(); +} diff --git a/src/java/org/apache/cassandra/management/api/LocalCommand.java b/src/java/org/apache/cassandra/management/api/CommandsProvider.java similarity index 77% rename from src/java/org/apache/cassandra/management/api/LocalCommand.java rename to src/java/org/apache/cassandra/management/api/CommandsProvider.java index da724289a2fb..e9e2c51794a9 100644 --- a/src/java/org/apache/cassandra/management/api/LocalCommand.java +++ b/src/java/org/apache/cassandra/management/api/CommandsProvider.java @@ -18,12 +18,13 @@ package org.apache.cassandra.management.api; -import java.util.function.Consumer; +import java.util.Collection; /** - * A command that can be executed locally on the node, without the need to know the remote node. + * Pluggable component that is responsible for providing a list of commands for management API. */ -public interface LocalCommand extends Command +public interface CommandsProvider { - R execute(CommandContext context, Consumer printer); + /** Gets all supported by these provider commands. */ + Collection> commands(); } diff --git a/src/java/org/apache/cassandra/management/api/CommandsRegistry.java b/src/java/org/apache/cassandra/management/api/OptionMetadata.java similarity index 80% rename from src/java/org/apache/cassandra/management/api/CommandsRegistry.java rename to src/java/org/apache/cassandra/management/api/OptionMetadata.java index 858fb39bdedf..0809add926ce 100644 --- a/src/java/org/apache/cassandra/management/api/CommandsRegistry.java +++ b/src/java/org/apache/cassandra/management/api/OptionMetadata.java @@ -18,12 +18,6 @@ package org.apache.cassandra.management.api; -import java.util.Map; - -/** - * Registry that knows all of its subcommands. - */ -public interface CommandsRegistry extends Iterable>> +public interface OptionMetadata extends ArgumentMetadata { - Command command(String name); } diff --git a/src/java/org/apache/cassandra/management/api/ParameterMetadata.java b/src/java/org/apache/cassandra/management/api/ParameterMetadata.java new file mode 100644 index 000000000000..a1a2a45df85f --- /dev/null +++ b/src/java/org/apache/cassandra/management/api/ParameterMetadata.java @@ -0,0 +1,25 @@ +/* + * 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.cassandra.management.api; + +public interface ParameterMetadata extends ArgumentMetadata +{ + String COMMAND_POSITIONAL_PARAM_PREFIX = "param"; + int index(); +} diff --git a/src/java/org/apache/cassandra/management/picocli/PicocliCommandAdapter.java b/src/java/org/apache/cassandra/management/picocli/PicocliCommandAdapter.java new file mode 100644 index 000000000000..8aa9e12c3d9e --- /dev/null +++ b/src/java/org/apache/cassandra/management/picocli/PicocliCommandAdapter.java @@ -0,0 +1,125 @@ +/* + * 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.cassandra.management.picocli; + +import java.lang.reflect.Field; + +import javax.inject.Inject; + +import org.apache.cassandra.management.api.Command; +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.api.CommandExecutionContext; +import org.apache.cassandra.management.api.CommandMetadata; +import org.apache.cassandra.tools.Output; +import org.apache.cassandra.tools.nodetool.AbstractCommand; + +import picocli.CommandLine; + +public class PicocliCommandAdapter implements Command +{ + private final Class commandClass; + private final CommandMetadata commandMetadata; + + public PicocliCommandAdapter(Class commandClass) + { + this.commandClass = commandClass; + this.commandMetadata = PicocliCommandMetadata.from(commandClass); + if (!(commandMetadata instanceof PicocliCommandMetadata)) + throw new IllegalStateException("CommandMetadata must be PicocliCommandMetadata for picocli commands"); + } + + @Override + public CommandMetadata metadata() + { + return commandMetadata; + } + + @Override + public Void execute(CommandExecutionArgs arguments, CommandExecutionContext context) + { + try + { + CommandLine commandLine = new CommandLine(commandClass, new InjectCassandraContext(context.output())); + AbstractCommand userCommand = commandLine.getCommand(); + + userCommand.probe(context.nodeProbe()); + PicocliCommandArgsConverter.toCommand(arguments, userCommand); + + // TODO utilize picocli to invoke the command with proper error handling? + userCommand.run(); + + // The picocli commands in Cassandra do not return any value, so we return null here. + // If a command has a different return type, e.g. implements Callable, we would need to handle that accordingly. + return null; + } + catch (Exception e) + { + throw new RuntimeException("Failed to execute command: " + commandClass.getSimpleName(), e); + } + } + + private static class InjectCassandraContext implements CommandLine.IFactory + { + private final Output output; + private final CommandLine.IFactory fallback; + + public InjectCassandraContext(Output output) + { + this.fallback = CommandLine.defaultFactory(); + this.output = output; + } + + @Override + public K create(Class cls) + { + try + { + K bean = this.fallback.create(cls); + Class beanClass = bean.getClass(); + do + { + Field[] fields = beanClass.getDeclaredFields(); + for (Field field : fields) + { + if (!field.isAnnotationPresent(Inject.class)) + continue; + + field.setAccessible(true); + if (field.getType().equals(Output.class)) + { + field.set(bean, output); + } + else + { + throw new RuntimeException("Unsupported injectable field type: " + field.getType() + + " in class " + beanClass.getName() + ". " + + "Only Output is supported for injection."); + } + } + } + while ((beanClass = beanClass.getSuperclass()) != null); + return bean; + } + catch (Exception e) + { + throw new CommandLine.InitializationException("Failed to create instance of " + cls, e); + } + } + } +} diff --git a/src/java/org/apache/cassandra/management/picocli/PicocliCommandArgsConverter.java b/src/java/org/apache/cassandra/management/picocli/PicocliCommandArgsConverter.java new file mode 100644 index 000000000000..5630668fb4ce --- /dev/null +++ b/src/java/org/apache/cassandra/management/picocli/PicocliCommandArgsConverter.java @@ -0,0 +1,201 @@ +/* + * 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.cassandra.management.picocli; + +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.Map; + +import org.apache.cassandra.management.SimpleCommandExecutionArgs; +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.api.CommandMetadata; +import org.apache.cassandra.management.api.OptionMetadata; +import org.apache.cassandra.management.api.ParameterMetadata; + +import picocli.CommandLine; +import picocli.CommandLine.Model.CommandSpec; + +public class PicocliCommandArgsConverter +{ + /** + * Extract command arguments from an AbstractCommand instance using picocli reflection. + * + * @param command the command instance to extract arguments from + * @return CommandExecutionArgs containing all options and parameters from the command + */ + public static CommandExecutionArgs fromCommand(T command) + { + CommandMetadata metadata = PicocliCommandMetadata.from(command); + PicocliCommandMetadata picocliMetadata = (PicocliCommandMetadata) metadata; + CommandSpec spec = picocliMetadata.getCommandSpec(); + + Map options = new LinkedHashMap<>(); + Map parameters = new LinkedHashMap<>(); + + for (CommandLine.Model.OptionSpec optionSpec : spec.options()) + { + if (!optionSpec.isOption()) + continue; + + Object value = optionSpec.getValue(); + if (value != null) + { + if (optionSpec.type() == boolean.class || optionSpec.type() == Boolean.class) + { + if (Boolean.TRUE.equals(value)) + options.put(new PicocliOptionMetadata(optionSpec), Boolean.TRUE); + } + else + { + Object defaultValue = optionSpec.defaultValue(); + if (!value.equals(defaultValue)) + options.put(new PicocliOptionMetadata(optionSpec), value); + } + } + } + + for (CommandLine.Model.PositionalParamSpec paramSpec : spec.positionalParameters()) + { + if (!paramSpec.isPositional()) + continue; + + Object value = paramSpec.getValue(); + if (value != null) + parameters.put(new PicocliParameterMetadata(paramSpec), value); + } + + return new SimpleCommandExecutionArgs(options, parameters); + } + + /** + * Populate an AbstractCommand instance with values from CommandExecutionArgs using picocli setters. + * + * @param args the CommandExecutionArgs containing values to set + * @param command the command instance to populate + */ + public static void toCommand(CommandExecutionArgs args, T command) + { + CommandMetadata metadata = PicocliCommandMetadata.from(command); + PicocliCommandMetadata picocliMetadata = (PicocliCommandMetadata) metadata; + CommandSpec spec = picocliMetadata.getCommandSpec(); + Map optionMap = buildOptionSpecMap(spec); + Map paramMap = buildParameterSpecMap(spec); + + for (Map.Entry entry : args.options().entrySet()) + { + OptionMetadata optionMetadata = entry.getKey(); + // This value is already converted to the correct type in CommandExecutionArgs. + Object convertedValue = entry.getValue(); + + if (convertedValue == null) + continue; + + CommandLine.Model.OptionSpec optionSpec = optionMap.get(optionMetadata); + if (optionSpec == null) + { + optionSpec = findOptionSpecByName(spec, optionMetadata.names()); + if (optionSpec == null) + throw new IllegalArgumentException("Option not found in command spec: " + Arrays.toString(optionMetadata.names())); + } + + try + { + optionSpec.setter().set(convertedValue); + } + catch (Exception e) + { + throw new RuntimeException("Failed to set option " + optionMetadata.names()[0] + + " with value " + convertedValue, e); + } + } + + for (Map.Entry entry : args.parameters().entrySet()) + { + ParameterMetadata paramMetadata = entry.getKey(); + Object value = entry.getValue(); + + if (value == null) + continue; + + CommandLine.Model.PositionalParamSpec paramSpec = paramMap.get(paramMetadata); + if (paramSpec == null) + { + paramSpec = findParameterSpecByIndex(spec, paramMetadata.index()); + if (paramSpec == null) + throw new IllegalArgumentException("Parameter not found in command spec at index: " + paramMetadata.index()); + } + + try + { + paramSpec.setter().set(value); + } + catch (Exception e) + { + throw new RuntimeException("Failed to set parameter at index " + paramMetadata.index() + + " with value " + value, e); + } + } + } + + private static Map buildOptionSpecMap(CommandSpec spec) + { + Map map = new LinkedHashMap<>(); + for (CommandLine.Model.OptionSpec optionSpec : spec.options()) + { + if (optionSpec.isOption()) + map.put(new PicocliOptionMetadata(optionSpec), optionSpec); + } + return map; + } + + private static Map buildParameterSpecMap(CommandSpec spec) + { + Map map = new LinkedHashMap<>(); + for (CommandLine.Model.PositionalParamSpec paramSpec : spec.positionalParameters()) + { + if (paramSpec.isPositional()) + map.put(new PicocliParameterMetadata(paramSpec), paramSpec); + } + return map; + } + + private static CommandLine.Model.OptionSpec findOptionSpecByName(CommandSpec spec, String[] names) + { + for (CommandLine.Model.OptionSpec optionSpec : spec.options()) + { + for (String name : names) + { + if (java.util.Arrays.asList(optionSpec.names()).contains(name)) + return optionSpec; + } + } + return null; + } + + private static CommandLine.Model.PositionalParamSpec findParameterSpecByIndex(CommandSpec spec, int index) + { + for (CommandLine.Model.PositionalParamSpec paramSpec : spec.positionalParameters()) + { + CommandLine.Range indexRange = paramSpec.index(); + if (index >= indexRange.min() && index <= indexRange.max()) + return paramSpec; + } + return null; + } +} diff --git a/src/java/org/apache/cassandra/management/picocli/PicocliCommandMetadata.java b/src/java/org/apache/cassandra/management/picocli/PicocliCommandMetadata.java new file mode 100644 index 000000000000..b3b15c6ad66c --- /dev/null +++ b/src/java/org/apache/cassandra/management/picocli/PicocliCommandMetadata.java @@ -0,0 +1,123 @@ +/* + * 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.cassandra.management.picocli; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import org.apache.cassandra.management.api.CommandMetadata; +import org.apache.cassandra.management.api.OptionMetadata; +import org.apache.cassandra.management.api.ParameterMetadata; + +import picocli.CommandLine; +import picocli.CommandLine.Model.CommandSpec; + +/** + * Implementation of CommandMetadata that extracts metadata from picocli CommandSpec. + */ +public class PicocliCommandMetadata implements CommandMetadata +{ + private final CommandSpec commandSpec; + + public PicocliCommandMetadata(CommandSpec commandSpec) + { + this.commandSpec = commandSpec; + } + + /** + * Create CommandMetadata from a command class. + */ + public static CommandMetadata from(Class commandClass) + { + CommandSpec spec = CommandSpec.forAnnotatedObject(commandClass); + return new PicocliCommandMetadata(spec); + } + + /** + * Create CommandMetadata from a command instance. + */ + public static CommandMetadata from(Object commandInstance) + { + CommandSpec spec = CommandSpec.forAnnotatedObject(commandInstance); + return new PicocliCommandMetadata(spec); + } + + @Override + public String name() + { + return commandSpec.name(); + } + + @Override + public String description() + { + String[] description = commandSpec.usageMessage().description(); + if (description == null || description.length == 0) + return ""; + return String.join("\n", description); + } + + @Override + public List aliases() + { + return List.of(commandSpec.aliases()); + } + + @Override + public List options() + { + List options = new ArrayList<>(); + for (CommandLine.Model.OptionSpec option : commandSpec.options()) + { + if (option.isOption()) + options.add(new PicocliOptionMetadata(option)); + } + return options; + } + + @Override + public List parameters() + { + List parameters = new ArrayList<>(); + for (CommandLine.Model.PositionalParamSpec positional : commandSpec.positionalParameters()) + { + if (positional.isPositional()) + parameters.add(new PicocliParameterMetadata(positional)); + } + return parameters; + } + + @Override + public List subcommands() + { + return commandSpec.subcommands().values().stream() + .map(subcommand -> new PicocliCommandMetadata(subcommand.getCommandSpec())) + .collect(Collectors.toList()); + } + + /** + * Get the underlying CommandSpec for advanced usage. + */ + public CommandSpec getCommandSpec() + { + return commandSpec; + } +} + diff --git a/src/java/org/apache/cassandra/management/picocli/PicocliCommandRegistryAdapter.java b/src/java/org/apache/cassandra/management/picocli/PicocliCommandRegistryAdapter.java new file mode 100644 index 000000000000..f16bc9870ce6 --- /dev/null +++ b/src/java/org/apache/cassandra/management/picocli/PicocliCommandRegistryAdapter.java @@ -0,0 +1,137 @@ +/* + * 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.cassandra.management.picocli; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.cassandra.management.api.Command; +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.api.CommandExecutionContext; +import org.apache.cassandra.management.api.CommandMetadata; +import org.apache.cassandra.management.api.CommandRegistry; +import org.apache.cassandra.tools.nodetool.AbstractCommand; + +import picocli.CommandLine; +import picocli.CommandLine.Model.CommandSpec; + +/** + * Adapter that wraps a picocli command with subcommands and implements CommandRegistry + * using the Composite pattern. This allows hierarchical picocli commands to be used + * as CommandRegistry instances. + *

+ * For example, {@code CompressionDictionaryCommandGroup} has subcommands (train, list, export, import), + * so it would be wrapped by this adapter to provide a CommandRegistry interface. + *

+ * The adapter recursively adapts subcommands: + *

    + *
  • If a subcommand has subcommands -> creates another {@code PicocliCommandRegistryAdapter}
  • + *
  • If a subcommand is a leaf -> creates a {@code PicocliCommandAdapter}
  • + *
+ */ +public class PicocliCommandRegistryAdapter implements CommandRegistry +{ + private final CommandSpec commandSpec; + private final CommandMetadata commandMetadata; + private final Map> subcommandMap = new ConcurrentHashMap<>(); + + /** + * Create an adapter for a picocli command class that has subcommands. + * @param commandClass the picocli command class (must have subcommands) + */ + public PicocliCommandRegistryAdapter(Class commandClass) + { + this.commandSpec = CommandSpec.forAnnotatedObject(commandClass); + this.commandMetadata = new PicocliCommandMetadata(commandSpec); + + if (commandSpec.subcommands().isEmpty()) + { + throw new IllegalArgumentException( + String.format("Command class %s does not have subcommands. " + + "Use PicocliCommandAdapter for leaf commands.", + commandClass.getName())); + } + + for (Map.Entry e : commandSpec.subcommands().entrySet()) + adaptSubcommands(e.getKey(), e.getValue()); + } + + /** + * Recursively adapt all subcommands from the picocli command. Uses a Composite pattern: + * subcommands with subcommands become registries, leaf subcommands become command adapters. + */ + private void adaptSubcommands(String commandName, CommandLine command) + { + CommandSpec subcommandSpec = command.getCommandSpec(); + Command adaptedCommand; + + if (!subcommandSpec.subcommands().isEmpty()) + { + adaptedCommand = new PicocliCommandRegistryAdapter(command.getCommand()); + } + else + { + Class subcommandClass = command.getCommand().getClass(); + if (!AbstractCommand.class.isAssignableFrom(subcommandClass)) + { + throw new IllegalArgumentException( + String.format("Subcommand class %s is not an AbstractCommand and cannot be adapted. " + + "Only AbstractCommand subclasses are supported for leaf commands.", + subcommandClass.getName())); + } + + adaptedCommand = new PicocliCommandAdapter((Class) subcommandClass); + } + + subcommandMap.put(commandName, adaptedCommand); + + for (String alias : subcommandSpec.aliases()) + subcommandMap.putIfAbsent(alias, adaptedCommand); + } + + @Override + public CommandMetadata metadata() + { + return commandMetadata; + } + + @Override + public String execute(CommandExecutionArgs arguments, CommandExecutionContext context) + { + // CommandRegistry is not directly executable: routing happens at the invoker/CQL layer. + throw new UnsupportedOperationException( + String.format("CommandRegistry '%s' is not directly executable. " + + "Specify a subcommand. Available commands: %s", + name(), + String.join(", ", subcommandMap.keySet()))); + } + + @Override + public Command command(String name) + { + return subcommandMap.get(name); + } + + @Override + public Iterable>> commands() + { + return subcommandMap.entrySet(); + } +} + diff --git a/src/java/org/apache/cassandra/management/picocli/PicocliCommandsProvider.java b/src/java/org/apache/cassandra/management/picocli/PicocliCommandsProvider.java new file mode 100644 index 000000000000..129a684c2a09 --- /dev/null +++ b/src/java/org/apache/cassandra/management/picocli/PicocliCommandsProvider.java @@ -0,0 +1,63 @@ +/* + * 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.cassandra.management.picocli; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.apache.cassandra.management.api.Command; +import org.apache.cassandra.management.api.CommandsProvider; +import org.apache.cassandra.tools.nodetool.AbstractCommand; +import org.apache.cassandra.tools.nodetool.NodetoolCommand; + +import picocli.CommandLine; + +public class PicocliCommandsProvider implements CommandsProvider +{ + @Override + public Collection> commands() + { + CommandLine commandLine = new CommandLine(NodetoolCommand.class); + List> commands = new ArrayList<>(); + + commandLine.getSubcommands().forEach((name, subcommandLine) -> { + if (!subcommandLine.getCommandSpec().subcommands().isEmpty()) + { + @SuppressWarnings("unchecked") + Class abstractCommandClass = + (Class) subcommandLine.getCommand().getClass(); + commands.add(new PicocliCommandRegistryAdapter(abstractCommandClass)); + } + else + { + Class commandClass = subcommandLine.getCommand().getClass(); + if (AbstractCommand.class.isAssignableFrom(commandClass)) + { + @SuppressWarnings("unchecked") + Class abstractCommandClass = + (Class) commandClass; + commands.add(new PicocliCommandAdapter(abstractCommandClass)); + } + } + }); + + return commands; + } +} diff --git a/src/java/org/apache/cassandra/management/picocli/PicocliMetadataExtractor.java b/src/java/org/apache/cassandra/management/picocli/PicocliMetadataExtractor.java new file mode 100644 index 000000000000..0a5c71bf51a4 --- /dev/null +++ b/src/java/org/apache/cassandra/management/picocli/PicocliMetadataExtractor.java @@ -0,0 +1,53 @@ +/* + * 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.cassandra.management.picocli; + +import org.apache.cassandra.management.api.CommandMetadata; +import org.apache.cassandra.tools.nodetool.AbstractCommand; + +/** + * Utility class for extracting command metadata from picocli-annotated command classes. + */ +public class PicocliMetadataExtractor +{ + /** + * Extract CommandMetadata from an AbstractCommand class. + * + * @param commandClass The command class annotated with picocli @Command + * @return CommandMetadata extracted from the command class + */ + public static CommandMetadata extract(Class commandClass) + { + return PicocliCommandMetadata.from(commandClass); + } + + /** + * Extract CommandMetadata from a command instance. + * + * @param commandInstance The command instance + * @return CommandMetadata extracted from the command instance + */ + public static CommandMetadata extract(Object commandInstance) + { + if (commandInstance instanceof AbstractCommand) + return PicocliCommandMetadata.from(commandInstance); + throw new IllegalArgumentException("Unsupported command instance type: " + commandInstance.getClass().getName()); + } +} + diff --git a/src/java/org/apache/cassandra/management/picocli/PicocliOptionMetadata.java b/src/java/org/apache/cassandra/management/picocli/PicocliOptionMetadata.java new file mode 100644 index 000000000000..909699391810 --- /dev/null +++ b/src/java/org/apache/cassandra/management/picocli/PicocliOptionMetadata.java @@ -0,0 +1,141 @@ +/* + * 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.cassandra.management.picocli; + +import com.google.common.base.Strings; + +import org.apache.cassandra.management.api.OptionMetadata; + +import picocli.CommandLine; +import picocli.CommandLine.Model.OptionSpec; + +import static org.apache.cassandra.management.ManagementUtils.stripAngleBrackets; + +/** + * Implementation of OptionMetadata that extracts metadata from picocli OptionSpec. + */ +public class PicocliOptionMetadata implements OptionMetadata +{ + private final OptionSpec optionSpec; + + public PicocliOptionMetadata(OptionSpec optionSpec) + { + this.optionSpec = optionSpec; + } + + @Override + public String[] names() + { + return optionSpec.names(); + } + + @Override + public String description() + { + String[] description = optionSpec.description(); + if (description == null || description.length == 0) + return ""; + return String.join("\n", description); + } + + @Override + public Class type() + { + return optionSpec.type(); + } + + @Override + public String defaultValue() + { + Object defaultValue = optionSpec.defaultValue(); + if (defaultValue == null) + { + if (optionSpec.type() == boolean.class || optionSpec.type() == Boolean.class) + return "false"; + return ""; + } + + return defaultValue.toString(); + } + + @Override + public boolean required() + { + return optionSpec.required(); + } + + @Override + public String arity() + { + CommandLine.Range arity = optionSpec.arity(); + int min = arity.min(); + int max = arity.max(); + + if (min == 0 && max == 0) + return "0"; + + if (min == max) + return String.valueOf(min); + else if (max == Integer.MAX_VALUE) + return min + "..*"; + else + return min + ".." + max; + } + + @Override + public String paramLabel() + { + String paramLabel = optionSpec.paramLabel(); + Object userObject = optionSpec.userObject(); + + return Strings.isNullOrEmpty(paramLabel) ? + ((java.lang.reflect.Field) userObject).getName() : + stripAngleBrackets(paramLabel); + } + + /** + * Check if this is a boolean flag option. + */ + public boolean isFlag() + { + return optionSpec.arity().max() == 0; + } + + /** + * Get the underlying OptionSpec for advanced usage. + */ + public OptionSpec getOptionSpec() + { + return optionSpec; + } + + public Object convertValue(Object value) throws Exception + { + TypeConverter customConverter = typeConverter() == null ? null : typeConverter()[0]; + return customConverter == null ? + TypeConverterRegistry.convertValueBasic(value, type()) : + customConverter.convert(value.toString()); + } + + private TypeConverter[] typeConverter() + { + return TypeConverter.createFrom(optionSpec); + } +} + diff --git a/src/java/org/apache/cassandra/management/picocli/PicocliParameterMetadata.java b/src/java/org/apache/cassandra/management/picocli/PicocliParameterMetadata.java new file mode 100644 index 000000000000..6d3ca8d64d5e --- /dev/null +++ b/src/java/org/apache/cassandra/management/picocli/PicocliParameterMetadata.java @@ -0,0 +1,132 @@ +/* + * 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.cassandra.management.picocli; + +import org.apache.cassandra.management.api.ParameterMetadata; + +import picocli.CommandLine; +import picocli.CommandLine.Model.PositionalParamSpec; + +import static org.apache.cassandra.management.ManagementUtils.stripAngleBrackets; + +/** + * Implementation of ParameterMetadata that extracts metadata from picocli PositionalParamSpec. + */ +public class PicocliParameterMetadata implements ParameterMetadata +{ + private final PositionalParamSpec positionalParamSpec; + + public PicocliParameterMetadata(PositionalParamSpec positionalParamSpec) + { + this.positionalParamSpec = positionalParamSpec; + } + + @Override + public String[] names() + { + String name; + String paramLabel = positionalParamSpec.paramLabel(); + Object userObject = positionalParamSpec.userObject(); + + if (paramLabel != null && !paramLabel.isEmpty()) + name = stripAngleBrackets(paramLabel); + else if (userObject instanceof java.lang.reflect.Field) + name = ((java.lang.reflect.Field) userObject).getName(); + else + name = COMMAND_POSITIONAL_PARAM_PREFIX + index(); + + return new String[] { name }; + } + + @Override + public String paramLabel() + { + return stripAngleBrackets(positionalParamSpec.paramLabel()); + } + + @Override + public String description() + { + String[] description = positionalParamSpec.description(); + if (description == null || description.length == 0) + return ""; + return String.join("\n", description); + } + + @Override + public Class type() + { + return positionalParamSpec.type(); + } + + @Override + public String defaultValue() + { + return ""; + } + + @Override + public int index() + { + return positionalParamSpec.index().min(); + } + + @Override + public boolean required() + { + return positionalParamSpec.arity().min() > 0; + } + + @Override + public String arity() + { + CommandLine.Range arity = positionalParamSpec.arity(); + int min = arity.min(); + int max = arity.max(); + + if (min == max) + return String.valueOf(min); + else if (max == Integer.MAX_VALUE) + return min + "..*"; + else + return min + ".." + max; + } + + /** + * Get the underlying PositionalParamSpec for advanced usage. + */ + public PositionalParamSpec getPositionalParamSpec() + { + return positionalParamSpec; + } + + public Object convertValue(Object value) throws Exception + { + TypeConverter customConverter = typeConverter() == null ? null : typeConverter()[0]; + return customConverter == null ? + TypeConverterRegistry.convertValueBasic(value, type()) : + customConverter.convert(value.toString()); + } + + private TypeConverter[] typeConverter() + { + return TypeConverter.createFrom(positionalParamSpec); + } +} + diff --git a/src/java/org/apache/cassandra/management/picocli/TypeConverter.java b/src/java/org/apache/cassandra/management/picocli/TypeConverter.java new file mode 100644 index 000000000000..5132591b0e51 --- /dev/null +++ b/src/java/org/apache/cassandra/management/picocli/TypeConverter.java @@ -0,0 +1,38 @@ +/* + * 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.cassandra.management.picocli; + +import java.util.Arrays; + +import picocli.CommandLine; + +public interface TypeConverter +{ + R convert(String value) throws Exception; + + static TypeConverter[] createFrom(CommandLine.Model.ArgSpec argSpec) + { + CommandLine.ITypeConverter[] converters = argSpec.converters(); + return converters == null || converters.length == 0 ? + null : + Arrays.stream(converters) + .map(c -> (TypeConverter) c::convert) + .toArray(TypeConverter[]::new); + } +} diff --git a/src/java/org/apache/cassandra/management/picocli/TypeConverterRegistry.java b/src/java/org/apache/cassandra/management/picocli/TypeConverterRegistry.java new file mode 100644 index 000000000000..3f5dd3601863 --- /dev/null +++ b/src/java/org/apache/cassandra/management/picocli/TypeConverterRegistry.java @@ -0,0 +1,150 @@ +/* + * 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.cassandra.management.picocli; + +import java.lang.reflect.Array; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; + +public class TypeConverterRegistry +{ + private static final Map, TypeConverter> converters = new LinkedHashMap<>(); + + static + { + register(boolean.class, (TypeConverter) Boolean::parseBoolean); + register(int.class, (TypeConverter) Integer::parseInt); + register(long.class, (TypeConverter) Long::parseLong); + register(double.class, (TypeConverter) Double::parseDouble); + register(float.class, (TypeConverter) Float::parseFloat); + register(byte.class, (TypeConverter) Byte::parseByte); + register(short.class, (TypeConverter) Short::parseShort); + register(char.class, value -> { + if (value.length() != 1) + throw new IllegalArgumentException("Cannot convert to char: " + value); + return value.charAt(0); + }); + register(Boolean.class, (TypeConverter) Boolean::parseBoolean); + register(Integer.class, (TypeConverter) Integer::parseInt); + register(Long.class, (TypeConverter) Long::parseLong); + register(Double.class, (TypeConverter) Double::parseDouble); + register(Float.class, (TypeConverter) Float::parseFloat); + register(Byte.class, (TypeConverter) Byte::parseByte); + register(Short.class, (TypeConverter) Short::parseShort); + register(String.class, (TypeConverter) value -> value); + register(Character.class, value -> { + if (value.length() != 1) + throw new IllegalArgumentException("Cannot convert to Character: " + value); + return value.charAt(0); + }); + register(Object.class, value -> value); + } + + private static void register(Class type, TypeConverter converter) + { + converters.put(type, converter); + } + + public static Object convertValueBasic(Object value, Class targetType) throws Exception + { + if (value == null) + return null; + + if (targetType.isAssignableFrom(value.getClass())) + return value; + + String stringValue = value.toString(); + if (isBasicType(targetType)) + { + TypeConverter converter = converters.get(targetType); + if (converter == null) + throw new IllegalStateException(String.format("No converter registered for basic type: %s", + targetType.getName())); + return converter.convert(stringValue); + } + + if (targetType.isArray() || Collection.class.isAssignableFrom(targetType)) + return convertToArrayOrCollection(value, targetType); + + throw new IllegalArgumentException(String.format("No converter available for type: %s.", targetType.getName())); + } + + private static Object convertToArrayOrCollection(Object value, Class targetType) throws Exception + { + Collection sourceCollection; + if (value instanceof Collection) + sourceCollection = (Collection) value; + else if (value.getClass().isArray()) + { + int length = Array.getLength(value); + List list = new ArrayList<>(length); + for (int i = 0; i < length; i++) + list.add(Array.get(value, i)); + sourceCollection = list; + } + else + throw new IllegalArgumentException("Cannot convert " + value.getClass() + " to " + targetType); + + if (targetType.isArray()) + { + Class componentType = targetType.getComponentType(); + Object targetArray = Array.newInstance(componentType, sourceCollection.size()); + int index = 0; + for (Object item : sourceCollection) + { + Object convertedItem = convertValueBasic(item, componentType); + Array.set(targetArray, index++, convertedItem); + } + return targetArray; + } + + Collection targetCollection; + if (targetType == List.class || List.class.isAssignableFrom(targetType)) + targetCollection = new ArrayList<>(); + else if (targetType == Set.class || Set.class.isAssignableFrom(targetType)) + targetCollection = new LinkedHashSet<>(); + else if (SortedSet.class.isAssignableFrom(targetType)) + targetCollection = new TreeSet<>(); + else + targetCollection = new ArrayList<>(); + + targetCollection.addAll(sourceCollection); + return targetCollection; + } + + private static boolean isBasicType(Class type) + { + return type == String.class || + type == boolean.class || type == Boolean.class || + type == int.class || type == Integer.class || + type == long.class || type == Long.class || + type == double.class || type == Double.class || + type == float.class || type == Float.class || + type == byte.class || type == Byte.class || + type == short.class || type == Short.class || + type == char.class || type == Character.class; + } +} diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index 58f69a179347..7450315dd45f 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -76,6 +76,7 @@ import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Locator; +import org.apache.cassandra.management.CommandService; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.DefaultNameFactory; import org.apache.cassandra.net.StartupClusterConnectivityChecker; @@ -383,6 +384,9 @@ protected void setup() // Prepared statements QueryProcessor.instance.preloadPreparedStatements(); + // Initialize command service (after JMX, before StorageService.initServer) + CommandService.instance.start(); + // start server internals StorageService.instance.registerDaemon(this); try @@ -721,6 +725,7 @@ public void stop() // jsvc takes care of taking the rest down logger.info("Cassandra shutting down..."); destroyClientTransports(); + CommandService.instance.stop(); StorageService.instance.setRpcReady(false); if (jmxServer != null) diff --git a/src/java/org/apache/cassandra/tools/INodeProbeFactory.java b/src/java/org/apache/cassandra/tools/INodeProbeFactory.java index fec4a2b9bf6b..a5bf732495e0 100644 --- a/src/java/org/apache/cassandra/tools/INodeProbeFactory.java +++ b/src/java/org/apache/cassandra/tools/INodeProbeFactory.java @@ -32,11 +32,11 @@ class NodeProbeFactory implements INodeProbeFactory public NodeProbe create(String host, int port) throws IOException { - return new NodeProbe(host, port); + return new NodeProbe(new RemoteJmxMBeanAccessor(host, port)); } public NodeProbe create(String host, int port, String username, String password) throws IOException { - return new NodeProbe(host, port, username, password); + return new NodeProbe(new RemoteJmxMBeanAccessor(host, port, username, password)); } } diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index aa98d2218ed5..83579e6d7a1c 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -19,21 +19,16 @@ import java.io.IOException; import java.io.PrintStream; -import java.lang.management.ManagementFactory; import java.lang.management.MemoryMXBean; import java.lang.management.MemoryUsage; import java.lang.management.RuntimeMXBean; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; import java.net.InetAddress; import java.net.UnknownHostException; -import java.rmi.ConnectException; -import java.rmi.server.RMIClientSocketFactory; -import java.rmi.server.RMISocketFactory; -import java.util.AbstractMap; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -45,17 +40,10 @@ import javax.annotation.Nullable; import javax.management.InstanceNotFoundException; -import javax.management.JMX; -import javax.management.MBeanServerConnection; import javax.management.MalformedObjectNameException; -import javax.management.ObjectName; import javax.management.openmbean.CompositeData; import javax.management.openmbean.OpenDataException; import javax.management.openmbean.TabularData; -import javax.management.remote.JMXConnector; -import javax.management.remote.JMXConnectorFactory; -import javax.management.remote.JMXServiceURL; -import javax.rmi.ssl.SslRMIClientSocketFactory; import com.google.common.base.Function; import com.google.common.base.Strings; @@ -67,43 +55,29 @@ import com.google.common.collect.Sets; import com.google.common.util.concurrent.Uninterruptibles; -import org.apache.cassandra.audit.AuditLogManager; import org.apache.cassandra.audit.AuditLogManagerMBean; import org.apache.cassandra.audit.AuditLogOptions; import org.apache.cassandra.audit.AuditLogOptionsCompositeData; -import org.apache.cassandra.auth.AuthCache; import org.apache.cassandra.auth.AuthCacheMBean; -import org.apache.cassandra.auth.CIDRGroupsMappingManager; import org.apache.cassandra.auth.CIDRGroupsMappingManagerMBean; -import org.apache.cassandra.auth.CIDRPermissionsManager; import org.apache.cassandra.auth.CIDRPermissionsManagerMBean; -import org.apache.cassandra.auth.NetworkPermissionsCache; import org.apache.cassandra.auth.NetworkPermissionsCacheMBean; import org.apache.cassandra.auth.PasswordAuthenticator; -import org.apache.cassandra.auth.PermissionsCache; import org.apache.cassandra.auth.PermissionsCacheMBean; -import org.apache.cassandra.auth.RolesCache; import org.apache.cassandra.auth.RolesCacheMBean; import org.apache.cassandra.auth.jmx.AuthorizationProxy; -import org.apache.cassandra.batchlog.BatchlogManager; import org.apache.cassandra.batchlog.BatchlogManagerMBean; import org.apache.cassandra.db.ColumnFamilyStoreMBean; -import org.apache.cassandra.db.compaction.CompactionManager; import org.apache.cassandra.db.compaction.CompactionManagerMBean; import org.apache.cassandra.db.compression.CompressionDictionaryDetailsTabularData; import org.apache.cassandra.db.compression.CompressionDictionaryManagerMBean; import org.apache.cassandra.db.compression.TrainingState; -import org.apache.cassandra.db.guardrails.Guardrails; import org.apache.cassandra.db.guardrails.GuardrailsMBean; -import org.apache.cassandra.db.virtual.CIDRFilteringMetricsTable; import org.apache.cassandra.db.virtual.CIDRFilteringMetricsTableMBean; import org.apache.cassandra.fql.FullQueryLoggerOptions; import org.apache.cassandra.fql.FullQueryLoggerOptionsCompositeData; -import org.apache.cassandra.gms.FailureDetector; import org.apache.cassandra.gms.FailureDetectorMBean; -import org.apache.cassandra.gms.Gossiper; import org.apache.cassandra.gms.GossiperMBean; -import org.apache.cassandra.hints.HintsService; import org.apache.cassandra.hints.HintsServiceMBean; import org.apache.cassandra.index.sai.metrics.IndexGroupMetrics; import org.apache.cassandra.index.sai.metrics.TableQueryMetrics; @@ -111,57 +85,82 @@ import org.apache.cassandra.locator.DynamicEndpointSnitchMBean; import org.apache.cassandra.locator.EndpointSnitchInfoMBean; import org.apache.cassandra.locator.LocationInfoMBean; -import org.apache.cassandra.metrics.CIDRAuthorizerMetrics; +import org.apache.cassandra.management.MBeanAccessor; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.StorageMetrics; import org.apache.cassandra.metrics.TableMetrics; import org.apache.cassandra.metrics.ThreadPoolMetrics; -import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.MessagingServiceMBean; import org.apache.cassandra.service.ActiveRepairServiceMBean; -import org.apache.cassandra.service.AutoRepairService; import org.apache.cassandra.service.AutoRepairServiceMBean; -import org.apache.cassandra.service.CacheService; import org.apache.cassandra.service.CacheServiceMBean; -import org.apache.cassandra.service.GCInspector; import org.apache.cassandra.service.GCInspectorMXBean; -import org.apache.cassandra.service.StorageProxy; import org.apache.cassandra.service.StorageProxyMBean; import org.apache.cassandra.service.StorageServiceMBean; -import org.apache.cassandra.service.accord.AccordOperations; import org.apache.cassandra.service.accord.AccordOperationsMBean; import org.apache.cassandra.service.snapshot.SnapshotManagerMBean; import org.apache.cassandra.streaming.StreamManagerMBean; import org.apache.cassandra.streaming.StreamState; import org.apache.cassandra.streaming.management.StreamStateCompositeData; -import org.apache.cassandra.tcm.CMSOperations; import org.apache.cassandra.tcm.CMSOperationsMBean; import org.apache.cassandra.tools.RepairRunner.RepairCmd; import org.apache.cassandra.tools.nodetool.GetTimeout; import org.apache.cassandra.utils.NativeLibrary; -import static org.apache.cassandra.config.CassandraRelevantProperties.NODETOOL_JMX_NOTIFICATION_POLL_INTERVAL_SECONDS; -import static org.apache.cassandra.config.CassandraRelevantProperties.SSL_ENABLE; - /** - * JMX client operations for Cassandra. + * A client wrapper (or Receiver in the command pattern) for performing administrative + * and monitoring operations on a Cassandra node. + *

+ * NodeProbe provides a high-level interface to interact with a Cassandra node through JMX. + * It abstracts the complexity of JMX connections and MBean lookups, providing convenient methods for common + * administrative tasks such as compaction, repair, snapshots, and cluster management. + *

+ * This class is primarily used by command-line tools like {@code nodetool} to execute administrative + * commands against remote or local Cassandra nodes. It can work with both remote JMX connections (via + * {@link RemoteJmxMBeanAccessor}) and local in-process accessors for server-side execution. + * + *

Execution Modes

+ *

Client-Side Execution (up to 5.x)

+ *

+ * When used with {@link RemoteJmxMBeanAccessor}, NodeProbe connects to a remote Cassandra node via JMX: + *

{@code
+ * MBeanAccessor accessor = new RemoteJmxMBeanAccessor("localhost", 7199);
+ * try (NodeProbe probe = new NodeProbe(accessor)) {
+ *     probe.forceKeyspaceCleanup(System.out, 1, "mykeyspace");
+ * }
+ * }
+ * + *

Server-Side Execution (CEP-38)

+ *

+ * As part of CEP-38, NodeProbe can be + * used for server-side command execution through the management API. In this mode, NodeProbe operates in-process + * using a local {@link MBeanAccessor} that directly accesses the platform MBean server, eliminating the need for + * remote JMX connections. + * + *

Server-side execution provides several advantages: + *

    + *
  • No Network Overhead: Direct in-process access to MBeans without JMX/RMI overhead
  • + *
  • Better Performance: No serialization/deserialization of JMX calls
  • + *
  • Unified API: Same NodeProbe interface works for both client and server-side execution
  • + *
  • Management API Integration: Commands can be executed via {@link org.apache.cassandra.management.CommandService} and exposed + * through various protocols (native protocol, REST, etc.)
  • + *
+ * + *

Thread Safety

+ *

NodeProbe instances are not thread-safe. Each thread should use its own instance, or external + * synchronization must be provided when sharing instances across threads. + * + *

Lazy Initialization

+ *

MBean proxies are initialized lazily on first access to reduce overhead for operations that + * don't require all MBeans. This is particularly beneficial for testing scenarios where only a + * subset of functionality is needed. + * + * @see MBeanAccessor + * @see RemoteJmxMBeanAccessor + * @see org.apache.cassandra.management.CommandService */ public class NodeProbe implements AutoCloseable { - private static final String fmtUrl = "service:jmx:rmi:///jndi/rmi://%s:%d/jmxrmi"; - private static final String ssObjName = "org.apache.cassandra.db:type=StorageService"; - public static final int defaultPort = 7199; - - static long JMX_NOTIFICATION_POLL_INTERVAL_SECONDS = NODETOOL_JMX_NOTIFICATION_POLL_INTERVAL_SECONDS.getLong(); - - final String host; - final int port; - private String username; - private String password; - - - protected JMXConnector jmxc; - protected MBeanServerConnection mbeanServerConn; protected CompactionManagerMBean compactionProxy; protected StorageServiceMBean ssProxy; protected SnapshotManagerMBean snapshotProxy; @@ -189,187 +188,66 @@ public class NodeProbe implements AutoCloseable protected RolesCacheMBean rcProxy; protected AutoRepairServiceMBean autoRepairProxy; protected GuardrailsMBean grProxy; - protected volatile Output output; - protected CIDRFilteringMetricsTableMBean cfmProxy; - /** - * Creates a NodeProbe using the specified JMX host, port, username, and password. - * - * @param host hostname or IP address of the JMX agent - * @param port TCP port of the remote JMX agent - * @throws IOException on connection failures - */ - public NodeProbe(String host, int port, String username, String password) throws IOException - { - assert username != null && !username.isEmpty() && password != null && !password.isEmpty() - : "neither username nor password can be blank"; - - this.host = host; - this.port = port; - this.username = username; - this.password = password; - this.output = Output.CONSOLE; - connect(); - } + private final MBeanAccessor mBeanAccessor; + protected volatile Output output; /** - * Creates a NodeProbe using the specified JMX host and port. - * - * @param host hostname or IP address of the JMX agent - * @param port TCP port of the remote JMX agent - * @throws IOException on connection failures + * Creates a NodeProbe using the specified MBeanAccessor. + * @param mBeanAccessor the provider to use for obtaining MBeans. */ - public NodeProbe(String host, int port) throws IOException + public NodeProbe(MBeanAccessor mBeanAccessor) { - this.host = host; - this.port = port; - this.output = Output.CONSOLE; - connect(); + this(mBeanAccessor, Output.CONSOLE); } - /** - * Creates a NodeProbe using the specified JMX host and default port. - * - * @param host hostname or IP address of the JMX agent - * @throws IOException on connection failures - */ - public NodeProbe(String host) throws IOException + public NodeProbe(MBeanAccessor mBeanAccessor, Output output) { - this.host = host; - this.port = defaultPort; - this.output = Output.CONSOLE; - connect(); + this.mBeanAccessor = mBeanAccessor; + this.output = output; + lazyInitMBeans(); } - protected NodeProbe() + public void close() throws Exception { - // this constructor is only used for extensions to rewrite their own connect method - this.host = ""; - this.port = 0; - this.output = Output.CONSOLE; + mBeanAccessor.close(); } /** - * Create a connection to the JMX agent and setup the M[X]Bean proxies. - * - * @throws IOException on connection failures + * Initialize all MBeans lazily, so that tests that don't need them + * don't pay the cost of creating all the proxies. */ - protected void connect() throws IOException - { - String host = this.host; - if (host.contains(":")) - { - // Use square brackets to surround IPv6 addresses to fix CASSANDRA-7669 and CASSANDRA-17581 - host = "[" + host + "]"; - } - JMXServiceURL jmxUrl = new JMXServiceURL(String.format(fmtUrl, host, port)); - Map env = new HashMap(); - if (username != null) - { - String[] creds = { username, password }; - env.put(JMXConnector.CREDENTIALS, creds); - } - - env.put("com.sun.jndi.rmi.factory.socket", getRMIClientSocketFactory()); - - jmxc = JMXConnectorFactory.connect(jmxUrl, env); - mbeanServerConn = jmxc.getMBeanServerConnection(); - - try - { - ObjectName name = new ObjectName(ssObjName); - ssProxy = JMX.newMBeanProxy(mbeanServerConn, name, StorageServiceMBean.class); - name = new ObjectName(SnapshotManagerMBean.MBEAN_NAME); - snapshotProxy = JMX.newMBeanProxy(mbeanServerConn, name, SnapshotManagerMBean.class); - name = new ObjectName(CMSOperations.MBEAN_OBJECT_NAME); - cmsProxy = JMX.newMBeanProxy(mbeanServerConn, name, CMSOperationsMBean.class); - name = new ObjectName(AccordOperations.MBEAN_OBJECT_NAME); - accordProxy = JMX.newMBeanProxy(mbeanServerConn, name, AccordOperationsMBean.class); - name = new ObjectName(MessagingService.MBEAN_NAME); - msProxy = JMX.newMBeanProxy(mbeanServerConn, name, MessagingServiceMBean.class); - name = new ObjectName(StreamManagerMBean.OBJECT_NAME); - streamProxy = JMX.newMBeanProxy(mbeanServerConn, name, StreamManagerMBean.class); - name = new ObjectName(CompactionManager.MBEAN_OBJECT_NAME); - compactionProxy = JMX.newMBeanProxy(mbeanServerConn, name, CompactionManagerMBean.class); - name = new ObjectName(FailureDetector.MBEAN_NAME); - fdProxy = JMX.newMBeanProxy(mbeanServerConn, name, FailureDetectorMBean.class); - name = new ObjectName(CacheService.MBEAN_NAME); - cacheService = JMX.newMBeanProxy(mbeanServerConn, name, CacheServiceMBean.class); - name = new ObjectName(StorageProxy.MBEAN_NAME); - spProxy = JMX.newMBeanProxy(mbeanServerConn, name, StorageProxyMBean.class); - name = new ObjectName(HintsService.MBEAN_NAME); - hsProxy = JMX.newMBeanProxy(mbeanServerConn, name, HintsServiceMBean.class); - name = new ObjectName(GCInspector.MBEAN_NAME); - gcProxy = JMX.newMBeanProxy(mbeanServerConn, name, GCInspectorMXBean.class); - name = new ObjectName(Gossiper.MBEAN_NAME); - gossProxy = JMX.newMBeanProxy(mbeanServerConn, name, GossiperMBean.class); - name = new ObjectName(BatchlogManager.MBEAN_NAME); - bmProxy = JMX.newMBeanProxy(mbeanServerConn, name, BatchlogManagerMBean.class); - name = new ObjectName(ActiveRepairServiceMBean.MBEAN_NAME); - arsProxy = JMX.newMBeanProxy(mbeanServerConn, name, ActiveRepairServiceMBean.class); - name = new ObjectName(AuditLogManager.MBEAN_NAME); - almProxy = JMX.newMBeanProxy(mbeanServerConn, name, AuditLogManagerMBean.class); - name = new ObjectName(AuthCache.MBEAN_NAME_BASE + PasswordAuthenticator.CredentialsCacheMBean.CACHE_NAME); - ccProxy = JMX.newMBeanProxy(mbeanServerConn, name, PasswordAuthenticator.CredentialsCacheMBean.class); - name = new ObjectName(AuthCache.MBEAN_NAME_BASE + AuthorizationProxy.JmxPermissionsCacheMBean.CACHE_NAME); - jpcProxy = JMX.newMBeanProxy(mbeanServerConn, name, AuthorizationProxy.JmxPermissionsCacheMBean.class); - - name = new ObjectName(AuthCache.MBEAN_NAME_BASE + NetworkPermissionsCache.CACHE_NAME); - npcProxy = JMX.newMBeanProxy(mbeanServerConn, name, NetworkPermissionsCacheMBean.class); - - name = new ObjectName(AuthCache.MBEAN_NAME_BASE + PermissionsCache.CACHE_NAME); - pcProxy = JMX.newMBeanProxy(mbeanServerConn, name, PermissionsCacheMBean.class); - - name = new ObjectName(AuthCache.MBEAN_NAME_BASE + RolesCache.CACHE_NAME); - rcProxy = JMX.newMBeanProxy(mbeanServerConn, name, RolesCacheMBean.class); - - name = new ObjectName(CIDRPermissionsManager.MBEAN_NAME); - cpbProxy = JMX.newMBeanProxy(mbeanServerConn, name, CIDRPermissionsManagerMBean.class); - - name = new ObjectName(CIDRGroupsMappingManager.MBEAN_NAME); - cmbProxy = JMX.newMBeanProxy(mbeanServerConn, name, CIDRGroupsMappingManagerMBean.class); - - name = new ObjectName(CIDRFilteringMetricsTable.MBEAN_NAME); - cfmProxy = JMX.newMBeanProxy(mbeanServerConn, name, CIDRFilteringMetricsTableMBean.class); - - name = new ObjectName(AutoRepairService.MBEAN_NAME); - autoRepairProxy = JMX.newMBeanProxy(mbeanServerConn, name, AutoRepairServiceMBean.class); - - name = new ObjectName(Guardrails.MBEAN_NAME); - grProxy = JMX.newMBeanProxy(mbeanServerConn, name, GuardrailsMBean.class); - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException( - "Invalid ObjectName? Please report this as a bug.", e); - } - - memProxy = ManagementFactory.newPlatformMXBeanProxy(mbeanServerConn, - ManagementFactory.MEMORY_MXBEAN_NAME, MemoryMXBean.class); - runtimeProxy = ManagementFactory.newPlatformMXBeanProxy( - mbeanServerConn, ManagementFactory.RUNTIME_MXBEAN_NAME, RuntimeMXBean.class); - } - - private RMIClientSocketFactory getRMIClientSocketFactory() - { - if (SSL_ENABLE.getBoolean()) - return new SslRMIClientSocketFactory(); - else - return RMISocketFactory.getDefaultSocketFactory(); - } - - public void close() throws IOException - { - try - { - jmxc.close(); - } - catch (ConnectException e) - { - // result of 'stopdaemon' command - i.e. if close() call fails, the daemon is shutdown - System.out.println("Cassandra has shutdown."); - } + private void lazyInitMBeans() + { + ssProxy = LazyMBeanProxy.create(mBeanAccessor, StorageServiceMBean.class); + compactionProxy = LazyMBeanProxy.create(mBeanAccessor, CompactionManagerMBean.class); + snapshotProxy = LazyMBeanProxy.create(mBeanAccessor, SnapshotManagerMBean.class); + cmsProxy = LazyMBeanProxy.create(mBeanAccessor, CMSOperationsMBean.class); + accordProxy = LazyMBeanProxy.create(mBeanAccessor, AccordOperationsMBean.class); + gossProxy = LazyMBeanProxy.create(mBeanAccessor, GossiperMBean.class); + memProxy = LazyMBeanProxy.create(mBeanAccessor, MemoryMXBean.class); + gcProxy = LazyMBeanProxy.create(mBeanAccessor, GCInspectorMXBean.class); + runtimeProxy = LazyMBeanProxy.create(mBeanAccessor, RuntimeMXBean.class); + streamProxy = LazyMBeanProxy.create(mBeanAccessor, StreamManagerMBean.class); + msProxy = LazyMBeanProxy.create(mBeanAccessor, MessagingServiceMBean.class); + fdProxy = LazyMBeanProxy.create(mBeanAccessor, FailureDetectorMBean.class); + cacheService = LazyMBeanProxy.create(mBeanAccessor, CacheServiceMBean.class); + spProxy = LazyMBeanProxy.create(mBeanAccessor, StorageProxyMBean.class); + hsProxy = LazyMBeanProxy.create(mBeanAccessor, HintsServiceMBean.class); + bmProxy = LazyMBeanProxy.create(mBeanAccessor, BatchlogManagerMBean.class); + arsProxy = LazyMBeanProxy.create(mBeanAccessor, ActiveRepairServiceMBean.class); + almProxy = LazyMBeanProxy.create(mBeanAccessor, AuditLogManagerMBean.class); + ccProxy = LazyMBeanProxy.create(mBeanAccessor, PasswordAuthenticator.CredentialsCacheMBean.class); + jpcProxy = LazyMBeanProxy.create(mBeanAccessor, AuthorizationProxy.JmxPermissionsCacheMBean.class); + npcProxy = LazyMBeanProxy.create(mBeanAccessor, NetworkPermissionsCacheMBean.class); + cpbProxy = LazyMBeanProxy.create(mBeanAccessor, CIDRPermissionsManagerMBean.class); + cmbProxy = LazyMBeanProxy.create(mBeanAccessor, CIDRGroupsMappingManagerMBean.class); + pcProxy = LazyMBeanProxy.create(mBeanAccessor, PermissionsCacheMBean.class); + rcProxy = LazyMBeanProxy.create(mBeanAccessor, RolesCacheMBean.class); + autoRepairProxy = LazyMBeanProxy.create(mBeanAccessor, AutoRepairServiceMBean.class); + grProxy = LazyMBeanProxy.create(mBeanAccessor, GuardrailsMBean.class); + cfmProxy = LazyMBeanProxy.create(mBeanAccessor, CIDRFilteringMetricsTableMBean.class); } public void setOutput(Output output) @@ -555,7 +433,7 @@ public void startAndBlockOnAsyncRepairs(final PrintStream out, Collection runners = new ArrayList<>(cmds.size()); for (RepairCmd cmd : cmds) - runners.add(new RepairRunner(out, jmxc, ssProxy, cmd)); + runners.add(new RepairRunner(out, mBeanAccessor, ssProxy, cmd)); try { @@ -805,23 +683,14 @@ public Map effectiveOwnershipWithPort(String keyspace) throws Ill return ssProxy.effectiveOwnershipWithPort(keyspace); } - public MBeanServerConnection getMbeanServerConn() + public MBeanAccessor getMBeanAccessor() { - return mbeanServerConn; + return mBeanAccessor; } public CacheServiceMBean getCacheServiceMBean() { - String cachePath = "org.apache.cassandra.db:type=Caches"; - - try - { - return JMX.newMBeanProxy(mbeanServerConn, new ObjectName(cachePath), CacheServiceMBean.class); - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException(e); - } + return mBeanAccessor.findMBean(CacheServiceMBean.class); } public double[] getAndResetGCStats() @@ -833,7 +702,7 @@ public Iterator> getColumnFamilyStoreM { try { - return new ColumnFamilyStoreMBeanIterator(mbeanServerConn); + return new ColumnFamilyStoreMBeanIterator(mBeanAccessor); } catch (MalformedObjectNameException e) { @@ -1247,38 +1116,17 @@ public void truncate(String keyspaceName, String tableName) public EndpointSnitchInfoMBean getEndpointSnitchInfoProxy() { - try - { - return JMX.newMBeanProxy(mbeanServerConn, new ObjectName("org.apache.cassandra.db:type=EndpointSnitchInfo"), EndpointSnitchInfoMBean.class); - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException(e); - } + return mBeanAccessor.findMBean(EndpointSnitchInfoMBean.class); } public DynamicEndpointSnitchMBean getDynamicEndpointSnitchInfoProxy() { - try - { - return JMX.newMBeanProxy(mbeanServerConn, new ObjectName("org.apache.cassandra.db:type=DynamicEndpointSnitch"), DynamicEndpointSnitchMBean.class); - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException(e); - } + return mBeanAccessor.findMBean(DynamicEndpointSnitchMBean.class); } public LocationInfoMBean getLocationInfoProxy() { - try - { - return JMX.newMBeanProxy(mbeanServerConn, new ObjectName("org.apache.cassandra.db:type=LocationInfo"), LocationInfoMBean.class); - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException(e); - } + return mBeanAccessor.findMBean(LocationInfoMBean.class); } public ColumnFamilyStoreMBean getCfsProxy(String ks, String cf) @@ -1287,24 +1135,11 @@ public ColumnFamilyStoreMBean getCfsProxy(String ks, String cf) try { String type = cf.contains(".") ? "IndexColumnFamilies" : "ColumnFamilies"; - Set beans = mbeanServerConn.queryNames( - new ObjectName("org.apache.cassandra.db:type=*" + type +",keyspace=" + ks + ",columnfamily=" + cf), null); - - if (beans.isEmpty()) - throw new MalformedObjectNameException("couldn't find that bean"); - assert beans.size() == 1; - for (ObjectName bean : beans) - cfsProxy = JMX.newMBeanProxy(mbeanServerConn, bean, ColumnFamilyStoreMBean.class); - } - catch (MalformedObjectNameException mone) - { - System.err.println("ColumnFamilyStore for " + ks + "/" + cf + " not found."); - System.exit(1); + return mBeanAccessor.findColumnFamily(type, ks, cf); } - catch (IOException e) + catch (Exception e) { System.err.println("ColumnFamilyStore for " + ks + "/" + cf + " not found: " + e); - System.exit(1); } return cfsProxy; @@ -1852,29 +1687,21 @@ public Object getCacheMetric(String cacheType, String metricName) case "Entries": case "HitRate": case "Size": - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=Cache,scope=" + cacheType + ",name=" + metricName), - CassandraMetricsRegistry.JmxGaugeMBean.class).getValue(); + return mBeanAccessor.findMBeanGauge(MBeanAccessor.Props.scoped("Cache", cacheType, metricName)).getValue(); case "Requests": case "Hits": case "Misses": - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=Cache,scope=" + cacheType + ",name=" + metricName), - CassandraMetricsRegistry.JmxMeterMBean.class).getCount(); + return mBeanAccessor.findMBeanMeter(MBeanAccessor.Props.scoped("Cache", cacheType, metricName)).getCount(); case "MissLatency": - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=Cache,scope=" + cacheType + ",name=" + metricName), - CassandraMetricsRegistry.JmxTimerMBean.class).getMean(); + return mBeanAccessor.findMBeanTimer(MBeanAccessor.Props.scoped("Cache", cacheType, metricName)).getMean(); case "MissLatencyUnit": - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=Cache,scope=" + cacheType + ",name=MissLatency"), - CassandraMetricsRegistry.JmxTimerMBean.class).getDurationUnit(); + return mBeanAccessor.findMBeanTimer(MBeanAccessor.Props.scoped("Cache", cacheType, "MissLatency")).getDurationUnit(); default: throw new RuntimeException("Unknown Cache metric name " + metricName); } } - catch (MalformedObjectNameException e) + catch (Exception e) { throw new RuntimeException(e); } @@ -1896,51 +1723,20 @@ public Object getBufferPoolMetric(String poolType, String metricName) case "OverflowSize": case "Capacity": case "Size": - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=BufferPool,scope=" + poolType + ",name=" + metricName), - CassandraMetricsRegistry.JmxGaugeMBean.class).getValue(); + return mBeanAccessor.findMBeanGauge(MBeanAccessor.Props.scoped("BufferPool", poolType, metricName)).getValue(); case "Hits": case "Misses": - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=BufferPool,scope=" + poolType + ",name=" + metricName), - CassandraMetricsRegistry.JmxMeterMBean.class).getCount(); + return mBeanAccessor.findMBeanMeter(MBeanAccessor.Props.scoped("BufferPool", poolType, metricName)).getCount(); default: throw new RuntimeException("Unknown BufferPool metric name " + metricName); } } - catch (MalformedObjectNameException e) + catch (Exception e) { throw new RuntimeException(e); } } - private static Multimap getJmxThreadPools(MBeanServerConnection mbeanServerConn) - { - try - { - Multimap threadPools = HashMultimap.create(); - - Set threadPoolObjectNames = mbeanServerConn.queryNames( - new ObjectName("org.apache.cassandra.metrics:type=ThreadPools,*"), - null); - - for (ObjectName oName : threadPoolObjectNames) - { - threadPools.put(oName.getKeyProperty("path"), oName.getKeyProperty("scope")); - } - - return threadPools; - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException("Bad query to JMX server: ", e); - } - catch (IOException e) - { - throw new RuntimeException("Error getting threadpool names from JMX", e); - } - } - public Object getThreadPoolMetric(String pathName, String poolName, String metricName) { String name = String.format("org.apache.cassandra.metrics:type=ThreadPools,path=%s,scope=%s,name=%s", @@ -1948,8 +1744,7 @@ public Object getThreadPoolMetric(String pathName, String poolName, String metri try { - ObjectName oName = new ObjectName(name); - if (!mbeanServerConn.isRegistered(oName)) + if (mBeanAccessor.findMBeanMetric(Object.class, MBeanAccessor.Props.threadPool("ThreadPools", pathName, poolName, metricName)) == null) { return "N/A"; } @@ -1962,10 +1757,10 @@ public Object getThreadPoolMetric(String pathName, String poolName, String metri case ThreadPoolMetrics.CORE_POOL_SIZE: case ThreadPoolMetrics.MAX_POOL_SIZE: case ThreadPoolMetrics.MAX_TASKS_QUEUED: - return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxGaugeMBean.class).getValue(); + return mBeanAccessor.findMBeanGauge(MBeanAccessor.Props.threadPool("ThreadPools", pathName, poolName, metricName)).getValue(); case ThreadPoolMetrics.TOTAL_BLOCKED_TASKS: case ThreadPoolMetrics.CURRENTLY_BLOCKED_TASKS: - return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxCounterMBean.class).getCount(); + return mBeanAccessor.findMBeanCounter(MBeanAccessor.Props.threadPool("ThreadPools", pathName, poolName, metricName)).getCount(); default: throw new AssertionError("Unknown ThreadPools metric name " + metricName); } @@ -1978,45 +1773,26 @@ public Object getThreadPoolMetric(String pathName, String poolName, String metri public Object getSaiMetric(String ks, String cf, String metricName) { - try - { - String scope = getSaiMetricScope(metricName); - String objectNameStr = String.format("org.apache.cassandra.metrics:type=StorageAttachedIndex,keyspace=%s,table=%s,scope=%s,name=%s",ks, cf, scope, metricName); - ObjectName oName = new ObjectName(objectNameStr); - - Set matchingMBeans = mbeanServerConn.queryNames(oName, null); - if (matchingMBeans.isEmpty()) - return null; - - return getSaiMetricValue(metricName, oName); - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException("Invalid ObjectName format: " + e.getMessage(), e); - } - catch (IOException e) - { - throw new RuntimeException("Error accessing MBean server: " + e.getMessage(), e); - } - } + String scope = getSaiMetricScope(metricName); + MBeanAccessor.Props props = MBeanAccessor.Props.sai("StorageAttachedIndex", ks, cf, scope, metricName); + if (mBeanAccessor.findMBeanMetric(Object.class, props) == null) + return null; - private Object getSaiMetricValue(String metricName, ObjectName oName) throws IOException - { switch (metricName) { case "QueryLatency": - return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxTimerMBean.class); + return mBeanAccessor.findMBeanTimer(props); case "PostFilteringReadLatency": case "SSTableIndexesHit": case "IndexSegmentsHit": case "RowsFiltered": - return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxHistogramMBean.class); + return mBeanAccessor.findMBeanHistogram(props); case "DiskUsedBytes": case "TotalIndexCount": case "TotalQueryableIndexCount": - return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxGaugeMBean.class).getValue(); + return mBeanAccessor.findMBeanGauge(props).getValue(); case "TotalQueryTimeouts": - return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxCounterMBean.class).getCount(); + return mBeanAccessor.findMBeanCounter(props).getCount(); default: throw new IllegalArgumentException("Unknown metric name: " + metricName); } @@ -2048,10 +1824,13 @@ private String getSaiMetricScope(String metricName) * Retrieve threadpool paths and names for threadpools with metrics. * @return Multimap from path (internal, request, etc.) to name */ - public Multimap getThreadPools() - { - return getJmxThreadPools(mbeanServerConn); - } + public Multimap getThreadPools() + { + Multimap threadPools = HashMultimap.create(); + for (MBeanAccessor.ThreadPoolInfo info : mBeanAccessor.threadPoolInfos()) + threadPools.put(info.path(), info.scope()); + return threadPools; + } public int getNumberOfTables() { @@ -2066,92 +1845,85 @@ public int getNumberOfTables() */ public Object getColumnFamilyMetric(String ks, String cf, String metricName) { - try + MBeanAccessor.Props props; + if (!Strings.isNullOrEmpty(ks) && !Strings.isNullOrEmpty(cf)) { - ObjectName oName = null; - if (!Strings.isNullOrEmpty(ks) && !Strings.isNullOrEmpty(cf)) - { - String type = cf.contains(".") ? "IndexTable" : "Table"; - oName = new ObjectName(String.format("org.apache.cassandra.metrics:type=%s,keyspace=%s,scope=%s,name=%s", type, ks, cf, metricName)); - } - else if (!Strings.isNullOrEmpty(ks)) - { - oName = new ObjectName(String.format("org.apache.cassandra.metrics:type=Keyspace,keyspace=%s,name=%s", ks, metricName)); - } - else - { - oName = new ObjectName(String.format("org.apache.cassandra.metrics:type=Table,name=%s", metricName)); - } - switch(metricName) + String type = cf.contains(".") ? "IndexTable" : "Table"; + props = MBeanAccessor.Props.columnFamily(type, ks, cf, metricName); + } + else if (!Strings.isNullOrEmpty(ks)) + { + props = MBeanAccessor.Props.keyspace("Keyspace", ks, metricName); + } + else + { + props = MBeanAccessor.Props.metric("Table", metricName); + } + switch (metricName) + { + case "BloomFilterDiskSpaceUsed": + case "BloomFilterFalsePositives": + case "BloomFilterFalseRatio": + case "BloomFilterOffHeapMemoryUsed": + case "IndexSummaryOffHeapMemoryUsed": + case "CompressionDictionariesMemoryUsed": + case "CompressionMetadataOffHeapMemoryUsed": + case "CompressionRatio": + case "EstimatedColumnCountHistogram": + case "EstimatedPartitionSizeHistogram": + case "EstimatedPartitionCount": + case "KeyCacheHitRate": + case "LiveSSTableCount": + case "MaxSSTableDuration": + case "MaxSSTableSize": + case "OldVersionSSTableCount": + case "MaxPartitionSize": + case "MeanPartitionSize": + case "MemtableColumnsCount": + case "MemtableLiveDataSize": + case "MemtableOffHeapSize": + case "MinPartitionSize": + case "PercentRepaired": + case "BytesRepaired": + case "BytesUnrepaired": + case "BytesPendingRepair": + case "RecentBloomFilterFalsePositives": + case "RecentBloomFilterFalseRatio": + case "SnapshotsSize": + return mBeanAccessor.findMBeanGauge(props).getValue(); + case "LiveDiskSpaceUsed": + case "MemtableSwitchCount": + case "SpeculativeRetries": + case "TotalDiskSpaceUsed": + case "WriteTotalLatency": + case "ReadTotalLatency": + case "PendingFlushes": { - case "BloomFilterDiskSpaceUsed": - case "BloomFilterFalsePositives": - case "BloomFilterFalseRatio": - case "BloomFilterOffHeapMemoryUsed": - case "IndexSummaryOffHeapMemoryUsed": - case "CompressionDictionariesMemoryUsed": - case "CompressionMetadataOffHeapMemoryUsed": - case "CompressionRatio": - case "EstimatedColumnCountHistogram": - case "EstimatedPartitionSizeHistogram": - case "EstimatedPartitionCount": - case "KeyCacheHitRate": - case "LiveSSTableCount": - case "MaxSSTableDuration": - case "MaxSSTableSize": - case "OldVersionSSTableCount": - case "MaxPartitionSize": - case "MeanPartitionSize": - case "MemtableColumnsCount": - case "MemtableLiveDataSize": - case "MemtableOffHeapSize": - case "MinPartitionSize": - case "PercentRepaired": - case "BytesRepaired": - case "BytesUnrepaired": - case "BytesPendingRepair": - case "RecentBloomFilterFalsePositives": - case "RecentBloomFilterFalseRatio": - case "SnapshotsSize": - return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxGaugeMBean.class).getValue(); - case "LiveDiskSpaceUsed": - case "MemtableSwitchCount": - case "SpeculativeRetries": - case "TotalDiskSpaceUsed": - case "WriteTotalLatency": - case "ReadTotalLatency": - case "PendingFlushes": + // these are gauges for keyspace metrics, not counters + if (!Strings.isNullOrEmpty(ks) && + Strings.isNullOrEmpty(cf) && + (metricName.equals("TotalDiskSpaceUsed") || + metricName.equals("LiveDiskSpaceUsed") || + metricName.equals("MemtableSwitchCount"))) { - // these are gauges for keyspace metrics, not counters - if (!Strings.isNullOrEmpty(ks) && - Strings.isNullOrEmpty(cf) && - (metricName.equals("TotalDiskSpaceUsed") || - metricName.equals("LiveDiskSpaceUsed") || - metricName.equals("MemtableSwitchCount"))) - { - return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxGaugeMBean.class).getValue(); - } - else - { - return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxCounterMBean.class).getCount(); - } + return mBeanAccessor.findMBeanGauge(props).getValue(); + } + else + { + return mBeanAccessor.findMBeanCounter(props).getCount(); } - case "CoordinatorReadLatency": - case "CoordinatorScanLatency": - case "ReadLatency": - case "WriteLatency": - return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxTimerMBean.class); - case "LiveScannedHistogram": - case "SSTablesPerReadHistogram": - case "TombstoneScannedHistogram": - return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxHistogramMBean.class); - default: - throw new RuntimeException("Unknown table metric " + metricName); } - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException(e); + case "CoordinatorReadLatency": + case "CoordinatorScanLatency": + case "ReadLatency": + case "WriteLatency": + return mBeanAccessor.findMBeanTimer(props); + case "LiveScannedHistogram": + case "SSTablesPerReadHistogram": + case "TombstoneScannedHistogram": + return mBeanAccessor.findMBeanHistogram(props); + default: + throw new RuntimeException("Unknown table metric " + metricName); } } @@ -2161,30 +1933,12 @@ else if (!Strings.isNullOrEmpty(ks)) */ public CassandraMetricsRegistry.JmxTimerMBean getProxyMetric(String scope) { - try - { - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=ClientRequest,scope=" + scope + ",name=Latency"), - CassandraMetricsRegistry.JmxTimerMBean.class); - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException(e); - } + return mBeanAccessor.findMBeanTimer(MBeanAccessor.Props.scoped("ClientRequest", scope, "Latency")); } public CassandraMetricsRegistry.JmxTimerMBean getMessagingQueueWaitMetrics(String verb) { - try - { - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:name=" + verb + "-WaitLatency,type=Messaging"), - CassandraMetricsRegistry.JmxTimerMBean.class); - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException(e); - } + return mBeanAccessor.findMBeanTimer(MBeanAccessor.Props.metric("Messaging", verb + "-WaitLatency")); } /** @@ -2195,34 +1949,21 @@ public CassandraMetricsRegistry.JmxTimerMBean getMessagingQueueWaitMetrics(Strin */ public Object getCompactionMetric(String metricName) { - try - { - switch(metricName) - { - case "BytesCompacted": - case "CompactionsAborted": - case "CompactionsReduced": - case "SSTablesDroppedFromCompaction": - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=Compaction,name=" + metricName), - CassandraMetricsRegistry.JmxCounterMBean.class); - case "CompletedTasks": - case "PendingTasks": - case "PendingTasksByTableName": - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=Compaction,name=" + metricName), - CassandraMetricsRegistry.JmxGaugeMBean.class).getValue(); - case "TotalCompactionsCompleted": - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=Compaction,name=" + metricName), - CassandraMetricsRegistry.JmxMeterMBean.class); - default: - throw new RuntimeException("Unknown compaction metric " + metricName); - } - } - catch (MalformedObjectNameException e) + switch (metricName) { - throw new RuntimeException(e); + case "BytesCompacted": + case "CompactionsAborted": + case "CompactionsReduced": + case "SSTablesDroppedFromCompaction": + return mBeanAccessor.findMBeanCounter(MBeanAccessor.Props.metric("Compaction", metricName)); + case "CompletedTasks": + case "PendingTasks": + case "PendingTasksByTableName": + return mBeanAccessor.findMBeanGauge(MBeanAccessor.Props.metric("Compaction", metricName)).getValue(); + case "TotalCompactionsCompleted": + return mBeanAccessor.findMBeanMeter(MBeanAccessor.Props.metric("Compaction", metricName)); + default: + throw new RuntimeException("Unknown compaction metric " + metricName); } } @@ -2232,65 +1973,15 @@ public Object getCompactionMetric(String metricName) */ public Object getClientMetric(String metricName) { - try - { - switch(metricName) - { - case "connections": // List> - list of all native connections and their properties - case "connectedNativeClients": // number of connected native clients - case "connectedNativeClientsByUser": // number of native clients by username - case "clientsByProtocolVersion": // number of native clients by protocol version - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=Client,name=" + metricName), - CassandraMetricsRegistry.JmxGaugeMBean.class).getValue(); - default: - throw new RuntimeException("Unknown client metric " + metricName); - } - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException(e); - } - } - - public Object getCidrFilteringMetric(String metricName) - { - try - { - switch(metricName) - { - case CIDRAuthorizerMetrics.CIDR_CHECKS_LATENCY: - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=CIDRAuthorization,name=" - + metricName), - CassandraMetricsRegistry.JmxTimerMBean.class).getMean(); - case CIDRAuthorizerMetrics.CIDR_GROUPS_CACHE_RELOAD_COUNT: - return JMX.newMBeanProxy( - mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=CIDRGroupsMappingCache,name=" + metricName), - CassandraMetricsRegistry.JmxCounterMBean.class).getCount(); - case CIDRAuthorizerMetrics.CIDR_GROUPS_CACHE_RELOAD_LATENCY: - case CIDRAuthorizerMetrics.LOOKUP_CIDR_GROUPS_FOR_IP_LATENCY: - return JMX.newMBeanProxy( - mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=CIDRGroupsMappingCache,name=" + metricName), - CassandraMetricsRegistry.JmxTimerMBean.class).getMean(); - default: - if (metricName.contains(CIDRAuthorizerMetrics.CIDR_ACCESSES_REJECTED_COUNT_PREFIX) || - metricName.contains(CIDRAuthorizerMetrics.CIDR_ACCESSES_ACCEPTED_COUNT_PREFIX)) - { - return JMX.newMBeanProxy( - mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=mymetricname,name=" + metricName), - CassandraMetricsRegistry.JmxCounterMBean.class).getCount(); - } - - throw new RuntimeException("Unknown metric " + metricName); - } - } - catch (MalformedObjectNameException e) + switch (metricName) { - throw new RuntimeException(e); + case "connections": // List> - list of all native connections and their properties + case "connectedNativeClients": // number of connected native clients + case "connectedNativeClientsByUser": // number of native clients by username + case "clientsByProtocolVersion": // number of native clients by protocol version + return mBeanAccessor.findMBeanGauge(MBeanAccessor.Props.metric("Client", metricName)).getValue(); + default: + throw new RuntimeException("Unknown client metric " + metricName); } } @@ -2310,16 +2001,7 @@ public Map> getLatenciesMetricsFromVtable() */ public long getStorageMetric(String metricName) { - try - { - return JMX.newMBeanProxy(mbeanServerConn, - new ObjectName("org.apache.cassandra.metrics:type=Storage,name=" + metricName), - CassandraMetricsRegistry.JmxCounterMBean.class).getCount(); - } - catch (MalformedObjectNameException e) - { - throw new RuntimeException(e); - } + return mBeanAccessor.findMBeanCounter(MBeanAccessor.Props.metric("Storage", metricName)).getCount(); } public Double[] metricPercentilesAsArray(CassandraMetricsRegistry.JmxHistogramMBean metric) @@ -2381,8 +2063,8 @@ public void resumeBootstrap(PrintStream out) throws IOException BootstrapMonitor monitor = new BootstrapMonitor(out); try { - if (jmxc != null) - jmxc.addConnectionNotificationListener(monitor, null, null); + if (mBeanAccessor instanceof RemoteJmxMBeanAccessor) + ((RemoteJmxMBeanAccessor) mBeanAccessor).getJmxConnector().addConnectionNotificationListener(monitor, null, null); ssProxy.addNotificationListener(monitor, null, null); if (ssProxy.resumeBootstrap()) { @@ -2405,8 +2087,8 @@ public void resumeBootstrap(PrintStream out) throws IOException try { ssProxy.removeNotificationListener(monitor); - if (jmxc != null) - jmxc.removeConnectionNotificationListener(monitor); + if (mBeanAccessor instanceof RemoteJmxMBeanAccessor) + ((RemoteJmxMBeanAccessor) mBeanAccessor).getJmxConnector().removeConnectionNotificationListener(monitor); } catch (Throwable e) { @@ -2806,80 +2488,89 @@ public TrainingState getCompressionDictionaryTrainingState(String keyspace, Stri private CompressionDictionaryManagerMBean getDictionaryManagerProxy(String keyspace, String table) throws IOException { - // Construct table-specific MBean name - String mbeanName = CompressionDictionaryManagerMBean.MBEAN_NAME + ",keyspace=" + keyspace + ",table=" + table; - try + return mBeanAccessor.findCompressionDictionary(keyspace, table); + } + + /** + * A dynamic proxy that lazily looks up the MBean the first time a method is invoked. + * @param the MBean interface type. + */ + private static class LazyMBeanProxy implements java.lang.reflect.InvocationHandler + { + private final MBeanAccessor provider; + private final Class mbeanClass; + private volatile T delegate; + + LazyMBeanProxy(MBeanAccessor provider, Class mbeanClass) { - ObjectName objectName = new ObjectName(mbeanName); - return JMX.newMBeanProxy(mbeanServerConn, objectName, CompressionDictionaryManagerMBean.class); + this.provider = provider; + this.mbeanClass = mbeanClass; } - catch (MalformedObjectNameException e) + + @Override + public Object invoke(Object proxy, Method method, Object[] args) + throws Throwable { - throw new IOException("Invalid keyspace or table name", e); + if (delegate == null) + { + synchronized (this) + { + if (delegate == null) + delegate = provider.findMBean(mbeanClass); + } + } + return method.invoke(delegate, args); + } + + @SuppressWarnings("unchecked") + static T create(MBeanAccessor provider, Class mbeanClass) + { + return (T) Proxy.newProxyInstance(mbeanClass.getClassLoader(), + new Class[]{ mbeanClass }, + new LazyMBeanProxy<>(provider, mbeanClass)); } } } class ColumnFamilyStoreMBeanIterator implements Iterator> { - private MBeanServerConnection mbeanServerConn; Iterator> mbeans; - public ColumnFamilyStoreMBeanIterator(MBeanServerConnection mbeanServerConn) + public ColumnFamilyStoreMBeanIterator(MBeanAccessor accessor) throws MalformedObjectNameException, NullPointerException, IOException { - this.mbeanServerConn = mbeanServerConn; - List> cfMbeans = getCFSMBeans(mbeanServerConn, "ColumnFamilies"); - cfMbeans.addAll(getCFSMBeans(mbeanServerConn, "IndexColumnFamilies")); - Collections.sort(cfMbeans, new Comparator>() - { - public int compare(Entry e1, Entry e2) - { - //compare keyspace, then CF name, then normal vs. index - int keyspaceNameCmp = e1.getKey().compareTo(e2.getKey()); - if(keyspaceNameCmp != 0) - return keyspaceNameCmp; - - // get CF name and split it for index name - String e1CF[] = e1.getValue().getTableName().split("\\."); - String e2CF[] = e2.getValue().getTableName().split("\\."); - assert e1CF.length <= 2 && e2CF.length <= 2 : "unexpected split count for table name"; - - //if neither are indexes, just compare CF names - if(e1CF.length == 1 && e2CF.length == 1) - return e1CF[0].compareTo(e2CF[0]); - - //check if it's the same CF - int cfNameCmp = e1CF[0].compareTo(e2CF[0]); - if(cfNameCmp != 0) - return cfNameCmp; - - // if both are indexes (for the same CF), compare them - if(e1CF.length == 2 && e2CF.length == 2) - return e1CF[1].compareTo(e2CF[1]); - - //if length of e1CF is 1, it's not an index, so sort it higher - return e1CF.length == 1 ? 1 : -1; - } + List> cfMbeans = accessor.findColumnFamilies("ColumnFamilies"); + cfMbeans.addAll(accessor.findColumnFamilies("IndexColumnFamilies")); + cfMbeans.sort((e1, e2) -> { + //compare keyspace, then CF name, then normal vs. index + int keyspaceNameCmp = e1.getKey().compareTo(e2.getKey()); + if (keyspaceNameCmp != 0) + return keyspaceNameCmp; + + // get CF name and split it for index name + String e1CF[] = e1.getValue().getTableName().split("\\."); + String e2CF[] = e2.getValue().getTableName().split("\\."); + assert e1CF.length <= 2 && e2CF.length <= 2 : "unexpected split count for table name"; + + //if neither are indexes, just compare CF names + if (e1CF.length == 1 && e2CF.length == 1) + return e1CF[0].compareTo(e2CF[0]); + + //check if it's the same CF + int cfNameCmp = e1CF[0].compareTo(e2CF[0]); + if (cfNameCmp != 0) + return cfNameCmp; + + // if both are indexes (for the same CF), compare them + if (e1CF.length == 2 && e2CF.length == 2) + return e1CF[1].compareTo(e2CF[1]); + + //if length of e1CF is 1, it's not an index, so sort it higher + return e1CF.length == 1 ? 1 : -1; }); mbeans = cfMbeans.iterator(); } - private List> getCFSMBeans(MBeanServerConnection mbeanServerConn, String type) - throws MalformedObjectNameException, IOException - { - ObjectName query = new ObjectName("org.apache.cassandra.db:type=" + type +",*"); - Set cfObjects = mbeanServerConn.queryNames(query, null); - List> mbeans = new ArrayList>(cfObjects.size()); - for(ObjectName n : cfObjects) - { - String keyspaceName = n.getKeyProperty("keyspace"); - ColumnFamilyStoreMBean cfsProxy = JMX.newMBeanProxy(mbeanServerConn, n, ColumnFamilyStoreMBean.class); - mbeans.add(new AbstractMap.SimpleImmutableEntry(keyspaceName, cfsProxy)); - } - return mbeans; - } - public boolean hasNext() { return mbeans.hasNext(); diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index 005e5304edf4..1ee4a6169846 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -40,6 +40,7 @@ import org.apache.cassandra.tools.nodetool.JmxConnect; import org.apache.cassandra.tools.nodetool.NodetoolCommand; import org.apache.cassandra.tools.nodetool.layout.CassandraCliHelpLayout; +import org.apache.cassandra.tools.nodetool.strategy.ProtocolAwareExecutionStrategy; import org.apache.cassandra.utils.FBUtilities; import picocli.CommandLine; @@ -86,7 +87,7 @@ public int execute(String... args) commandLine.setErr(new PrintWriter(output.err, true)); configureCliLayout(commandLine); - commandLine.setExecutionStrategy(JmxConnect::executionStrategy) + commandLine.setExecutionStrategy(ProtocolAwareExecutionStrategy::executionStrategy) .setExecutionExceptionHandler((ex, c, arg) -> { // Used for backward compatibility, some commands are validated when a command is run. if (ex instanceof IllegalArgumentException | @@ -178,7 +179,7 @@ private static void getCommandsWithoutRoot(CommandLine cli, List command public static CommandLine createCommandLine(CommandLine.IFactory factory) throws Exception { return new CommandLine(new NodetoolCommand(), factory) - .addMixin(JmxConnect.MIXIN_KEY, factory.create(JmxConnect.class)); + .addMixin(ProtocolAwareExecutionStrategy.MIXIN_KEY, factory.create(JmxConnect.class)); } private static void configureCliLayout(CommandLine commandLine) diff --git a/src/java/org/apache/cassandra/tools/RemoteJmxMBeanAccessor.java b/src/java/org/apache/cassandra/tools/RemoteJmxMBeanAccessor.java new file mode 100644 index 000000000000..763620133054 --- /dev/null +++ b/src/java/org/apache/cassandra/tools/RemoteJmxMBeanAccessor.java @@ -0,0 +1,439 @@ +/* + * 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.cassandra.tools; + +import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; +import java.lang.management.RuntimeMXBean; +import java.rmi.server.RMIClientSocketFactory; +import java.rmi.server.RMISocketFactory; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Hashtable; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import javax.management.JMX; +import javax.management.MBeanServerConnection; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; +import javax.management.remote.JMXConnector; +import javax.management.remote.JMXConnectorFactory; +import javax.management.remote.JMXServiceURL; +import javax.rmi.ssl.SslRMIClientSocketFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.audit.AuditLogManager; +import org.apache.cassandra.audit.AuditLogManagerMBean; +import org.apache.cassandra.auth.AuthCache; +import org.apache.cassandra.auth.CIDRGroupsMappingManager; +import org.apache.cassandra.auth.CIDRGroupsMappingManagerMBean; +import org.apache.cassandra.auth.CIDRPermissionsManager; +import org.apache.cassandra.auth.CIDRPermissionsManagerMBean; +import org.apache.cassandra.auth.NetworkPermissionsCache; +import org.apache.cassandra.auth.NetworkPermissionsCacheMBean; +import org.apache.cassandra.auth.PasswordAuthenticator; +import org.apache.cassandra.auth.PermissionsCache; +import org.apache.cassandra.auth.PermissionsCacheMBean; +import org.apache.cassandra.auth.RolesCache; +import org.apache.cassandra.auth.RolesCacheMBean; +import org.apache.cassandra.auth.jmx.AuthorizationProxy; +import org.apache.cassandra.batchlog.BatchlogManager; +import org.apache.cassandra.batchlog.BatchlogManagerMBean; +import org.apache.cassandra.db.ColumnFamilyStoreMBean; +import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.db.compaction.CompactionManagerMBean; +import org.apache.cassandra.db.compression.CompressionDictionaryManagerMBean; +import org.apache.cassandra.db.guardrails.Guardrails; +import org.apache.cassandra.db.guardrails.GuardrailsMBean; +import org.apache.cassandra.db.virtual.CIDRFilteringMetricsTable; +import org.apache.cassandra.db.virtual.CIDRFilteringMetricsTableMBean; +import org.apache.cassandra.gms.FailureDetector; +import org.apache.cassandra.gms.FailureDetectorMBean; +import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.gms.GossiperMBean; +import org.apache.cassandra.hints.HintsService; +import org.apache.cassandra.hints.HintsServiceMBean; +import org.apache.cassandra.locator.DynamicEndpointSnitchMBean; +import org.apache.cassandra.locator.EndpointSnitchInfoMBean; +import org.apache.cassandra.locator.LocationInfoMBean; +import org.apache.cassandra.management.MBeanAccessor; +import org.apache.cassandra.metrics.CassandraMetricsRegistry; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.MessagingServiceMBean; +import org.apache.cassandra.service.ActiveRepairServiceMBean; +import org.apache.cassandra.service.AutoRepairService; +import org.apache.cassandra.service.AutoRepairServiceMBean; +import org.apache.cassandra.service.CacheService; +import org.apache.cassandra.service.CacheServiceMBean; +import org.apache.cassandra.service.GCInspector; +import org.apache.cassandra.service.GCInspectorMXBean; +import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.service.StorageProxyMBean; +import org.apache.cassandra.service.StorageServiceMBean; +import org.apache.cassandra.service.accord.AccordOperations; +import org.apache.cassandra.service.accord.AccordOperationsMBean; +import org.apache.cassandra.service.snapshot.SnapshotManagerMBean; +import org.apache.cassandra.streaming.StreamManagerMBean; +import org.apache.cassandra.tcm.CMSOperations; +import org.apache.cassandra.tcm.CMSOperationsMBean; + +import static org.apache.cassandra.config.CassandraRelevantProperties.SSL_ENABLE; + +public class RemoteJmxMBeanAccessor implements MBeanAccessor +{ + private final Map, Object> clazzMBanRegistry = new HashMap<>(); + private final Map namedMBeanRegistry = new ConcurrentHashMap<>(); + + public static final int defaultPort = 7199; + + private static final Logger logger = LoggerFactory.getLogger(RemoteJmxMBeanAccessor.class); + private static final String fmtUrl = "service:jmx:rmi:///jndi/rmi://%s:%d/jmxrmi"; + + final String host; + final int port; + private String username; + private String password; + + protected JMXConnector jmxc; + protected MBeanServerConnection mbeanServerConn; + + private volatile boolean connected = false; + + /** + * Creates a NodeProbe using the specified JMX host, port, username, and password. + * + * @param host hostname or IP address of the JMX agent + * @param port TCP port of the remote JMX agent + */ + public RemoteJmxMBeanAccessor(String host, int port, String username, String password) + { + assert username != null && !username.isEmpty() && password != null && !password.isEmpty() + : "neither username nor password can be blank"; + + this.host = host; + this.port = port; + this.username = username; + this.password = password; + } + + /** + * Creates a NodeProbe using the specified JMX host and port. + * + * @param host hostname or IP address of the JMX agent + * @param port TCP port of the remote JMX agent + */ + public RemoteJmxMBeanAccessor(String host, int port) + { + this.host = host; + this.port = port; + } + + /** + * Creates a NodeProbe using the specified JMX host and default port. + * + * @param host hostname or IP address of the JMX agent + */ + public RemoteJmxMBeanAccessor(String host) + { + this(host, defaultPort); + } + + /** + * Create a connection to the JMX agent and set up the M[X]Bean proxies. + */ + protected void connect() + { + if (connected) + return; + + synchronized (this) + { + if (connected) + return; + + try + { + String host = this.host; + if (host.contains(":")) + { + // Use square brackets to surround IPv6 addresses to fix CASSANDRA-7669 and CASSANDRA-17581 + host = '[' + host + ']'; + } + JMXServiceURL jmxUrl = new JMXServiceURL(String.format(fmtUrl, host, port)); + Map env = new HashMap<>(); + if (username != null) + { + String[] creds = { username, password }; + env.put(JMXConnector.CREDENTIALS, creds); + } + + env.put("com.sun.jndi.rmi.factory.socket", getRMIClientSocketFactory()); + + jmxc = JMXConnectorFactory.connect(jmxUrl, env); + mbeanServerConn = jmxc.getMBeanServerConnection(); + + registerMBeanProxy(StorageServiceMBean.class, "org.apache.cassandra.db:type=StorageService"); + registerMBeanProxy(SnapshotManagerMBean.class, SnapshotManagerMBean.MBEAN_NAME); + registerMBeanProxy(CMSOperationsMBean.class, CMSOperations.MBEAN_OBJECT_NAME); + registerMBeanProxy(AccordOperationsMBean.class, AccordOperations.MBEAN_OBJECT_NAME); + registerMBeanProxy(MessagingServiceMBean.class, MessagingService.MBEAN_NAME); + registerMBeanProxy(StreamManagerMBean.class, StreamManagerMBean.OBJECT_NAME); + registerMBeanProxy(CompactionManagerMBean.class, CompactionManager.MBEAN_OBJECT_NAME); + registerMBeanProxy(FailureDetectorMBean.class, FailureDetector.MBEAN_NAME); + registerMBeanProxy(CacheServiceMBean.class, CacheService.MBEAN_NAME); + registerMBeanProxy(StorageProxyMBean.class, StorageProxy.MBEAN_NAME); + registerMBeanProxy(HintsServiceMBean.class, HintsService.MBEAN_NAME); + registerMBeanProxy(GCInspectorMXBean.class, GCInspector.MBEAN_NAME); + registerMBeanProxy(GossiperMBean.class, Gossiper.MBEAN_NAME); + registerMBeanProxy(BatchlogManagerMBean.class, BatchlogManager.MBEAN_NAME); + registerMBeanProxy(ActiveRepairServiceMBean.class, ActiveRepairServiceMBean.MBEAN_NAME); + registerMBeanProxy(AuditLogManagerMBean.class, AuditLogManager.MBEAN_NAME); + registerMBeanProxy(PasswordAuthenticator.CredentialsCacheMBean.class, + AuthCache.MBEAN_NAME_BASE + PasswordAuthenticator.CredentialsCacheMBean.CACHE_NAME); + registerMBeanProxy(AuthorizationProxy.JmxPermissionsCacheMBean.class, + AuthCache.MBEAN_NAME_BASE + AuthorizationProxy.JmxPermissionsCacheMBean.CACHE_NAME); + registerMBeanProxy(NetworkPermissionsCacheMBean.class, + AuthCache.MBEAN_NAME_BASE + NetworkPermissionsCache.CACHE_NAME); + registerMBeanProxy(PermissionsCacheMBean.class, + AuthCache.MBEAN_NAME_BASE + PermissionsCache.CACHE_NAME); + registerMBeanProxy(RolesCacheMBean.class, + AuthCache.MBEAN_NAME_BASE + RolesCache.CACHE_NAME); + registerMBeanProxy(CIDRPermissionsManagerMBean.class, CIDRPermissionsManager.MBEAN_NAME); + registerMBeanProxy(CIDRGroupsMappingManagerMBean.class, CIDRGroupsMappingManager.MBEAN_NAME); + registerMBeanProxy(CIDRFilteringMetricsTableMBean.class, CIDRFilteringMetricsTable.MBEAN_NAME); + registerMBeanProxy(AutoRepairServiceMBean.class, AutoRepairService.MBEAN_NAME); + registerMBeanProxy(GuardrailsMBean.class, Guardrails.MBEAN_NAME); + + registerPlatformMBeanProxy(MemoryMXBean.class, ManagementFactory.MEMORY_MXBEAN_NAME); + registerPlatformMBeanProxy(RuntimeMXBean.class, ManagementFactory.RUNTIME_MXBEAN_NAME); + + registerMBeanProxy(EndpointSnitchInfoMBean.class, "org.apache.cassandra.db:type=EndpointSnitchInfo"); + registerMBeanProxy(DynamicEndpointSnitchMBean.class, "org.apache.cassandra.db:type=DynamicEndpointSnitch"); + registerMBeanProxy(LocationInfoMBean.class, "org.apache.cassandra.db:type=LocationInfo"); + } + catch (MalformedObjectNameException e) + { + close(); + throw new RuntimeException("Invalid ObjectName? Please report this as a bug.", e); + } + catch (IOException e) + { + close(); + throw new RuntimeException("Could not connect to MBean server. Please check that the JMX port is correct and open.", e); + } + + connected = true; + } + } + + protected void registerMBean(Class clazz, T mbean) + { + clazzMBanRegistry.put(clazz, mbean); + } + + @Override + public T findMBean(Class clazz) + { + connect(); + return clazzMBanRegistry.get(clazz) == null ? null : clazz.cast(clazzMBanRegistry.get(clazz)); + } + + @SuppressWarnings("unchecked") + public T findMBeanMetric(Class clazz, Props props) + { + return withExceptionHandling(() -> { + connect(); + ObjectName objectName = new ObjectName("org.apache.cassandra.metrics", new Hashtable<>(props.toMap())); + if (!mbeanServerConn.isRegistered(objectName)) + return null; + + T result = (T) namedMBeanRegistry.computeIfAbsent(objectName.getCanonicalName(), + ignore -> JMX.newMBeanProxy(mbeanServerConn, objectName, clazz)); + return clazz.cast(result); + }); + } + + @Override + public CassandraMetricsRegistry.JmxCounterMBean findMBeanCounter(Props props) + { + return findMBeanMetric(CassandraMetricsRegistry.JmxCounterMBean.class, props); + } + + @Override + public CassandraMetricsRegistry.JmxGaugeMBean findMBeanGauge(Props props) + { + return findMBeanMetric(CassandraMetricsRegistry.JmxGaugeMBean.class, props); + } + + @Override + public CassandraMetricsRegistry.JmxMeterMBean findMBeanMeter(Props props) + { + return findMBeanMetric(CassandraMetricsRegistry.JmxMeterMBean.class, props); + } + + @Override + public CassandraMetricsRegistry.JmxTimerMBean findMBeanTimer(Props props) + { + return findMBeanMetric(CassandraMetricsRegistry.JmxTimerMBean.class, props); + } + + @Override + public CassandraMetricsRegistry.JmxHistogramMBean findMBeanHistogram(Props props) + { + return findMBeanMetric(CassandraMetricsRegistry.JmxHistogramMBean.class, props); + } + + @Override + public ColumnFamilyStoreMBean findColumnFamily(String type, String keyspace, String columnFamily) + { + return withExceptionHandling(() -> { + connect(); + Set beans = mbeanServerConn.queryNames(new ObjectName("org.apache.cassandra.db:type=*" + type + + ",keyspace=" + keyspace + + ",columnfamily=" + columnFamily), null); + if (beans.isEmpty()) + throw new MalformedObjectNameException("couldn't find that bean"); + + assert beans.size() == 1; + return JMX.newMBeanProxy(mbeanServerConn, beans.iterator().next(), ColumnFamilyStoreMBean.class); + }); + } + + @Override + public CompressionDictionaryManagerMBean findCompressionDictionary(String keyspace, String table) + { + return withExceptionHandling(() -> { + connect(); + String mbeanName = CompressionDictionaryManagerMBean.MBEAN_NAME + ",keyspace=" + keyspace + ",table=" + table; + return JMX.newMBeanProxy(mbeanServerConn, new ObjectName(mbeanName), CompressionDictionaryManagerMBean.class); + }); + } + + @Override + public List threadPoolInfos() + { + return withExceptionHandling(() -> { + connect(); + Set threadPoolObjectNames = mbeanServerConn.queryNames(new ObjectName("org.apache.cassandra.metrics:type=ThreadPools,*"), null); + return threadPoolObjectNames.stream() + .map(oName -> new ThreadPoolInfo(oName.getKeyProperty("path"), oName.getKeyProperty("scope"))) + .collect(Collectors.toList()); + }); + } + + @Override + public List> findColumnFamilies(String type) + { + return withExceptionHandling(() -> { + assert type.equals("IndexColumnFamilies") || type.equals("ColumnFamilies"); + connect(); + + ObjectName query = new ObjectName("org.apache.cassandra.db:type=" + type + ",*"); + Set cfObjects = mbeanServerConn.queryNames(query, null); + + List> mbeans = new ArrayList<>(cfObjects.size()); + for (ObjectName objectName : cfObjects) + { + ColumnFamilyStoreMBean cfsProxy = JMX.newMBeanProxy(mbeanServerConn, objectName, ColumnFamilyStoreMBean.class); + mbeans.add(new AbstractMap.SimpleImmutableEntry<>(objectName.getKeyProperty("keyspace"), cfsProxy)); + } + return mbeans; + }); + } + + public JMXConnector getJmxConnector() + { + connect(); + return jmxc; + } + + public MBeanServerConnection getMBeanServerConnection() + { + connect(); + return mbeanServerConn; + } + + @Override + public void close() + { + if (jmxc == null) + return; + + try + { + jmxc.close(); + jmxc = null; + mbeanServerConn = null; + connected = false; + } + catch (IOException e) + { + // result of 'stopdaemon' command - i.e. if close() call fails, the daemon is shutdown + logger.error("Cassandra has shutdown."); + } + } + + private void registerMBeanProxy(Class clazz, String objectName) throws MalformedObjectNameException + { + registerMBean(clazz, JMX.newMBeanProxy(mbeanServerConn, new ObjectName(objectName), clazz)); + } + + private void registerPlatformMBeanProxy(Class clazz, String objectName) throws IOException + { + registerMBean(clazz, ManagementFactory.newPlatformMXBeanProxy(mbeanServerConn, objectName, clazz)); + } + + private RMIClientSocketFactory getRMIClientSocketFactory() + { + if (SSL_ENABLE.getBoolean()) + return new SslRMIClientSocketFactory(); + else + return RMISocketFactory.getDefaultSocketFactory(); + } + + private static T withExceptionHandling(MBeanSupplier op) + { + try + { + return op.get(); + } + catch (MalformedObjectNameException e) + { + throw new RuntimeException("Invalid ObjectName? Please report this as a bug.", e); + } + catch (IOException e) + { + throw new RuntimeException("Could not connect to MBean server. Please check that the JMX port is correct and open.", e); + } + } + + @FunctionalInterface + private interface MBeanSupplier + { + T get() throws MalformedObjectNameException, IOException; + } +} diff --git a/src/java/org/apache/cassandra/tools/RepairRunner.java b/src/java/org/apache/cassandra/tools/RepairRunner.java index acd784ac6080..147888ad53b4 100644 --- a/src/java/org/apache/cassandra/tools/RepairRunner.java +++ b/src/java/org/apache/cassandra/tools/RepairRunner.java @@ -25,6 +25,7 @@ import javax.management.ListenerNotFoundException; import javax.management.remote.JMXConnector; +import org.apache.cassandra.management.MBeanAccessor; import org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus; import org.apache.cassandra.service.StorageServiceMBean; import org.apache.cassandra.utils.Closeable; @@ -34,9 +35,9 @@ import org.apache.cassandra.utils.progress.jmx.JMXNotificationProgressListener; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.cassandra.config.CassandraRelevantProperties.NODETOOL_JMX_NOTIFICATION_POLL_INTERVAL_SECONDS; import static org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus.FAILED; import static org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus.valueOf; -import static org.apache.cassandra.tools.NodeProbe.JMX_NOTIFICATION_POLL_INTERVAL_SECONDS; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.LocalizeString.toLowerCaseLocalized; import static org.apache.cassandra.utils.concurrent.Condition.newOneTimeCondition; @@ -46,6 +47,8 @@ public class RepairRunner extends JMXNotificationProgressListener implements Closeable { + private final long JMX_NOTIFICATION_POLL_INTERVAL_SECONDS = NODETOOL_JMX_NOTIFICATION_POLL_INTERVAL_SECONDS.getLong(); + public static abstract class RepairCmd { private final String keyspace; @@ -60,6 +63,7 @@ public RepairCmd(String keyspace) private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS"); private final PrintStream out; + /** The connector to monitor the JMX connection state, so it can be {@code null} for server-side tools. */ private final JMXConnector jmxc; private final StorageServiceMBean ssProxy; private final Condition condition = newOneTimeCondition(); @@ -68,10 +72,10 @@ public RepairCmd(String keyspace) private Integer cmd; private volatile Exception error; - public RepairRunner(PrintStream out, JMXConnector jmxc, StorageServiceMBean ssProxy, RepairCmd repairCmd) + public RepairRunner(PrintStream out, MBeanAccessor accessor, StorageServiceMBean ssProxy, RepairCmd repairCmd) { this.out = out; - this.jmxc = jmxc; + this.jmxc = accessor instanceof RemoteJmxMBeanAccessor ? ((RemoteJmxMBeanAccessor) accessor).getJmxConnector() : null; this.ssProxy = ssProxy; this.repairCmd = repairCmd; } diff --git a/src/java/org/apache/cassandra/tools/nodetool/AbstractCommand.java b/src/java/org/apache/cassandra/tools/nodetool/AbstractCommand.java index 5c2530756392..3fb6c6b7b5aa 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/AbstractCommand.java +++ b/src/java/org/apache/cassandra/tools/nodetool/AbstractCommand.java @@ -73,7 +73,7 @@ public void run() * @return {@code true} if the command is required to connect to the node, {@code false} otherwise. * @throws ExecutionException if an error occurs during preparation and execution must be aborted. */ - protected boolean shouldConnect() throws ExecutionException + public boolean shouldConnect() throws ExecutionException { return true; } diff --git a/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java b/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java new file mode 100644 index 000000000000..7d758796954f --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java @@ -0,0 +1,107 @@ +/* + * 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.cassandra.tools.nodetool; + +import java.io.IOException; + +import com.google.common.base.Throwables; + +import org.apache.cassandra.transport.ProtocolVersion; +import org.apache.cassandra.transport.SimpleClient; + +import picocli.CommandLine.Command; +import picocli.CommandLine.InitializationException; +import picocli.CommandLine.Model.CommandSpec; +import picocli.CommandLine.Option; +import picocli.CommandLine.Spec; + +import static java.lang.Integer.parseInt; + +/** + * Command options for NodeTool commands that are executed via CQL. + */ +@Command(name = "cqlconnect", description = "Connect to a Cassandra node via CQL") +public class CqlConnect extends AbstractCommand implements AutoCloseable +{ + private static final int DEFAULT_CQL_PORT = 9042; + + /** The command specification, used to access command-specific properties. */ + @Spec + protected CommandSpec spec; // injected by picocli + + @Option(names = { "-h", "--host" }, description = "Node hostname or ip address", arity = "0..1") + private String host = "127.0.0.1"; + + @Option(names = { "-p", "--port" }, description = "Remote CQL native transport port number", arity = "0..1") + private String port = String.valueOf(DEFAULT_CQL_PORT); + + private volatile SimpleClient client; + + /** + * Initialize the CQL connection to the Cassandra node using the provided options. + */ + public void run() + { + if (client != null) + return; + + try + { + SimpleClient.Builder builder = SimpleClient.builder(host, parseInt(port)) + .protocolVersion(ProtocolVersion.V5); + client = builder.build(); + client.connect(false); + } + catch (IOException e) + { + Throwable rootCause = Throwables.getRootCause(e); + output.printError("nodetool: Failed to connect to '%s:%s' via CQL - %s: '%s'.%n", host, port, + rootCause.getClass().getSimpleName(), rootCause.getMessage()); + throw new InitializationException("Failed to connect to CQL", e); + } + } + + public SimpleClient client() + { + return client; + } + + @Override + protected void execute(org.apache.cassandra.tools.NodeProbe probe) + { + assert probe == null; + run(); + } + + @Override + public void close() throws Exception + { + if (client != null) + { + try + { + client.close(); + } + finally + { + client = null; + } + } + } +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/ForceCompact.java b/src/java/org/apache/cassandra/tools/nodetool/ForceCompact.java index 3e83ff2e062b..431bb94266d3 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/ForceCompact.java +++ b/src/java/org/apache/cassandra/tools/nodetool/ForceCompact.java @@ -61,6 +61,7 @@ public void execute(NodeProbe probe) try { probe.forceCompactionKeysIgnoringGcGrace(keyspaceName, tableName, partitionKeysIgnoreGcGrace); + probe.output().printInfo("Force compaction performed for keyspace '%s', table '%s'", keyspaceName, tableName); } catch (Exception e) { diff --git a/src/java/org/apache/cassandra/tools/nodetool/History.java b/src/java/org/apache/cassandra/tools/nodetool/History.java index 15ece16df2ba..105029e69532 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/History.java +++ b/src/java/org/apache/cassandra/tools/nodetool/History.java @@ -51,7 +51,7 @@ protected void execute(NodeProbe probe) } @Override - protected boolean shouldConnect() throws CommandLine.ExecutionException + public boolean shouldConnect() throws CommandLine.ExecutionException { return false; } diff --git a/src/java/org/apache/cassandra/tools/nodetool/JmxConnect.java b/src/java/org/apache/cassandra/tools/nodetool/JmxConnect.java index 4fec76e348c6..4a5c02d6bcdd 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/JmxConnect.java +++ b/src/java/org/apache/cassandra/tools/nodetool/JmxConnect.java @@ -22,7 +22,6 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.UncheckedIOException; -import java.util.List; import java.util.Scanner; import javax.inject.Inject; @@ -33,20 +32,14 @@ import org.apache.cassandra.tools.INodeProbeFactory; import org.apache.cassandra.tools.NodeProbe; -import picocli.CommandLine; import picocli.CommandLine.Command; -import picocli.CommandLine.ExecutionException; -import picocli.CommandLine.IExecutionStrategy; import picocli.CommandLine.InitializationException; import picocli.CommandLine.Model.CommandSpec; import picocli.CommandLine.Option; -import picocli.CommandLine.ParameterException; -import picocli.CommandLine.ParseResult; -import picocli.CommandLine.RunLast; import picocli.CommandLine.Spec; import static java.lang.Integer.parseInt; -import static org.apache.cassandra.tools.NodeProbe.defaultPort; +import static org.apache.cassandra.tools.RemoteJmxMBeanAccessor.defaultPort; import static org.apache.commons.lang3.StringUtils.EMPTY; import static org.apache.commons.lang3.StringUtils.isEmpty; import static org.apache.commons.lang3.StringUtils.isNotEmpty; @@ -57,8 +50,6 @@ @Command(name = "connect", description = "Connect to a Cassandra node via JMX") public class JmxConnect extends AbstractCommand implements AutoCloseable { - public static final String MIXIN_KEY = "jmx"; - /** The command specification, used to access command-specific properties. */ @Spec protected CommandSpec spec; // injected by picocli @@ -81,30 +72,6 @@ public class JmxConnect extends AbstractCommand implements AutoCloseable @Inject private INodeProbeFactory nodeProbeFactory; - /** - * This method is called by picocli and used depending on the execution strategy. - * @param parseResult The parsed command line. - * @return The exit code. - */ - public static int executionStrategy(ParseResult parseResult) - { - CommandSpec jmx = parseResult.commandSpec().mixins().get(MIXIN_KEY); - if (jmx == null) - throw new InitializationException("No JmxConnect command found in the top-level hierarchy"); - - try (JmxConnectionCommandInvoker invoker = new JmxConnectionCommandInvoker((JmxConnect) jmx.userObject())) - { - return invoker.execute(parseResult); - } - catch (JmxConnectionCommandInvoker.CloseException e) - { - jmx.commandLine() - .getErr() - .println("Failed to connect to JMX: " + e.getMessage()); - return jmx.commandLine().getExitCodeExceptionMapper().getExitCode(e); - } - } - /** * Initialize the JMX connection to the Cassandra node using the provided options. */ @@ -183,61 +150,13 @@ private static String promptAndReadPassword() return password; } - private static class JmxConnectionCommandInvoker implements IExecutionStrategy, AutoCloseable + public String getHost() { - private final JmxConnect connect; - - public JmxConnectionCommandInvoker(JmxConnect connect) - { - this.connect = connect; - } - - @Override - public int execute(ParseResult parseResult) throws ExecutionException, ParameterException - { - CommandSpec lastParent = lastExecutableSubcommandWithSameParent(parseResult.asCommandLineList()); - if (lastParent.userObject() instanceof AbstractCommand) - { - AbstractCommand command = (AbstractCommand) lastParent.userObject(); - if (command.shouldConnect()) - connect.run(); - command.probe(connect.probe()); - } - return new RunLast().execute(parseResult); - } - - @Override - public void close() throws CloseException - { - try - { - if (connect.probe() != null) - ((AutoCloseable) connect.probe()).close(); - } - catch (Exception e) - { - throw new CloseException("Failed to close JMX connection", e); - } - } - - private static CommandLine.Model.CommandSpec lastExecutableSubcommandWithSameParent(List parsedCommands) - { - int start = parsedCommands.size() - 1; - for (int i = parsedCommands.size() - 2; i >= 0; i--) - { - if (parsedCommands.get(i).getParent() != parsedCommands.get(i + 1).getParent()) - break; - start = i; - } - return parsedCommands.get(start).getCommandSpec(); - } + return host; + } - private static class CloseException extends RuntimeException - { - public CloseException(String message, Throwable cause) - { - super(message, cause); - } - } + public String getPort() + { + return port; } } diff --git a/src/java/org/apache/cassandra/tools/nodetool/Sjk.java b/src/java/org/apache/cassandra/tools/nodetool/Sjk.java index 1f2d5db77704..950836b56bb7 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Sjk.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Sjk.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.io.PrintStream; +import java.lang.management.ManagementFactory; import java.lang.reflect.Field; import java.net.URL; import java.net.URLDecoder; @@ -47,6 +48,7 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.Output; +import org.apache.cassandra.tools.RemoteJmxMBeanAccessor; import picocli.CommandLine; import picocli.CommandLine.Command; @@ -64,7 +66,7 @@ public class Sjk extends AbstractCommand private final Wrapper wrapper = new Wrapper(); @Override - protected boolean shouldConnect() throws ExecutionException + public boolean shouldConnect() throws ExecutionException { // We want to parse the given arguments in advance to determine if the SJK command requires an MBeanServerConnection or not. wrapper.prepare(args.isEmpty() ? new String[]{ "--help" } : args.toArray(new String[0]), output.out, output.err); @@ -242,7 +244,9 @@ private void setJmxConnInfo(final NodeProbe probe) throws IllegalAccessException { public MBeanServerConnection getMServer() { - return probe.getMbeanServerConn(); + return probe.getMBeanAccessor() instanceof RemoteJmxMBeanAccessor ? + ((RemoteJmxMBeanAccessor) probe.getMBeanAccessor()).getMBeanServerConnection() : + ManagementFactory.getPlatformMBeanServer(); } }); } diff --git a/src/java/org/apache/cassandra/tools/nodetool/WithPortDisplayAbstractCommand.java b/src/java/org/apache/cassandra/tools/nodetool/WithPortDisplayAbstractCommand.java index fe67f126f3f3..d12e26371d8e 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/WithPortDisplayAbstractCommand.java +++ b/src/java/org/apache/cassandra/tools/nodetool/WithPortDisplayAbstractCommand.java @@ -32,7 +32,7 @@ abstract class WithPortDisplayAbstractCommand extends AbstractCommand protected boolean printPort; @Override - protected boolean shouldConnect() + public boolean shouldConnect() { printPort = parent.printPort; return true; diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/CommandExecutionStraregy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/CommandExecutionStraregy.java new file mode 100644 index 000000000000..3e572749d721 --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/CommandExecutionStraregy.java @@ -0,0 +1,42 @@ +/* + * 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.cassandra.tools.nodetool.strategy; + +import picocli.CommandLine; + +public interface CommandExecutionStraregy extends CommandLine.IExecutionStrategy, AutoCloseable +{ + @Override + default void close() throws ExecutionStrategyCloseException {} + + class ExecutionStrategyCloseException extends RuntimeException + { + public ExecutionStrategyCloseException(String message, Throwable cause) + { + super(message, cause); + } + } + + enum Type + { + CQL, + STATIC_MBEAN, + COMMAND_MBEAN, + } +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/CommandMBeanExecutionStrategy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/CommandMBeanExecutionStrategy.java new file mode 100644 index 000000000000..fa3a50a67706 --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/CommandMBeanExecutionStrategy.java @@ -0,0 +1,162 @@ +/* + * 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.cassandra.tools.nodetool.strategy; + +import java.util.List; + +import javax.management.InstanceNotFoundException; +import javax.management.MBeanException; +import javax.management.MBeanServerConnection; +import javax.management.ObjectName; +import javax.management.ReflectionException; + +import org.apache.cassandra.management.CommandExecutionArgsSerde; +import org.apache.cassandra.management.CommandMBean; +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.picocli.PicocliCommandArgsConverter; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.RemoteJmxMBeanAccessor; +import org.apache.cassandra.tools.nodetool.AbstractCommand; +import org.apache.cassandra.tools.nodetool.JmxConnect; + +import picocli.CommandLine; + +public class CommandMBeanExecutionStrategy implements CommandExecutionStraregy +{ + private static final String MBEAN_DOMAIN = "org.apache.cassandra.management"; + private static final String MBEAN_TYPE_COMMAND = "Command"; + + private final JmxConnect connect; + + public CommandMBeanExecutionStrategy(JmxConnect connect) + { + this.connect = connect; + } + + @Override + public int execute(CommandLine.ParseResult parseResult) throws CommandLine.ExecutionException, CommandLine.ParameterException + { + CommandLine.Model.CommandSpec lastParent = StaticMBeanExecutionStrategy.lastExecutableSubcommandWithSameParent(parseResult.asCommandLineList()); + + NodeProbe probe = null; + Object userObject = lastParent.userObject(); + + if (userObject instanceof AbstractCommand) + { + AbstractCommand command = (AbstractCommand) userObject; + if (command.shouldConnect()) + connect.run(); + probe = connect.probe(); + } + + // Local command execution with no JMX connection. + if (probe == null || parseResult.isUsageHelpRequested() || parseResult.isVersionHelpRequested()) + return new CommandLine.RunLast().execute(parseResult); + + String commandName = extractCommandName(parseResult); + if (commandName == null || commandName.isEmpty()) + return new CommandLine.RunLast().execute(parseResult); + + // Command is already populated with args from CommandLine.parseArgs(), convert to CommandExecutionArgs + CommandExecutionArgs args = PicocliCommandArgsConverter.fromCommand(userObject); + String jsonParams = CommandExecutionArgsSerde.toJson(args); + + try + { + String result = executeViaRemoteMBean(((RemoteJmxMBeanAccessor) probe.getMBeanAccessor()).getMBeanServerConnection(), + commandName, jsonParams); + + probe.output().out.println(result); + return 0; + } + catch (Exception e) + { + throw new CommandLine.ExecutionException(parseResult.commandSpec().commandLine(), + "Failed to execute command via MBean: " + e.getMessage(), e); + } + } + + @Override + public void close() throws ExecutionStrategyCloseException + { + if (connect.probe() == null) + return; + try + { + connect.probe().close(); + } + catch (Exception e) + { + throw new ExecutionStrategyCloseException("Failed to close JMX connection", e); + } + } + + /** + * Handles nested commands (e.g., "compressiondictionary.train"). + */ + private String extractCommandName(CommandLine.ParseResult parseResult) + { + List commandLineList = parseResult.asCommandLineList(); + if (commandLineList.size() <= 1) // Only root "nodetool" + return null; + + StringBuilder commandName = new StringBuilder(); + for (int i = 1; i < commandLineList.size(); i++) + { + CommandLine cmdLine = commandLineList.get(i); + String name = cmdLine.getCommandName(); + if (name != null && !name.isEmpty()) + { + if (commandName.length() > 0) + commandName.append("."); + commandName.append(name); + } + } + + return commandName.length() > 0 ? commandName.toString() : null; + } + + private String executeViaRemoteMBean(MBeanServerConnection mbs, String commandName, String jsonParams) throws Exception + { + ObjectName mbeanName = constructCommandMBeanName(commandName); + + if (!mbs.isRegistered(mbeanName)) + throw new InstanceNotFoundException("Command MBean not found: " + mbeanName); + + try + { + Object result = mbs.invoke(mbeanName, CommandMBean.INVOKE_METHOD, + new Object[]{ jsonParams }, + new String[]{ String.class.getName() }); + + return result != null ? result.toString() : ""; + } + catch (MBeanException | ReflectionException e) + { + throw new RuntimeException("Failed to invoke CommandMBean: " + commandName, e); + } + } + + private ObjectName constructCommandMBeanName(String commandName) throws Exception + { + String escapedName = ObjectName.quote(commandName); + String objectNameStr = String.format("%s:type=%s,name=%s", MBEAN_DOMAIN, MBEAN_TYPE_COMMAND, escapedName); + return new ObjectName(objectNameStr); + } +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java new file mode 100644 index 000000000000..53094724cf5d --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java @@ -0,0 +1,230 @@ +/* + * 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.cassandra.tools.nodetool.strategy; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; + +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.management.CommandExecutionArgsSerde; +import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.picocli.PicocliCommandArgsConverter; +import org.apache.cassandra.tools.nodetool.AbstractCommand; +import org.apache.cassandra.tools.nodetool.CqlConnect; +import org.apache.cassandra.transport.messages.ResultMessage; + +import picocli.CommandLine; + +public class CqlCommandExecutionStrategy implements CommandExecutionStraregy +{ + private final CqlConnect connect; + + public CqlCommandExecutionStrategy(CqlConnect connect) + { + this.connect = connect; + } + + @Override + public int execute(CommandLine.ParseResult parseResult) throws CommandLine.ExecutionException, CommandLine.ParameterException + { + CommandLine.Model.CommandSpec lastParent = StaticMBeanExecutionStrategy.lastExecutableSubcommandWithSameParent(parseResult.asCommandLineList()); + + Object userObject = lastParent.userObject(); + + if (userObject instanceof AbstractCommand) + { + AbstractCommand command = (AbstractCommand) userObject; + if (command.shouldConnect()) + connect.run(); + } + + // Local command execution with no CQL connection. + if (connect.client() == null || parseResult.isUsageHelpRequested() || parseResult.isVersionHelpRequested()) + return new CommandLine.RunLast().execute(parseResult); + + String commandName = extractCommandName(parseResult); + if (commandName == null || commandName.isEmpty()) + return new CommandLine.RunLast().execute(parseResult); + + // Command is already populated with args from CommandLine.parseArgs(), convert to CommandExecutionArgs + CommandExecutionArgs args = PicocliCommandArgsConverter.fromCommand(userObject); + String jsonParams = CommandExecutionArgsSerde.toJson(args); + + try + { + String cqlCommand = buildCqlCommandString(commandName, jsonParams); + ResultMessage result = connect.client().execute(cqlCommand, ConsistencyLevel.ONE); + + if (result instanceof ResultMessage.Rows) + { + ResultMessage.Rows rows = (ResultMessage.Rows) result; + if (!rows.result.isEmpty() && !rows.result.metadata.names.isEmpty()) + { + List firstRow = rows.result.rows.get(0); + if (!firstRow.isEmpty()) + { + String output = UTF8Type.instance.getSerializer().deserialize(firstRow.get(0)); + // For CQL strategy, we don't have a NodeProbe, so use CommandLine's output directly + parseResult.commandSpec().commandLine().getOut().println(output); + } + } + } + + return 0; + } + catch (Exception e) + { + throw new CommandLine.ExecutionException(parseResult.commandSpec().commandLine(), + "Failed to execute command via CQL: " + e.getMessage(), e); + } + } + + @Override + public void close() throws ExecutionStrategyCloseException + { + if (connect == null) + return; + try + { + connect.close(); + } + catch (Exception e) + { + throw new ExecutionStrategyCloseException("Failed to close CQL connection", e); + } + } + + /** + * Handles nested commands (e.g., "compressiondictionary.train"). + */ + private String extractCommandName(CommandLine.ParseResult parseResult) + { + List commandLineList = parseResult.asCommandLineList(); + if (commandLineList.size() <= 1) // Only root "nodetool" + return null; + + StringBuilder commandName = new StringBuilder(); + for (int i = 1; i < commandLineList.size(); i++) + { + CommandLine cmdLine = commandLineList.get(i); + String name = cmdLine.getCommandName(); + if (name != null && !name.isEmpty()) + { + if (commandName.length() > 0) + commandName.append("."); + commandName.append(name); + } + } + + return commandName.length() > 0 ? commandName.toString() : null; + } + + /** + * Build a CQL COMMAND statement string from command name and JSON parameters. + * The JSON is parsed and converted to the WITH clause format: + * COMMAND commandName WITH "key1" = 'value1' AND "key2" = 'value2'; + */ + private String buildCqlCommandString(String commandName, String jsonParams) + { + Map paramsMap = org.apache.cassandra.utils.JsonUtils.fromJsonMap(jsonParams); + + StringBuilder cql = new StringBuilder("COMMAND "); + cql.append(commandName); + + if (!paramsMap.isEmpty()) + { + cql.append(" WITH"); + boolean first = true; + for (Map.Entry entry : paramsMap.entrySet()) + { + if (first) + cql.append(" "); + else + cql.append(" AND "); + + String key = entry.getKey(); + Object value = entry.getValue(); + + // Quote the key if it contains special characters + cql.append('"').append(escapeString(key)).append('"'); + cql.append(" = "); + cql.append(formatCqlValue(value)); + + first = false; + } + } + + cql.append(";"); + return cql.toString(); + } + + private String formatCqlValue(Object value) + { + if (value == null) + return "NULL"; + + if (value instanceof String) + { + return "'" + escapeSingleQuotes((String) value) + "'"; + } + else if (value instanceof Number) + { + return value.toString(); + } + else if (value instanceof Boolean) + { + return value.toString(); + } + else if (value instanceof List) + { + @SuppressWarnings("unchecked") + List list = (List) value; + StringBuilder sb = new StringBuilder("["); + for (int i = 0; i < list.size(); i++) + { + if (i > 0) + sb.append(", "); + Object item = list.get(i); + if (item instanceof String) + sb.append("'").append(escapeSingleQuotes((String) item)).append("'"); + else + sb.append(formatCqlValue(item)); + } + sb.append("]"); + return sb.toString(); + } + else + { + // For other types, convert to string and quote + return "'" + escapeSingleQuotes(value.toString()) + "'"; + } + } + + private static String escapeSingleQuotes(String str) + { + return str.replace("'", "''"); + } + + private static String escapeString(String str) + { + return str.replace("\"", "\\\""); + } +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java new file mode 100644 index 000000000000..de34119abe4b --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java @@ -0,0 +1,135 @@ +/* + * 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.cassandra.tools.nodetool.strategy; + +import java.lang.reflect.Field; + +import org.apache.cassandra.config.CassandraRelevantEnv; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.tools.nodetool.AbstractCommand; +import org.apache.cassandra.tools.nodetool.CqlConnect; +import org.apache.cassandra.tools.nodetool.JmxConnect; + +import picocli.CommandLine; + +import static org.apache.cassandra.utils.LocalizeString.toUpperCaseLocalized; + +public class ProtocolAwareExecutionStrategy implements CommandExecutionStraregy +{ + public static final String MIXIN_KEY = "jmx"; + + public static int executionStrategy(CommandLine.ParseResult parseResult) + { + try (ProtocolAwareExecutionStrategy strategy = new ProtocolAwareExecutionStrategy()) + { + return strategy.execute(parseResult); + } + } + + /** + * This method is called by picocli and used depending on the execution strategy. + * @param parseResult The parsed command line. + * @return The exit code. + */ + @Override + public int execute(CommandLine.ParseResult parseResult) + { + CommandLine.Model.CommandSpec jmx = parseResult.commandSpec().mixins().get(MIXIN_KEY); + if (jmx == null) + throw new CommandLine.InitializationException("No JmxConnect command found in the top-level hierarchy"); + + try (CommandExecutionStraregy invoker = createInvoker((JmxConnect) jmx.userObject())) + { + return invoker.execute(parseResult); + } + catch (ExecutionStrategyCloseException e) + { + jmx.commandLine() + .getErr() + .println("Failed to connect: " + e.getMessage()); + return jmx.commandLine().getExitCodeExceptionMapper().getExitCode(e); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + } + + static CommandExecutionStraregy createInvoker(JmxConnect jmxConnect) + { + Type defaultStrategy = Type.valueOf(toUpperCaseLocalized(CassandraRelevantProperties.CASSANDRA_CLI_EXECUTION_PROTOCOL.getDefaultValue())); + Type strategyEnv = CassandraRelevantEnv.CASSANDRA_CLI_EXECUTION_PROTOCOL.getEnum(true, Type.class, defaultStrategy.name()); + Type strategySys = CassandraRelevantProperties.CASSANDRA_CLI_EXECUTION_PROTOCOL.getEnum(true, Type.class); + Type strategy = strategyEnv != defaultStrategy ? strategyEnv : strategySys; + + switch (strategy) + { + case CQL: + CqlConnect cqlConnect = createCqlConnectFromJmxConnect(jmxConnect); + return new CqlCommandExecutionStrategy(cqlConnect); + case COMMAND_MBEAN: + return new CommandMBeanExecutionStrategy(jmxConnect); + case STATIC_MBEAN: + return new StaticMBeanExecutionStrategy(jmxConnect); + default: + throw new IllegalStateException("Unknown execution strategy: " + strategy); + } + } + + private static CqlConnect createCqlConnectFromJmxConnect(JmxConnect jmxConnect) + { + // Initialize DatabaseDescriptor before creating CqlConnect to ensure it's ready + // when transport classes (like Envelope.Decoder) are loaded. This is critical + // for forked nodetool processes where DatabaseDescriptor might not be initialized. + if (!DatabaseDescriptor.isClientOrToolInitialized()) + DatabaseDescriptor.clientInitialization(false); + + try + { + // TODO This is a bit of a hack to transfer the output from JmxConnect to CqlConnect. + // Consider refactoring Output handling in nodetool commands to be more flexible. + Field outputField = AbstractCommand.class.getDeclaredField("output"); + outputField.setAccessible(true); + org.apache.cassandra.tools.Output output = (org.apache.cassandra.tools.Output) outputField.get(jmxConnect); + + CqlConnect cqlConnect = new CqlConnect(); + Field cqlHostField = CqlConnect.class.getDeclaredField("host"); + cqlHostField.setAccessible(true); + cqlHostField.set(cqlConnect, jmxConnect.getHost()); + + Field cqlPortField = CqlConnect.class.getDeclaredField("port"); + cqlPortField.setAccessible(true); + cqlPortField.set(cqlConnect, jmxConnect.getPort()); + + if (output != null) + { + Field cqlOutputField = CqlConnect.class.getSuperclass().getDeclaredField("output"); + cqlOutputField.setAccessible(true); + cqlOutputField.set(cqlConnect, output); + } + + return cqlConnect; + } + catch (Exception e) + { + throw new RuntimeException("Failed to create CqlConnect from JmxConnect", e); + } + } +} diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/StaticMBeanExecutionStrategy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/StaticMBeanExecutionStrategy.java new file mode 100644 index 000000000000..c0d787c1bf2e --- /dev/null +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/StaticMBeanExecutionStrategy.java @@ -0,0 +1,77 @@ +/* + * 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.cassandra.tools.nodetool.strategy; + +import java.util.List; + +import org.apache.cassandra.tools.nodetool.AbstractCommand; +import org.apache.cassandra.tools.nodetool.JmxConnect; + +import picocli.CommandLine; + +public class StaticMBeanExecutionStrategy implements CommandExecutionStraregy +{ + private final JmxConnect connect; + + public StaticMBeanExecutionStrategy(JmxConnect connect) + { + this.connect = connect; + } + + @Override + public int execute(CommandLine.ParseResult parseResult) throws CommandLine.ExecutionException, CommandLine.ParameterException + { + CommandLine.Model.CommandSpec lastParent = lastExecutableSubcommandWithSameParent(parseResult.asCommandLineList()); + if (lastParent.userObject() instanceof AbstractCommand) + { + AbstractCommand command = (AbstractCommand) lastParent.userObject(); + if (command.shouldConnect()) + connect.run(); + command.probe(connect.probe()); + } + return new CommandLine.RunLast().execute(parseResult); + } + + @Override + public void close() throws ExecutionStrategyCloseException + { + if (connect.probe() == null) + return; + try + { + connect.probe().close(); + } + catch (Exception e) + { + throw new ExecutionStrategyCloseException("Failed to close JMX connection", e); + } + } + + static CommandLine.Model.CommandSpec lastExecutableSubcommandWithSameParent(List parsedCommands) + { + int start = parsedCommands.size() - 1; + for (int i = parsedCommands.size() - 2; i >= 0; i--) + { + if (parsedCommands.get(i).getParent() != parsedCommands.get(i + 1).getParent()) + break; + start = i; + } + return parsedCommands.get(start).getCommandSpec(); + } +} diff --git a/src/java/org/apache/cassandra/utils/JsonUtils.java b/src/java/org/apache/cassandra/utils/JsonUtils.java index 3c589c2c5b71..6ebf6e97d737 100644 --- a/src/java/org/apache/cassandra/utils/JsonUtils.java +++ b/src/java/org/apache/cassandra/utils/JsonUtils.java @@ -222,4 +222,51 @@ public static void handleCaseSensitivity(Map valueMap) valueMap.put(lowered, valueMap.remove(mapKey)); } } + + public static String getJsonType(Class type) + { + if (type == String.class) + return "string"; + else if (type == int.class || type == Integer.class) + return "integer"; + else if (type == long.class || type == Long.class) + return "integer"; + else if (type == boolean.class || type == Boolean.class) + return "boolean"; + else if (type == double.class || type == Double.class || + type == float.class || type == Float.class) + return "number"; + else if (type.isArray() || List.class.isAssignableFrom(type)) + return "array"; + else if (Map.class.isAssignableFrom(type)) + return "object"; + else if (type.isEnum()) + return "string"; + else + return "string"; + } + + public static Object convertDefaultValue(String defaultValue, Class type) + { + try + { + if (type == boolean.class || type == Boolean.class) + return Boolean.parseBoolean(defaultValue); + else if (type == int.class || type == Integer.class) + return Integer.parseInt(defaultValue); + else if (type == long.class || type == Long.class) + return Long.parseLong(defaultValue); + else if (type == double.class || type == Double.class) + return Double.parseDouble(defaultValue); + else if (type == float.class || type == Float.class) + return Float.parseFloat(defaultValue); + else + return defaultValue; + } + catch (Exception e) + { + // Fall back to string default value if parsing fails. + return defaultValue; + } + } } diff --git a/src/resources/META-INF/services/org.apache.cassandra.management.api.CommandsProvider b/src/resources/META-INF/services/org.apache.cassandra.management.api.CommandsProvider new file mode 100644 index 000000000000..4a25d60f4aae --- /dev/null +++ b/src/resources/META-INF/services/org.apache.cassandra.management.api.CommandsProvider @@ -0,0 +1,20 @@ +# 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. +# +# + +org.apache.cassandra.management.picocli.PicocliCommandsProvider + diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index ac1da245d569..5496839c484c 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -118,6 +118,7 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.PathUtils; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.management.CommandService; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.Sampler; import org.apache.cassandra.metrics.ThreadLocalMetrics; @@ -1023,6 +1024,7 @@ public Future shutdown(boolean runOnExitThreads, boolean shutdownMessaging SnapshotManager.instance::close, () -> IndexStatusManager.instance.shutdownAndWait(1L, MINUTES), DiskErrorsHandlerService::close, + CommandService::shutdown, () -> ThreadLocalMetrics.shutdownCleaner(1L, MINUTES) ); diff --git a/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java b/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java index 1ab4f843b7fb..a7fb18463973 100644 --- a/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java +++ b/test/distributed/org/apache/cassandra/distributed/mock/nodetool/InternalNodeProbe.java @@ -19,77 +19,66 @@ package org.apache.cassandra.distributed.mock.nodetool; import java.lang.management.ManagementFactory; +import java.lang.management.MemoryMXBean; +import java.lang.management.RuntimeMXBean; +import java.util.HashMap; import java.util.Iterator; +import java.util.List; import java.util.Map; import com.google.common.collect.Multimap; import org.apache.cassandra.batchlog.BatchlogManager; +import org.apache.cassandra.batchlog.BatchlogManagerMBean; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.ColumnFamilyStoreMBean; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.compaction.CompactionManager; +import org.apache.cassandra.db.compression.CompressionDictionaryManagerMBean; import org.apache.cassandra.gms.FailureDetector; import org.apache.cassandra.gms.FailureDetectorMBean; import org.apache.cassandra.gms.Gossiper; +import org.apache.cassandra.gms.GossiperMBean; import org.apache.cassandra.hints.HintsService; +import org.apache.cassandra.hints.HintsServiceMBean; import org.apache.cassandra.locator.DynamicEndpointSnitch; import org.apache.cassandra.locator.DynamicEndpointSnitchMBean; import org.apache.cassandra.locator.EndpointSnitchInfo; import org.apache.cassandra.locator.EndpointSnitchInfoMBean; import org.apache.cassandra.locator.SnitchAdapter; +import org.apache.cassandra.management.MBeanAccessor; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.MessagingServiceMBean; import org.apache.cassandra.service.ActiveRepairService; +import org.apache.cassandra.service.ActiveRepairServiceMBean; import org.apache.cassandra.service.CacheService; import org.apache.cassandra.service.CacheServiceMBean; import org.apache.cassandra.service.GCInspector; +import org.apache.cassandra.service.GCInspectorMXBean; import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.service.StorageProxyMBean; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.StorageServiceMBean; import org.apache.cassandra.service.accord.AccordOperations; +import org.apache.cassandra.service.accord.AccordOperationsMBean; import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.service.snapshot.SnapshotManagerMBean; import org.apache.cassandra.streaming.StreamManager; +import org.apache.cassandra.streaming.StreamManagerMBean; import org.apache.cassandra.tcm.CMSOperations; +import org.apache.cassandra.tcm.CMSOperationsMBean; import org.apache.cassandra.tools.NodeProbe; public class InternalNodeProbe extends NodeProbe { - private final boolean withNotifications; private boolean previousSkipNotificationListeners = false; public InternalNodeProbe(boolean withNotifications) { - this.withNotifications = withNotifications; - connect(); - } - - protected void connect() - { - // note that we are not connecting via JMX for testing - mbeanServerConn = null; - jmxc = null; - + super(new TestMockMBeanAccessor()); // host/port are unused in InternalNodeProbe previousSkipNotificationListeners = StorageService.instance.skipNotificationListeners; StorageService.instance.skipNotificationListeners = !withNotifications; - - ssProxy = StorageService.instance; - snapshotProxy = SnapshotManager.instance; - cmsProxy = CMSOperations.instance; - accordProxy = AccordOperations.instance; - msProxy = MessagingService.instance(); - streamProxy = StreamManager.instance; - compactionProxy = CompactionManager.instance; - fdProxy = (FailureDetectorMBean) FailureDetector.instance; - cacheService = CacheService.instance; - spProxy = StorageProxy.instance; - hsProxy = HintsService.instance; - - gcProxy = new GCInspector(); - gossProxy = Gossiper.instance; - bmProxy = BatchlogManager.instance; - arsProxy = ActiveRepairService.instance(); - memProxy = ManagementFactory.getMemoryMXBean(); - runtimeProxy = ManagementFactory.getRuntimeMXBean(); } @Override @@ -183,4 +172,71 @@ public long getStorageMetric(String metricName) { throw new UnsupportedOperationException(); } + + private static class TestMockMBeanAccessor implements MBeanAccessor + { + private final Map, Object> mbeanRegistry = new HashMap<>(); + + public TestMockMBeanAccessor() + { + registerMBean(StorageServiceMBean.class, StorageService.instance); + registerMBean(SnapshotManagerMBean.class, SnapshotManager.instance); + registerMBean(CMSOperationsMBean.class, CMSOperations.instance); + registerMBean(AccordOperationsMBean.class, AccordOperations.instance); + registerMBean(MessagingServiceMBean.class, MessagingService.instance()); + registerMBean(StreamManagerMBean.class, StreamManager.instance); + registerMBean(CompactionManager.class, CompactionManager.instance); + registerMBean(FailureDetectorMBean.class, (FailureDetectorMBean) FailureDetector.instance); + registerMBean(CacheServiceMBean.class, CacheService.instance); + registerMBean(StorageProxyMBean.class, StorageProxy.instance); + registerMBean(HintsServiceMBean.class, HintsService.instance); + registerMBean(GCInspectorMXBean.class, new GCInspector()); + registerMBean(GossiperMBean.class, Gossiper.instance); + registerMBean(BatchlogManagerMBean.class, BatchlogManager.instance); + registerMBean(ActiveRepairServiceMBean.class, ActiveRepairService.instance()); + registerMBean(MemoryMXBean.class, ManagementFactory.getMemoryMXBean()); + registerMBean(RuntimeMXBean.class, ManagementFactory.getRuntimeMXBean()); + } + + protected void registerMBean(Class clazz, T mbean) + { + mbeanRegistry.put(clazz, mbean); + } + + @Override + public T findMBean(Class clazz) + { + return mbeanRegistry.get(clazz) == null ? null : clazz.cast(mbeanRegistry.get(clazz)); + } + + @Override + public T findMBeanMetric(Class clazz, Props props) + { + return null; + } + + @Override + public ColumnFamilyStoreMBean findColumnFamily(String type, String keyspace, String columnFamily) + { + return null; + } + + @Override + public CompressionDictionaryManagerMBean findCompressionDictionary(String keyspace, String table) + { + return null; + } + + @Override + public List threadPoolInfos() + { + return List.of(); + } + + @Override + public List> findColumnFamilies(String type) + { + return List.of(); + } + } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/management/JmxManagementDistributedTest.java b/test/distributed/org/apache/cassandra/distributed/test/management/JmxManagementDistributedTest.java deleted file mode 100644 index 02a9da943b2b..000000000000 --- a/test/distributed/org/apache/cassandra/distributed/test/management/JmxManagementDistributedTest.java +++ /dev/null @@ -1,89 +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 org.apache.cassandra.distributed.test.management; - -import java.util.HashSet; -import java.util.Set; -import javax.management.MBeanServerConnection; -import javax.management.remote.JMXConnector; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.cassandra.distributed.Cluster; -import org.apache.cassandra.distributed.api.Feature; -import org.apache.cassandra.distributed.api.IInstanceConfig; -import org.apache.cassandra.distributed.api.IInvokableInstance; -import org.apache.cassandra.distributed.impl.INodeProvisionStrategy; -import org.apache.cassandra.distributed.shared.JMXUtil; -import org.apache.cassandra.distributed.test.TestBaseImpl; - -import static org.apache.cassandra.distributed.test.jmx.JMXGetterCheckTest.testAllValidGetters; -import static org.hamcrest.Matchers.startsWith; - -public class JmxManagementDistributedTest extends TestBaseImpl -{ - @Test - public void testOneNetworkInterfaceProvisioning() throws Exception - { - Set allInstances = new HashSet<>(); - int iterations = 2; // Make sure the JMX infrastructure all cleans up properly by running this multiple times. - - try (Cluster cluster = Cluster.build(1) - .withDynamicPortAllocation(true) - .withNodeProvisionStrategy(INodeProvisionStrategy.Strategy.OneNetworkInterface) - .withConfig(c -> c.with(Feature.values())).start()) - { - Set instancesContacted = new HashSet<>(); - for (IInvokableInstance instance : cluster) - { - testInstance(instancesContacted, instance); - } - - while (!Thread.currentThread().isInterrupted()) - { - Thread.sleep(1000); - System.out.println(">>>>> sleeping"); - } - allInstances.addAll(instancesContacted); - // Make sure we actually exercise the mbeans by testing a bunch of getters. - // Without this it's possible for the test to pass as we don't touch any mBeans that we register. - testAllValidGetters(cluster); - } - Assert.assertEquals("Each instance from each cluster should have been unique", iterations * 2, allInstances.size()); - } - - private void testInstance(Set instancesContacted, IInvokableInstance instance) - { - IInstanceConfig config = instance.config(); - try (JMXConnector jmxc = JMXUtil.getJmxConnector(config)) - { - MBeanServerConnection mbsc = jmxc.getMBeanServerConnection(); - // instances get their default domain set to their IP address, so us it - // to check that we are actually connecting to the correct instance - String defaultDomain = mbsc.getDefaultDomain(); - instancesContacted.add(defaultDomain); - Assert.assertThat(defaultDomain, startsWith(JMXUtil.getJmxHost(config) + ':' + config.jmxPort())); - } - catch (Throwable t) - { - throw new RuntimeException("Could not connect to JMX", t); - } - } -} diff --git a/test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java b/test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java new file mode 100644 index 000000000000..29ea5702be15 --- /dev/null +++ b/test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java @@ -0,0 +1,64 @@ +/* + * 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.cassandra.cql3; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.shared.WithProperties; +import org.apache.cassandra.tools.NodeTool; +import org.apache.cassandra.tools.ToolRunner; +import org.apache.cassandra.tools.nodetool.strategy.CommandExecutionStraregy; + +@RunWith(Parameterized.class) +public abstract class CQLNodetoolProtocolTester extends CQLTester +{ + @Parameterized.Parameter + public CommandExecutionStraregy.Type strategy; + + @Parameterized.Parameters(name = "runner={0}") + public static Collection data() + { + List params = new ArrayList<>(); + for (CommandExecutionStraregy.Type type : CommandExecutionStraregy.Type.values()) + params.add(new Object[]{type}); + return params; + } + + public ToolRunner.ToolResult invokeNodetool(String... args) + { + // Use invokeNodetoolInJvm for faster execution of the command operations and + // enabling easier debugging when running in debug mode. Forking a new process is not necessary + // when debugging as we can attach to the same JVM. + try (WithProperties with = new WithProperties().set(CassandraRelevantProperties.CASSANDRA_CLI_EXECUTION_PROTOCOL, + strategy.name().toLowerCase())) + { + return ToolRunner.invokeNodetoolInJvm(NodeTool::new, + strategy == CommandExecutionStraregy.Type.CQL ? + CQLTester::buildNodetoolCqlArgs : + CQLTester::buildNodetoolArgs, + args); + } + } +} diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 9f0369b79cf9..8e1ca3faf720 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -172,6 +172,7 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileSystems; import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.management.CommandService; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.ClientMetrics; import org.apache.cassandra.net.MessagingService; @@ -484,6 +485,7 @@ protected static void prePrepareServer() StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance); SnapshotManager.instance.registerMBean(); SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); + CommandService.instance.start(); } // So derived classes can get enough intialization to start setting DatabaseDescriptor options @@ -518,6 +520,8 @@ public static void tearDownClass() { logger.warn("Error shutting down jmx", e); } + + CommandService.instance.stop(); } } @@ -600,6 +604,19 @@ public static List buildNodetoolArgs(List args) return allArgs; } + public static List buildNodetoolCqlArgs(List args) + { + List allArgs = new ArrayList<>(); + allArgs.add("bin/nodetool"); + allArgs.add("-p"); + allArgs.add(Integer.toString(nativePort)); + allArgs.add("-h"); + allArgs.add(nativeAddr.getHostAddress()); + allArgs.addAll(args); + System.out.println(">>>> " + allArgs); + return allArgs; + } + public static List buildCqlshArgs(List args) { List allArgs = new ArrayList<>(); diff --git a/test/unit/org/apache/cassandra/management/CommandServiceTest.java b/test/unit/org/apache/cassandra/management/CommandServiceTest.java new file mode 100644 index 000000000000..71e8f50b55c3 --- /dev/null +++ b/test/unit/org/apache/cassandra/management/CommandServiceTest.java @@ -0,0 +1,107 @@ +/* + * 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.cassandra.management; + +import java.util.Set; + +import javax.management.MBeanServer; +import javax.management.ObjectName; + +import org.junit.Test; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.utils.MBeanWrapper; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertNotNull; + +public class CommandServiceTest extends CQLTester +{ + private static final String COMMAND_MBEAN_PATTERN = "org.apache.cassandra.management:type=Command,name=*"; + + @Test + public void testCommandServiceMBeanRegistered() + { + try + { + ObjectName serviceName = new ObjectName(CommandServiceMBean.MBEAN_NAME); + assertThat(MBeanWrapper.instance.isRegistered(serviceName)).as("CommandService MBean should be registered after start()").isTrue(); + } + catch (Exception e) + { + throw new AssertionError("Failed to check MBean registration", e); + } + } + + @Test + public void testCommandMBeansRegistered() + { + CommandService service = CommandService.instance; + try + { + ObjectName pattern = new ObjectName(COMMAND_MBEAN_PATTERN); + Set commandMBeans = MBeanWrapper.instance.queryNames(pattern, null); + + assertThat(commandMBeans).as("At least one Command MBean should be registered").isNotEmpty(); + + String[] commandNames = service.getCommandNames(); + assertThat(commandNames).as("Service should return command names").isNotEmpty(); + + for (String commandName : commandNames) + { + String mbeanName = service.getCommandMBeanName(commandName); + assertNotNull("MBean name should not be null for command: " + commandName, mbeanName); + + ObjectName objectName = new ObjectName(mbeanName); + assertThat(MBeanWrapper.instance.isRegistered(objectName)).as("Command MBean should be registered for: " + commandName).isTrue(); + } + } + catch (Exception e) + { + throw new AssertionError("Failed to verify Command MBeans", e); + } + } + + @Test + public void testCommandMBeanInvoke() + { + CommandService service = CommandService.instance; + + try + { + for (String testCommand : service.getCommandNames()) + { + String mbeanName = service.getCommandMBeanName(testCommand); + ObjectName commandObjectName = new ObjectName(mbeanName); + assertThat(MBeanWrapper.instance.isRegistered(commandObjectName)).as("Command MBean should be registered").isTrue(); + + MBeanServer mbs = MBeanWrapper.instance.getMBeanServer(); + String schema = (String) mbs.invoke(commandObjectName, "getJsonSchema", null, null); + System.out.println(">>> JSON Schema for command " + testCommand + ":\n" + schema); + assertThat(schema).as("getJsonSchema() should return non-null JSON string").isNotNull().isNotEmpty(); + + assertThat(schema.trim()).as("Schema should start with '{'").startsWith("{"); + } + } + catch (Exception e) + { + throw new AssertionError("Failed to test CommandMBean invoke", e); + } + } +} diff --git a/test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java b/test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java new file mode 100644 index 000000000000..575c7fad70ec --- /dev/null +++ b/test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java @@ -0,0 +1,77 @@ +/* + * 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.cassandra.management; + +import org.junit.Test; + +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.exceptions.InvalidRequestException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +public class CqlExecuteCommandTest extends CQLTester +{ + @Test + public void testExecuteCommandForcecompact() throws Throwable + { + String keyspaceName = createKeyspace("CREATE KEYSPACE %s WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }"); + String tableName = createTable(keyspaceName, "CREATE TABLE %s (k text PRIMARY KEY, v int)"); + + execute(String.format("INSERT INTO %s.%s (k, v) VALUES (?, ?)", keyspaceName, tableName), "k1", 1); + execute(String.format("INSERT INTO %s.%s (k, v) VALUES (?, ?)", keyspaceName, tableName), "k2", 2); + execute(String.format("INSERT INTO %s.%s (k, v) VALUES (?, ?)", keyspaceName, tableName), "k4", 4); + execute(String.format("INSERT INTO %s.%s (k, v) VALUES (?, ?)", keyspaceName, tableName), "k7", 7); + + flush(keyspaceName, tableName); + + // Execute the COMMAND statement using CQL-style syntax + String command = String.format("COMMAND forcecompact WITH \"keyspace\" = '%s' AND \"table\" = '%s' AND keys = ['k4', 'k2', 'k7'];", + keyspaceName, tableName); + + UntypedResultSet result = execute(command); + + // Verify that we got a result + assertNotNull("Result should not be null", result); + + // Verify that the result has one row with one column (the output) + assertEquals("Result should have one row", 1, result.size()); + + UntypedResultSet.Row row = result.one(); + assertNotNull("Row should not be null", row); + + // Verify that the output column exists and has a value + assertTrue("Result should have 'output' column", row.has("output")); + String output = row.getString("output"); + assertNotNull("Output should not be null", output); + + // The output should be non-empty (command execution output) + assertTrue("Output should not be empty", !output.trim().isEmpty()); + } + + @Test + public void testExecuteCommandWithInvalidCommand() throws Throwable + { + // Test that an invalid command name throws an exception + String command = "COMMAND nonexistentcommand WITH key = 'value';"; + + assertInvalidThrow(InvalidRequestException.class, command); + } +} diff --git a/test/unit/org/apache/cassandra/tools/NodeProbeTest.java b/test/unit/org/apache/cassandra/tools/NodeProbeTest.java index 1d41095ab8b3..d3c5b300134c 100644 --- a/test/unit/org/apache/cassandra/tools/NodeProbeTest.java +++ b/test/unit/org/apache/cassandra/tools/NodeProbeTest.java @@ -18,8 +18,6 @@ package org.apache.cassandra.tools; -import java.io.IOException; - import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -39,11 +37,11 @@ public static void setup() throws Exception { requireNetwork(); startJMXServer(); - probe = new NodeProbe(jmxHost, jmxPort); + probe = new NodeProbe(new RemoteJmxMBeanAccessor(jmxHost, jmxPort)); } @AfterClass - public static void teardown() throws IOException + public static void teardown() throws Exception { probe.close(); } diff --git a/test/unit/org/apache/cassandra/tools/ToolRunner.java b/test/unit/org/apache/cassandra/tools/ToolRunner.java index 0495a5afdcaf..8d42190fb47c 100644 --- a/test/unit/org/apache/cassandra/tools/ToolRunner.java +++ b/test/unit/org/apache/cassandra/tools/ToolRunner.java @@ -211,6 +211,11 @@ public static ToolResult invokeNodetool(Map env, List ar return invoke(env, CQLTester.buildNodetoolArgs(args)); } + public static ToolResult invokeCqlNodetool(Map env, List args) + { + return invoke(env, CQLTester.buildNodetoolCqlArgs(args)); + } + public static ToolRunner.ToolResult invokeNodetoolInJvm(String... args) { return ToolRunner.invokeNodetoolInJvm(NodeTool::new, args); @@ -356,6 +361,13 @@ public NodeToolResult get() } public static ToolRunner.ToolResult invokeNodetoolInJvm(BiFunction nodeTool, String... args) + { + return invokeNodetoolInJvm(nodeTool, CQLTester::buildNodetoolArgs, args); + } + + public static ToolRunner.ToolResult invokeNodetoolInJvm(BiFunction nodeTool, + Function, List> argsBuilder, + String... args) { PrintStream originalSysOut = System.out; PrintStream originalSysErr = System.err; @@ -364,7 +376,7 @@ public static ToolRunner.ToolResult invokeNodetoolInJvm(BiFunction clearedArgs = CQLTester.buildNodetoolArgs(isEmpty(args) ? new ArrayList<>() : List.of(args)); + List clearedArgs = argsBuilder.apply(isEmpty(args) ? new ArrayList<>() : List.of(args)); clearedArgs.remove("bin/nodetool"); try { diff --git a/test/unit/org/apache/cassandra/tools/nodetool/ClearSnapshotTest.java b/test/unit/org/apache/cassandra/tools/nodetool/ClearSnapshotTest.java index eb5e54310955..d54f3e9e3fd3 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/ClearSnapshotTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/ClearSnapshotTest.java @@ -18,7 +18,6 @@ package org.apache.cassandra.tools.nodetool; -import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -42,6 +41,7 @@ import org.apache.cassandra.service.snapshot.SnapshotManager; import org.apache.cassandra.service.snapshot.SnapshotManifest; import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.RemoteJmxMBeanAccessor; import org.apache.cassandra.tools.ToolRunner.ToolResult; import static java.lang.String.format; @@ -65,7 +65,7 @@ public static void setup() throws Exception { startJMXServer(); requireNetwork(); - probe = new NodeProbe(jmxHost, jmxPort); + probe = new NodeProbe(new RemoteJmxMBeanAccessor(jmxHost, jmxPort)); } @Before @@ -75,7 +75,7 @@ public void clearAllSnapshots() } @AfterClass - public static void teardown() throws IOException + public static void teardown() throws Exception { probe.close(); } diff --git a/test/unit/org/apache/cassandra/tools/nodetool/CompactTest.java b/test/unit/org/apache/cassandra/tools/nodetool/CompactTest.java index 1763147547ff..84e22c3ea516 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/CompactTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/CompactTest.java @@ -27,7 +27,7 @@ import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; -import static org.apache.cassandra.tools.ToolRunner.invokeNodetool; +import static org.apache.cassandra.tools.ToolRunner.invokeNodetoolInJvm; public class CompactTest extends CQLTester { @@ -52,7 +52,7 @@ public void keyPresent() throws Throwable flush(keyspace()); } Assertions.assertThat(cfs.getTracker().getView().liveSSTables()).hasSize(10); - invokeNodetool("compact", "--partition", Long.toString(key), keyspace(), currentTable()).assertOnCleanExit(); + invokeNodetoolInJvm("compact", "--partition", Long.toString(key), keyspace(), currentTable()).assertOnCleanExit(); // only 1 SSTable should exist Assertions.assertThat(cfs.getTracker().getView().liveSSTables()).hasSize(1); @@ -75,7 +75,7 @@ public void keyNotPresent() throws Throwable for (long keyNotFound : Arrays.asList(key - 1, key + 1)) { - invokeNodetool("compact", "--partition", Long.toString(keyNotFound), keyspace(), currentTable()).assertOnCleanExit(); + invokeNodetoolInJvm("compact", "--partition", Long.toString(keyNotFound), keyspace(), currentTable()).assertOnCleanExit(); // only 1 SSTable should exist Assertions.assertThat(cfs.getTracker().getView().liveSSTables()).hasSize(10); @@ -85,7 +85,7 @@ public void keyNotPresent() throws Throwable @Test public void tableNotFound() { - invokeNodetool("compact", "--partition", Long.toString(42), keyspace(), "doesnotexist") + invokeNodetoolInJvm("compact", "--partition", Long.toString(42), keyspace(), "doesnotexist") .asserts() .failure() .errorContains(String.format("java.lang.IllegalArgumentException: Unknown keyspace/cf pair (%s.doesnotexist)", keyspace())); @@ -96,7 +96,7 @@ public void keyWrongType() { createTable("CREATE TABLE %s (id bigint, value text, PRIMARY KEY ((id)))"); - invokeNodetool("compact", "--partition", "this_will_not_work", keyspace(), currentTable()) + invokeNodetoolInJvm("compact", "--partition", "this_will_not_work", keyspace(), currentTable()) .asserts() .failure() .errorContains(String.format("Unable to parse partition key 'this_will_not_work' for table %s.%s; Unable to make long from 'this_will_not_work'", keyspace(), currentTable())); diff --git a/test/unit/org/apache/cassandra/tools/nodetool/ForceCompactionTest.java b/test/unit/org/apache/cassandra/tools/nodetool/ForceCompactionTest.java index a78b80584ad5..3bfbbbd349da 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/ForceCompactionTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/ForceCompactionTest.java @@ -29,7 +29,8 @@ import org.junit.Test; import org.apache.cassandra.Util; -import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLNodetoolProtocolTester; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.db.rows.Cell; @@ -38,13 +39,12 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator; import org.apache.cassandra.io.sstable.ISSTableScanner; import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.tools.ToolRunner; import static org.apache.commons.lang3.ArrayUtils.addAll; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -public class ForceCompactionTest extends CQLTester +public class ForceCompactionTest extends CQLNodetoolProtocolTester { private final static int NUM_PARTITIONS = 10; private final static int NUM_ROWS = 100; @@ -54,6 +54,11 @@ public static void setupClass() throws Exception { requireNetwork(); startJMXServer(); + + // This ensures initialization happens before any transport classes (like Envelope.Decoder) + // are loaded, as they have static initializers that depend on DatabaseDescriptor. + if (!DatabaseDescriptor.isClientOrToolInitialized()) + DatabaseDescriptor.clientInitialization(false); } @Before @@ -251,7 +256,7 @@ private void forceCompact(String[] partitionKeysIgnoreGcGrace) if (cfs != null) { cfs.forceMajorCompaction(); - ToolRunner.invokeNodetool(addAll(new String[]{ "forcecompact", cfs.keyspace.getName(), cfs.getTableName() }, + invokeNodetool(addAll(new String[]{ "forcecompact", cfs.keyspace.getName(), cfs.getTableName() }, partitionKeysIgnoreGcGrace)).assertOnCleanExit(); } } diff --git a/tools/stress/src/org/apache/cassandra/stress/util/JmxCollector.java b/tools/stress/src/org/apache/cassandra/stress/util/JmxCollector.java index 24cb4c788b85..bb440550a743 100644 --- a/tools/stress/src/org/apache/cassandra/stress/util/JmxCollector.java +++ b/tools/stress/src/org/apache/cassandra/stress/util/JmxCollector.java @@ -18,7 +18,6 @@ */ package org.apache.cassandra.stress.util; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -30,6 +29,7 @@ import org.apache.cassandra.concurrent.NamedThreadFactory; import org.apache.cassandra.stress.settings.SettingsJMX; import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.RemoteJmxMBeanAccessor; public class JmxCollector implements Callable { @@ -91,17 +91,10 @@ public JmxCollector(Collection hosts, int port, SettingsJMX jmx) private static NodeProbe connect(String host, int port, SettingsJMX jmx) { - try - { - if (jmx.user != null && jmx.password != null) - return new NodeProbe(host, port, jmx.user, jmx.password); - else - return new NodeProbe(host, port); - } - catch (IOException e) - { - throw new RuntimeException(e); - } + if (jmx.user != null && jmx.password != null) + return new NodeProbe(new RemoteJmxMBeanAccessor(host, port, jmx.user, jmx.password)); + else + return new NodeProbe(new RemoteJmxMBeanAccessor(host, port)); } public GcStats call() throws Exception From fe7846ec8ce1833cc5ac98e65f7fb8fe9b503ae3 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 13 Jan 2026 14:04:24 +0100 Subject: [PATCH 03/14] Use CqlBuilder to generate a query string to run commands --- .../management/CommandExecutionArgsSerde.java | 10 +- .../cassandra/management/ManagementUtils.java | 10 + .../CommandMBeanExecutionStrategy.java | 6 +- .../strategy/CqlCommandExecutionStrategy.java | 178 ++++++++++-------- .../cql3/CQLNodetoolProtocolTester.java | 4 +- 5 files changed, 117 insertions(+), 91 deletions(-) diff --git a/src/java/org/apache/cassandra/management/CommandExecutionArgsSerde.java b/src/java/org/apache/cassandra/management/CommandExecutionArgsSerde.java index bfb54a42fc9c..35df212b14af 100644 --- a/src/java/org/apache/cassandra/management/CommandExecutionArgsSerde.java +++ b/src/java/org/apache/cassandra/management/CommandExecutionArgsSerde.java @@ -34,6 +34,7 @@ import org.apache.cassandra.management.picocli.TypeConverterRegistry; import org.apache.cassandra.utils.JsonUtils; +import static org.apache.cassandra.management.ManagementUtils.normalizeOptionName; import static org.apache.cassandra.management.api.ParameterMetadata.COMMAND_POSITIONAL_PARAM_PREFIX; public class CommandExecutionArgsSerde @@ -248,13 +249,4 @@ private static OptionMetadata findOptionByNameIgnoreCase(CommandMetadata metadat return null; } - - private static String normalizeOptionName(String name) - { - if (name.startsWith("--")) - return name.substring(2); - else if (name.startsWith("-")) - return name.substring(1); - return name; - } } diff --git a/src/java/org/apache/cassandra/management/ManagementUtils.java b/src/java/org/apache/cassandra/management/ManagementUtils.java index 45d347a1a8e0..36aa4e741237 100644 --- a/src/java/org/apache/cassandra/management/ManagementUtils.java +++ b/src/java/org/apache/cassandra/management/ManagementUtils.java @@ -63,4 +63,14 @@ public static String stripAngleBrackets(String name) return name; } + + /** Normalize the option name by stripping leading dashes. */ + public static String normalizeOptionName(String name) + { + if (name.startsWith("--")) + return name.substring(2); + else if (name.startsWith("-")) + return name.substring(1); + return name; + } } diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/CommandMBeanExecutionStrategy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/CommandMBeanExecutionStrategy.java index fa3a50a67706..e2490d8d581c 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/strategy/CommandMBeanExecutionStrategy.java +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/CommandMBeanExecutionStrategy.java @@ -107,10 +107,8 @@ public void close() throws ExecutionStrategyCloseException } } - /** - * Handles nested commands (e.g., "compressiondictionary.train"). - */ - private String extractCommandName(CommandLine.ParseResult parseResult) + /** Handles nested commands (e.g., "compressiondictionary.train"). */ + public static String extractCommandName(CommandLine.ParseResult parseResult) { List commandLineList = parseResult.asCommandLineList(); if (commandLineList.size() <= 1) // Only root "nodetool" diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java index 53094724cf5d..93010779d8c6 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java @@ -18,14 +18,20 @@ package org.apache.cassandra.tools.nodetool.strategy; +import java.lang.reflect.Array; import java.nio.ByteBuffer; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Set; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.management.CommandExecutionArgsSerde; import org.apache.cassandra.management.api.CommandExecutionArgs; +import org.apache.cassandra.management.api.OptionMetadata; +import org.apache.cassandra.management.api.ParameterMetadata; import org.apache.cassandra.management.picocli.PicocliCommandArgsConverter; import org.apache.cassandra.tools.nodetool.AbstractCommand; import org.apache.cassandra.tools.nodetool.CqlConnect; @@ -33,6 +39,10 @@ import picocli.CommandLine; +import static org.apache.cassandra.management.ManagementUtils.normalizeOptionName; +import static org.apache.cassandra.management.api.ParameterMetadata.COMMAND_POSITIONAL_PARAM_PREFIX; +import static org.apache.cassandra.tools.nodetool.strategy.CommandMBeanExecutionStrategy.extractCommandName; + public class CqlCommandExecutionStrategy implements CommandExecutionStraregy { private final CqlConnect connect; @@ -66,11 +76,10 @@ public int execute(CommandLine.ParseResult parseResult) throws CommandLine.Execu // Command is already populated with args from CommandLine.parseArgs(), convert to CommandExecutionArgs CommandExecutionArgs args = PicocliCommandArgsConverter.fromCommand(userObject); - String jsonParams = CommandExecutionArgsSerde.toJson(args); try { - String cqlCommand = buildCqlCommandString(commandName, jsonParams); + String cqlCommand = buildCqlCommandString(commandName, args); ResultMessage result = connect.client().execute(cqlCommand, ConsistencyLevel.ONE); if (result instanceof ResultMessage.Rows) @@ -113,118 +122,135 @@ public void close() throws ExecutionStrategyCloseException } /** - * Handles nested commands (e.g., "compressiondictionary.train"). + * Build a CQL COMMAND statement string from command name and CommandExecutionArgs. + * The format is: COMMAND commandName WITH "key1" = 'value1' AND "key2" = 'value2'; */ - private String extractCommandName(CommandLine.ParseResult parseResult) + private static String buildCqlCommandString(String commandName, CommandExecutionArgs args) { - List commandLineList = parseResult.asCommandLineList(); - if (commandLineList.size() <= 1) // Only root "nodetool" - return null; + CqlBuilder builder = new CqlBuilder(); + builder.append("COMMAND "); + builder.appendQuotingIfNeeded(commandName); - StringBuilder commandName = new StringBuilder(); - for (int i = 1; i < commandLineList.size(); i++) + Map paramsMap = new LinkedHashMap<>(); + for (Map.Entry entry : args.options().entrySet()) { - CommandLine cmdLine = commandLineList.get(i); - String name = cmdLine.getCommandName(); - if (name != null && !name.isEmpty()) - { - if (commandName.length() > 0) - commandName.append("."); - commandName.append(name); - } + OptionMetadata optionMetadata = entry.getKey(); + Object value = entry.getValue(); + + if (value == null) + continue; + + String key = normalizeOptionName(optionMetadata.paramLabel()); + paramsMap.put(key, value); } - return commandName.length() > 0 ? commandName.toString() : null; - } + for (Map.Entry entry : args.parameters().entrySet()) + { + ParameterMetadata paramMetadata = entry.getKey(); + Object value = entry.getValue(); - /** - * Build a CQL COMMAND statement string from command name and JSON parameters. - * The JSON is parsed and converted to the WITH clause format: - * COMMAND commandName WITH "key1" = 'value1' AND "key2" = 'value2'; - */ - private String buildCqlCommandString(String commandName, String jsonParams) - { - Map paramsMap = org.apache.cassandra.utils.JsonUtils.fromJsonMap(jsonParams); + if (value == null) + continue; - StringBuilder cql = new StringBuilder("COMMAND "); - cql.append(commandName); + String key = COMMAND_POSITIONAL_PARAM_PREFIX + paramMetadata.index(); + paramsMap.put(key, value); + } if (!paramsMap.isEmpty()) { - cql.append(" WITH"); + builder.append(" WITH"); boolean first = true; for (Map.Entry entry : paramsMap.entrySet()) { if (first) - cql.append(" "); + builder.append(" "); else - cql.append(" AND "); + builder.append(" AND "); - String key = entry.getKey(); - Object value = entry.getValue(); - - // Quote the key if it contains special characters - cql.append('"').append(escapeString(key)).append('"'); - cql.append(" = "); - cql.append(formatCqlValue(value)); + // Using ColumnIdentifier.maybeQuote as some of the keys + // may be reserved words in CQL (e.g., "keyspace", "table"). + builder.append(ColumnIdentifier.maybeQuote(entry.getKey())); + builder.append(" = "); + appendCqlValue(builder, entry.getValue()); first = false; } } - cql.append(";"); - return cql.toString(); + builder.append(";"); + return builder.toString(); } - private String formatCqlValue(Object value) + /** Append a value to CqlBuilder with proper type handling and escaping. */ + private static void appendCqlValue(CqlBuilder builder, Object value) { if (value == null) - return "NULL"; + { + builder.append("NULL"); + return; + } if (value instanceof String) + builder.appendWithSingleQuotes((String) value); + else if (value instanceof Number) + builder.append(value); + else if (value instanceof Boolean) + builder.append(value); + else if (value instanceof List) { - return "'" + escapeSingleQuotes((String) value) + "'"; + List list = (List) value; + builder.append("["); + for (int i = 0; i < list.size(); i++) + { + if (i > 0) + builder.append(", "); + appendCqlValue(builder, list.get(i)); + } + builder.append("]"); } - else if (value instanceof Number) + else if (value instanceof Map) { - return value.toString(); + Map map = (Map) value; + builder.append("{"); + boolean first = true; + for (Map.Entry entry : map.entrySet()) + { + if (!first) + builder.append(", "); + builder.appendWithSingleQuotes(entry.getKey()); + builder.append(": "); + appendCqlValue(builder, entry.getValue()); + first = false; + } + builder.append("}"); } - else if (value instanceof Boolean) + else if (value instanceof Set) { - return value.toString(); + Set set = (Set) value; + builder.append("{"); + boolean first = true; + for (Object item : set) + { + if (!first) + builder.append(", "); + appendCqlValue(builder, item); + first = false; + } + builder.append("}"); } - else if (value instanceof List) + else if (value.getClass().isArray()) { - @SuppressWarnings("unchecked") - List list = (List) value; - StringBuilder sb = new StringBuilder("["); - for (int i = 0; i < list.size(); i++) + int length = Array.getLength(value); + builder.append("["); + for (int i = 0; i < length; i++) { if (i > 0) - sb.append(", "); - Object item = list.get(i); - if (item instanceof String) - sb.append("'").append(escapeSingleQuotes((String) item)).append("'"); - else - sb.append(formatCqlValue(item)); + builder.append(", "); + appendCqlValue(builder, Array.get(value, i)); } - sb.append("]"); - return sb.toString(); + builder.append("]"); } else - { - // For other types, convert to string and quote - return "'" + escapeSingleQuotes(value.toString()) + "'"; - } - } - - private static String escapeSingleQuotes(String str) - { - return str.replace("'", "''"); - } - - private static String escapeString(String str) - { - return str.replace("\"", "\\\""); + builder.appendWithSingleQuotes(value.toString()); } } diff --git a/test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java b/test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java index 29ea5702be15..c8d695e6d6d0 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java @@ -49,8 +49,8 @@ public static Collection data() public ToolRunner.ToolResult invokeNodetool(String... args) { // Use invokeNodetoolInJvm for faster execution of the command operations and - // enabling easier debugging when running in debug mode. Forking a new process is not necessary - // when debugging as we can attach to the same JVM. + // enabling easier debugging when running in debug mode from IDE. There is also + // no need to build the jars to run nodetool commands in this test. try (WithProperties with = new WithProperties().set(CassandraRelevantProperties.CASSANDRA_CLI_EXECUTION_PROTOCOL, strategy.name().toLowerCase())) { From 2c863b1281d1d62a1c295b8db0ff628e70a7b9b3 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 13 Jan 2026 14:27:52 +0100 Subject: [PATCH 04/14] nodetool should respect command execution strategy --- .../org/apache/cassandra/tools/NodeTool.java | 16 ++++- .../ProtocolAwareExecutionStrategy.java | 68 ++++++------------- 2 files changed, 33 insertions(+), 51 deletions(-) diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java index 1ee4a6169846..c838fe92fb47 100644 --- a/src/java/org/apache/cassandra/tools/NodeTool.java +++ b/src/java/org/apache/cassandra/tools/NodeTool.java @@ -37,9 +37,11 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileWriter; +import org.apache.cassandra.tools.nodetool.CqlConnect; import org.apache.cassandra.tools.nodetool.JmxConnect; import org.apache.cassandra.tools.nodetool.NodetoolCommand; import org.apache.cassandra.tools.nodetool.layout.CassandraCliHelpLayout; +import org.apache.cassandra.tools.nodetool.strategy.CommandExecutionStraregy; import org.apache.cassandra.tools.nodetool.strategy.ProtocolAwareExecutionStrategy; import org.apache.cassandra.utils.FBUtilities; @@ -178,8 +180,18 @@ private static void getCommandsWithoutRoot(CommandLine cli, List command public static CommandLine createCommandLine(CommandLine.IFactory factory) throws Exception { - return new CommandLine(new NodetoolCommand(), factory) - .addMixin(ProtocolAwareExecutionStrategy.MIXIN_KEY, factory.create(JmxConnect.class)); + CommandLine commandLine = new CommandLine(new NodetoolCommand(), factory); + CommandExecutionStraregy.Type strategyType = ProtocolAwareExecutionStrategy.getExecutionStrategyTypeFromEnvAndSys(); + switch (strategyType) + { + case CQL: + return commandLine.addMixin(strategyType.toString(), factory.create(CqlConnect.class)); + case STATIC_MBEAN: + case COMMAND_MBEAN: + return commandLine.addMixin(strategyType.toString(), factory.create(JmxConnect.class)); + default: + throw new IllegalStateException("Unknown execution strategy: " + strategyType); + } } private static void configureCliLayout(CommandLine commandLine) diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java index de34119abe4b..a8f43b3d052b 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java @@ -18,12 +18,9 @@ package org.apache.cassandra.tools.nodetool.strategy; -import java.lang.reflect.Field; - import org.apache.cassandra.config.CassandraRelevantEnv; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.tools.nodetool.AbstractCommand; import org.apache.cassandra.tools.nodetool.CqlConnect; import org.apache.cassandra.tools.nodetool.JmxConnect; @@ -33,8 +30,6 @@ public class ProtocolAwareExecutionStrategy implements CommandExecutionStraregy { - public static final String MIXIN_KEY = "jmx"; - public static int executionStrategy(CommandLine.ParseResult parseResult) { try (ProtocolAwareExecutionStrategy strategy = new ProtocolAwareExecutionStrategy()) @@ -51,20 +46,22 @@ public static int executionStrategy(CommandLine.ParseResult parseResult) @Override public int execute(CommandLine.ParseResult parseResult) { - CommandLine.Model.CommandSpec jmx = parseResult.commandSpec().mixins().get(MIXIN_KEY); - if (jmx == null) - throw new CommandLine.InitializationException("No JmxConnect command found in the top-level hierarchy"); + CommandLine.Model.CommandSpec connectCmd = parseResult.commandSpec() + .mixins() + .get(getExecutionStrategyTypeFromEnvAndSys().toString()); + if (connectCmd == null) + throw new CommandLine.InitializationException("No 'connection' command found in the top-level hierarchy"); - try (CommandExecutionStraregy invoker = createInvoker((JmxConnect) jmx.userObject())) + try (CommandExecutionStraregy invoker = createInvoker(connectCmd.userObject())) { return invoker.execute(parseResult); } catch (ExecutionStrategyCloseException e) { - jmx.commandLine() + connectCmd.commandLine() .getErr() .println("Failed to connect: " + e.getMessage()); - return jmx.commandLine().getExitCodeExceptionMapper().getExitCode(e); + return connectCmd.commandLine().getExitCodeExceptionMapper().getExitCode(e); } catch (Exception e) { @@ -72,64 +69,37 @@ public int execute(CommandLine.ParseResult parseResult) } } - static CommandExecutionStraregy createInvoker(JmxConnect jmxConnect) + public static Type getExecutionStrategyTypeFromEnvAndSys() { Type defaultStrategy = Type.valueOf(toUpperCaseLocalized(CassandraRelevantProperties.CASSANDRA_CLI_EXECUTION_PROTOCOL.getDefaultValue())); Type strategyEnv = CassandraRelevantEnv.CASSANDRA_CLI_EXECUTION_PROTOCOL.getEnum(true, Type.class, defaultStrategy.name()); Type strategySys = CassandraRelevantProperties.CASSANDRA_CLI_EXECUTION_PROTOCOL.getEnum(true, Type.class); - Type strategy = strategyEnv != defaultStrategy ? strategyEnv : strategySys; + return strategyEnv != defaultStrategy ? strategyEnv : strategySys; + } + static CommandExecutionStraregy createInvoker(Object connectCmd) + { + Type strategy = getExecutionStrategyTypeFromEnvAndSys(); switch (strategy) { case CQL: - CqlConnect cqlConnect = createCqlConnectFromJmxConnect(jmxConnect); - return new CqlCommandExecutionStrategy(cqlConnect); + createCqlConnectFromJmxConnect(); + return new CqlCommandExecutionStrategy((CqlConnect) connectCmd); case COMMAND_MBEAN: - return new CommandMBeanExecutionStrategy(jmxConnect); + return new CommandMBeanExecutionStrategy((JmxConnect) connectCmd); case STATIC_MBEAN: - return new StaticMBeanExecutionStrategy(jmxConnect); + return new StaticMBeanExecutionStrategy((JmxConnect) connectCmd); default: throw new IllegalStateException("Unknown execution strategy: " + strategy); } } - private static CqlConnect createCqlConnectFromJmxConnect(JmxConnect jmxConnect) + private static void createCqlConnectFromJmxConnect() { // Initialize DatabaseDescriptor before creating CqlConnect to ensure it's ready // when transport classes (like Envelope.Decoder) are loaded. This is critical // for forked nodetool processes where DatabaseDescriptor might not be initialized. if (!DatabaseDescriptor.isClientOrToolInitialized()) DatabaseDescriptor.clientInitialization(false); - - try - { - // TODO This is a bit of a hack to transfer the output from JmxConnect to CqlConnect. - // Consider refactoring Output handling in nodetool commands to be more flexible. - Field outputField = AbstractCommand.class.getDeclaredField("output"); - outputField.setAccessible(true); - org.apache.cassandra.tools.Output output = (org.apache.cassandra.tools.Output) outputField.get(jmxConnect); - - CqlConnect cqlConnect = new CqlConnect(); - Field cqlHostField = CqlConnect.class.getDeclaredField("host"); - cqlHostField.setAccessible(true); - cqlHostField.set(cqlConnect, jmxConnect.getHost()); - - Field cqlPortField = CqlConnect.class.getDeclaredField("port"); - cqlPortField.setAccessible(true); - cqlPortField.set(cqlConnect, jmxConnect.getPort()); - - if (output != null) - { - Field cqlOutputField = CqlConnect.class.getSuperclass().getDeclaredField("output"); - cqlOutputField.setAccessible(true); - cqlOutputField.set(cqlConnect, output); - } - - return cqlConnect; - } - catch (Exception e) - { - throw new RuntimeException("Failed to create CqlConnect from JmxConnect", e); - } } } From e24412834d686a1950a5f8bf085a3bb43fdae6da Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 18 Jan 2026 00:47:52 +0100 Subject: [PATCH 05/14] Add support of a new management port to execute commands --- conf/cassandra.yaml | 44 +++++ conf/cassandra_latest.yaml | 44 +++++ .../config/CassandraRelevantProperties.java | 3 + .../org/apache/cassandra/config/Config.java | 13 ++ .../cassandra/config/DatabaseDescriptor.java | 60 +++++++ .../statements/ExecuteCommandStatement.java | 4 + .../cassandra/service/CassandraDaemon.java | 28 +++- .../apache/cassandra/service/ClientState.java | 16 +- .../NativeTransportManagementService.java | 157 ++++++++++++++++++ .../service/NativeTransportService.java | 39 ++--- .../NativeTransportSharedResourceManager.java | 113 +++++++++++++ .../cassandra/transport/Connection.java | 1 + .../transport/PipelineConfigurator.java | 9 +- .../apache/cassandra/transport/Server.java | 13 +- .../cassandra/transport/ServerConnection.java | 12 ++ .../transport/messages/StartupMessage.java | 6 + .../org/apache/cassandra/cql3/CQLTester.java | 28 +++- .../service/NativeTransportServiceTest.java | 10 +- 18 files changed, 567 insertions(+), 33 deletions(-) create mode 100644 src/java/org/apache/cassandra/service/NativeTransportManagementService.java create mode 100644 src/java/org/apache/cassandra/service/NativeTransportSharedResourceManager.java diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index 5736890b5144..c23e62987ffa 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1110,6 +1110,50 @@ rpc_address: localhost # enable or disable keepalive on rpc/native connections rpc_keepalive: true +# Whether to start the native management transport server. +# The address on which the native management transport is bound can be configured +# using rpc_management_address or rpc_management_interface. If neither is set, +# it defaults to rpc_address (or rpc_interface if that was used). +start_native_management_transport: false + +# The address or interface to bind the native management transport server to. +# This allows you to bind management operations to a different network interface +# than regular client connections, providing better security isolation. +# +# Set rpc_management_address OR rpc_management_interface, not both. +# +# If neither rpc_management_address nor rpc_management_interface is set, +# the management transport will use the same address as the regular native +# transport (rpc_address or rpc_interface). +# +# For security reasons, you should not expose this port to the internet. +# Firewall it if needed. Consider binding to a management network interface +# that is not accessible from public networks. +# rpc_management_address: localhost + +# Set rpc_management_address OR rpc_management_interface, not both. Interfaces +# must correspond to a single address, IP aliasing is not supported. +# +# This allows you to bind the management transport to a specific network +# interface by name (e.g., eth1, eth0). The IP address will be automatically +# resolved from the interface. This is useful when the interface IP may change +# (e.g., DHCP) or when you want to bind to a specific interface without +# hardcoding the IP address. +# rpc_management_interface: eth1 + +# If you choose to specify the management interface by name and the interface +# has both an ipv4 and an ipv6 address, you can specify which should be chosen +# using rpc_management_interface_prefer_ipv6. If false the first ipv4 address +# will be used. If true the first ipv6 address will be used. Defaults to false +# preferring ipv4. If there is only one address it will be selected regardless +# of ipv4/ipv6. +# rpc_management_interface_prefer_ipv6: false + +# Port for the management CQL native transport to listen for clients on. +# For security reasons, you should not expose this port to the internet. +# Firewall it if needed. +native_management_transport_port: 11211 + # Uncomment to set socket buffer size for internode communication # Note that when setting this, the buffer size is limited by net.core.wmem_max # and when not setting it it is defined by net.ipv4.tcp_wmem diff --git a/conf/cassandra_latest.yaml b/conf/cassandra_latest.yaml index b43f6cf45cb8..a89d7fb2a99a 100644 --- a/conf/cassandra_latest.yaml +++ b/conf/cassandra_latest.yaml @@ -1099,6 +1099,50 @@ rpc_address: localhost # enable or disable keepalive on rpc/native connections rpc_keepalive: true +# Whether to start the native management transport server. +# The address on which the native management transport is bound can be configured +# using rpc_management_address or rpc_management_interface. If neither is set, +# it defaults to rpc_address (or rpc_interface if that was used). +start_native_management_transport: false + +# The address or interface to bind the native management transport server to. +# This allows you to bind management operations to a different network interface +# than regular client connections, providing better security isolation. +# +# Set rpc_management_address OR rpc_management_interface, not both. +# +# If neither rpc_management_address nor rpc_management_interface is set, +# the management transport will use the same address as the regular native +# transport (rpc_address or rpc_interface). +# +# For security reasons, you should not expose this port to the internet. +# Firewall it if needed. Consider binding to a management network interface +# that is not accessible from public networks. +# rpc_management_address: localhost + +# Set rpc_management_address OR rpc_management_interface, not both. Interfaces +# must correspond to a single address, IP aliasing is not supported. +# +# This allows you to bind the management transport to a specific network +# interface by name (e.g., eth1, eth0). The IP address will be automatically +# resolved from the interface. This is useful when the interface IP may change +# (e.g., DHCP) or when you want to bind to a specific interface without +# hardcoding the IP address. +# rpc_management_interface: eth1 + +# If you choose to specify the management interface by name and the interface +# has both an ipv4 and an ipv6 address, you can specify which should be chosen +# using rpc_management_interface_prefer_ipv6. If false the first ipv4 address +# will be used. If true the first ipv6 address will be used. Defaults to false +# preferring ipv4. If there is only one address it will be selected regardless +# of ipv4/ipv6. +# rpc_management_interface_prefer_ipv6: false + +# Port for the management CQL native transport to listen for clients on. +# For security reasons, you should not expose this port to the internet. +# Firewall it if needed. +native_management_transport_port: 11211 + # Uncomment to set socket buffer size for internode communication # Note that when setting this, the buffer size is limited by net.core.wmem_max # and when not setting it it is defined by net.ipv4.tcp_wmem diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index 790c44f2c607..ae63668836d3 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -416,6 +416,8 @@ public enum CassandraRelevantProperties MX4JPORT("mx4jport"), NANOTIMETOMILLIS_TIMESTAMP_UPDATE_INTERVAL("cassandra.NANOTIMETOMILLIS_TIMESTAMP_UPDATE_INTERVAL", "10000"), NATIVE_EPOLL_ENABLED("cassandra.native.epoll.enabled", "true"), + /** This is the port used with RPC address for the management native protocol to communicate with clients that manage the node. */ + NATIVE_TRANSPORT_MANAGEMENT_PORT("cassandra.native_transport_management_port"), /** This is the port used with RPC address for the native protocol to communicate with clients. Now that thrift RPC is no longer in use there is no RPC port. */ NATIVE_TRANSPORT_PORT("cassandra.native_transport_port"), NEVER_PURGE_TOMBSTONES("cassandra.never_purge_tombstones"), @@ -563,6 +565,7 @@ public enum CassandraRelevantProperties SSL_STORAGE_PORT("cassandra.ssl_storage_port"), SSTABLE_FORMAT_DEFAULT("cassandra.sstable.format.default"), START_GOSSIP("cassandra.start_gossip", "true"), + START_NATIVE_MANAGEMENT_TRANSPORT("cassandra.start_native_management_transport", "false"), START_NATIVE_TRANSPORT("cassandra.start_native_transport"), STORAGE_DIR("cassandra.storagedir"), STORAGE_HOOK("cassandra.storage_hook"), diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index 7489b2992664..b3424e427963 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -327,6 +327,19 @@ public MemtableOptions() @Replaces(oldName = "native_transport_receive_queue_capacity_in_bytes", converter = Converters.BYTES_DATASTORAGE, deprecated = true) public DataStorageSpec.IntBytesBound native_transport_receive_queue_capacity = new DataStorageSpec.IntBytesBound("1MiB"); + /** + * Management RPC address and interface refer to the address/interface used for the native management protocol + * to communicate with management clients. If not explicitly configured, these default to the regular RPC address + * configuration (rpc_address or rpc_interface). + *

+ * native_transport_management_port is the port paired with the management RPC address to bind on. + */ + public String rpc_management_address; + public String rpc_management_interface; + public boolean rpc_management_interface_prefer_ipv6 = false; + public boolean start_native_transport_management = false; + public int native_transport_management_port = 11211; + /** * Max size of values in SSTables, in MebiBytes. * Default is the same as the native protocol frame limit: 256MiB. diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index 8565f72be8bb..bd718e433326 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -141,6 +141,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.DISABLE_STCS_IN_L0; import static org.apache.cassandra.config.CassandraRelevantProperties.INITIAL_TOKEN; import static org.apache.cassandra.config.CassandraRelevantProperties.IO_NETTY_TRANSPORT_ESTIMATE_SIZE_ON_SUBMIT; +import static org.apache.cassandra.config.CassandraRelevantProperties.NATIVE_TRANSPORT_MANAGEMENT_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.NATIVE_TRANSPORT_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.OS_ARCH; import static org.apache.cassandra.config.CassandraRelevantProperties.PARTITIONER; @@ -208,6 +209,7 @@ public class DatabaseDescriptor private static InetAddress broadcastAddress; private static InetAddress rpcAddress; private static InetAddress broadcastRpcAddress; + private static InetAddress rpcManagementAddress; private static SeedProvider seedProvider; private static IInternodeAuthenticator internodeAuthenticator = new AllowAllInternodeAuthenticator(); @@ -1389,6 +1391,7 @@ public static void applyAddressConfig(Config config) throws ConfigurationExcepti rpcAddress = null; broadcastAddress = null; broadcastRpcAddress = null; + rpcManagementAddress = null; /* Local IP, hostname or interface to bind services to */ if (config.listen_address != null && config.listen_interface != null) @@ -1476,6 +1479,32 @@ else if (config.rpc_interface != null) throw new ConfigurationException("If rpc_address is set to a wildcard address (" + config.rpc_address + "), then " + "you must set broadcast_rpc_address to a value other than " + config.rpc_address, false); } + + /* Local IP, hostname or interface to bind Management RPC server to */ + if (config.rpc_management_address != null && config.rpc_management_interface != null) + { + throw new ConfigurationException("Set rpc_management_address OR rpc_management_interface, not both", false); + } + else if (config.rpc_management_address != null) + { + try + { + rpcManagementAddress = InetAddress.getByName(config.rpc_management_address); + } + catch (UnknownHostException e) + { + throw new ConfigurationException("Unknown host in rpc_management_address " + config.rpc_management_address, false); + } + } + else if (config.rpc_management_interface != null) + { + rpcManagementAddress = getNetworkInterfaceAddress(config.rpc_management_interface, "rpc_management_interface", config.rpc_management_interface_prefer_ipv6); + } + else + { + // Default to regular rpc_address if not specified + rpcManagementAddress = rpcAddress; + } } public static void applyEncryptionContext() @@ -3388,6 +3417,18 @@ public static InetAddress getRpcAddress() return rpcAddress; } + /** + * This is the address used to bind for the native management protocol to communicate with management clients. + * If not explicitly configured via rpc_management_address or rpc_management_interface, defaults to the regular + * rpc_address. The address alone is not enough to uniquely identify this instance because multiple instances + * might use the same interface with different ports. + */ + public static InetAddress getRpcManagementAddress() + { + assert rpcManagementAddress != null; + return rpcManagementAddress; + } + public static void setBroadcastRpcAddress(InetAddress broadcastRPCAddr) { broadcastRpcAddress = broadcastRPCAddr; @@ -3632,6 +3673,25 @@ private static long calculateDefaultNativeTransportMaxMessageSizeInBytes() ); } + public static boolean startNativeTransportManagement() + { + return conf.start_native_transport_management; + } + + /** + * This is the port used for the native management protocol to communicate with clients. + */ + public static int getNativeTransportManagementPort() + { + return NATIVE_TRANSPORT_MANAGEMENT_PORT.getInt(conf.native_transport_management_port); + } + + @VisibleForTesting + public static void setNativeTransportPortManagement(int port) + { + conf.native_transport_management_port = port; + } + public static Config.PaxosVariant getPaxosVariant() { return conf.paxos_variant; diff --git a/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java b/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java index 28c009ca0b83..32f7228f4506 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java @@ -82,6 +82,10 @@ public ResultMessage execute(QueryState state, QueryOptions options, Dispatcher. { try { + ClientState clientState = state.getClientState(); + if (!clientState.isInternal && !clientState.isManagement()) + throw new InvalidRequestException("Command execution is only allowed via native management interface"); + Command command = CommandService.instance.getRegistry().command(commandName); if (command == null) throw new InvalidRequestException("Command not found: " + commandName); diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index 7450315dd45f..6053d360d9f3 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -118,6 +118,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VERSION; import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VM_NAME; import static org.apache.cassandra.config.CassandraRelevantProperties.SIZE_RECORDER_INTERVAL; +import static org.apache.cassandra.config.CassandraRelevantProperties.START_NATIVE_MANAGEMENT_TRANSPORT; import static org.apache.cassandra.config.CassandraRelevantProperties.START_NATIVE_TRANSPORT; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.createMetricsKeyspaceTables; import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_METRICS; @@ -213,6 +214,7 @@ private void maybeInitJmx() static final CassandraDaemon instance = new CassandraDaemon(); + private volatile NativeTransportManagementService nativeTransportManagementService; private volatile NativeTransportService nativeTransportService; private JMXConnectorServer jmxServer; @@ -599,6 +601,9 @@ public void setupVirtualKeyspaces() public synchronized void initializeClientTransports() { + if (nativeTransportManagementService == null) + nativeTransportManagementService = new NativeTransportManagementService(); + // Native transport if (nativeTransportService == null) nativeTransportService = new NativeTransportService(); @@ -686,6 +691,9 @@ public void start() Set peers = new HashSet<>(ClusterMetadata.current().directory.allJoinedEndpoints()); connectivityChecker.execute(peers, ep -> locator.location(ep).datacenter); + // start management transports first. + startManagementTransport(); + // check to see if transports may start else return without starting. This is needed when in survey mode or // when bootstrap has not completed. try @@ -714,6 +722,16 @@ private void startClientTransports() logger.info("Not starting native transport as requested. Use JMX (StorageService->startNativeTransport()) or nodetool (enablebinary) to start it"); } + private void startManagementTransport() + { + if (START_NATIVE_MANAGEMENT_TRANSPORT.getBoolean() || DatabaseDescriptor.startNativeTransportManagement()) + { + if (nativeTransportManagementService == null) + throw new IllegalStateException("setup() must be called first for CassandraDaemon"); + nativeTransportManagementService.start(); + } + } + /** * Stop the daemon, ideally in an idempotent manner. * @@ -927,12 +945,18 @@ public static void main(String[] args) public void clearConnectionHistory() { - nativeTransportService.clearConnectionHistory(); + if (nativeTransportService != null) + nativeTransportService.clearConnectionHistory(); + if (nativeTransportManagementService != null) + nativeTransportManagementService.clearConnectionHistory(); } public void disconnectUser(Predicate userPredicate) { - nativeTransportService.disconnect(userPredicate); + if (nativeTransportService != null) + nativeTransportService.disconnect(userPredicate); + if (nativeTransportManagementService != null) + nativeTransportManagementService.disconnect(userPredicate); } private void exitOrFail(int code, String message) diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java index 7ec2160e8dc2..d58bcd34b3e5 100644 --- a/src/java/org/apache/cassandra/service/ClientState.java +++ b/src/java/org/apache/cassandra/service/ClientState.java @@ -147,6 +147,9 @@ public class ClientState // Driver String for the client private volatile String driverName; private volatile String driverVersion; + + // Whether this client is connected via the management native transport port + private volatile boolean isManagement; // Options provided by the client private volatile Map clientOptions; @@ -212,6 +215,7 @@ protected ClientState(ClientState source) this.driverName = source.driverName; this.driverVersion = source.driverVersion; this.clientOptions = source.clientOptions; + this.isManagement = source.isManagement; } /** @@ -356,7 +360,17 @@ public void setDriverVersion(String driverVersion) { this.driverVersion = driverVersion; } - + + public boolean isManagement() + { + return isManagement; + } + + public void setManagement(boolean isManagement) + { + this.isManagement = isManagement; + } + public void setClientOptions(Map clientOptions) { this.clientOptions = ImmutableMap.copyOf(clientOptions); diff --git a/src/java/org/apache/cassandra/service/NativeTransportManagementService.java b/src/java/org/apache/cassandra/service/NativeTransportManagementService.java new file mode 100644 index 000000000000..61092b9151c6 --- /dev/null +++ b/src/java/org/apache/cassandra/service/NativeTransportManagementService.java @@ -0,0 +1,157 @@ +/* + * 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.cassandra.service; + +import java.net.InetAddress; +import java.util.function.Predicate; + +import com.google.common.annotations.VisibleForTesting; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.auth.AuthenticatedUser; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.config.EncryptionOptions; +import org.apache.cassandra.transport.Server; + +import io.netty.channel.EventLoopGroup; +import io.netty.util.Version; + +/** + * Manages the native management transport server on port {@code 11211}. + * This service is independent of NativeTransportService and can only be + * enabled/disabled via configuration (not at runtime). + */ +public class NativeTransportManagementService +{ + private static final Logger logger = LoggerFactory.getLogger(NativeTransportManagementService.class); + + private Server server = null; + private boolean initialized = false; + private final NativeTransportSharedResourceManager resourceManager; + + public NativeTransportManagementService() + { + this(NativeTransportSharedResourceManager.instance()); + } + + @VisibleForTesting + NativeTransportManagementService(NativeTransportSharedResourceManager resourceManager) + { + this.resourceManager = resourceManager; + } + + @VisibleForTesting + synchronized void initialize() + { + if (initialized) + return; + + if (!DatabaseDescriptor.startNativeTransportManagement()) + { + logger.debug("Management transport is disabled via configuration"); + return; + } + + EventLoopGroup workerGroup = resourceManager.getOrCreateSharedWorkerGroup(); + int managementPort = DatabaseDescriptor.getNativeTransportManagementPort(); + InetAddress addr = DatabaseDescriptor.getRpcManagementAddress(); + + EncryptionOptions.TlsEncryptionPolicy encryptionPolicy = DatabaseDescriptor.getNativeProtocolEncryptionOptions() + .tlsEncryptionPolicy(); + + server = new Server.Builder() + .withEventLoopGroup(workerGroup) + .withHost(addr) + .withTlsEncryptionPolicy(encryptionPolicy) + .withPort(managementPort) + .withManagementConnectionFlag(true) + .build(); + + resourceManager.registerActiveService(); + initialized = true; + } + + public void start() + { + if (!DatabaseDescriptor.startNativeTransportManagement()) + { + logger.info("Management transport is disabled via configuration"); + return; + } + + logger.info("Using Netty Version: {}", Version.identify().entrySet()); + initialize(); + + if (server != null) + { + server.start(); + logger.info("Management transport started on port {}", DatabaseDescriptor.getNativeTransportManagementPort()); + } + } + + public void stop() + { + stop(false); + } + + public void stop(boolean force) + { + if (server != null) + server.stop(force); + } + + public void destroy() + { + stop(); + if (server == null) + return; + + server = null; + resourceManager.unregisterActiveService(); + initialized = false; + } + + /** @return true if the management transport server is running. */ + public boolean isRunning() + { + return server != null && server.isRunning(); + } + + /** Clears connection history for this service's server. */ + public void clearConnectionHistory() + { + if (server != null) + server.clearConnectionHistory(); + } + + /** Disconnects users matching the predicate from this service's server. */ + public void disconnect(Predicate userPredicate) + { + if (server != null) + server.disconnect(userPredicate); + } + + @VisibleForTesting + Server getServer() + { + return server; + } +} diff --git a/src/java/org/apache/cassandra/service/NativeTransportService.java b/src/java/org/apache/cassandra/service/NativeTransportService.java index 28817df396fc..92d2dc4db40f 100644 --- a/src/java/org/apache/cassandra/service/NativeTransportService.java +++ b/src/java/org/apache/cassandra/service/NativeTransportService.java @@ -18,7 +18,6 @@ package org.apache.cassandra.service; import java.net.InetAddress; -import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import com.google.common.annotations.VisibleForTesting; @@ -30,14 +29,11 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.EncryptionOptions; import org.apache.cassandra.metrics.ClientMetrics; -import org.apache.cassandra.transport.Dispatcher; import org.apache.cassandra.transport.Server; import org.apache.cassandra.utils.NativeLibrary; import io.netty.channel.EventLoopGroup; import io.netty.channel.epoll.Epoll; -import io.netty.channel.epoll.EpollEventLoopGroup; -import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.Version; import static org.apache.cassandra.config.CassandraRelevantProperties.NATIVE_EPOLL_ENABLED; @@ -51,9 +47,17 @@ public class NativeTransportService private static final Logger logger = LoggerFactory.getLogger(NativeTransportService.class); private Server server = null; - private boolean initialized = false; - private EventLoopGroup workerGroup; + private final NativeTransportSharedResourceManager sharedResourceManager; + + public NativeTransportService() { + this(NativeTransportSharedResourceManager.instance()); + } + + @VisibleForTesting + NativeTransportService(NativeTransportSharedResourceManager sharedResourceManager) { + this.sharedResourceManager = sharedResourceManager; + } /** * Creates netty thread pools and event loops. @@ -64,17 +68,7 @@ synchronized void initialize() if (initialized) return; - if (useEpoll()) - { - workerGroup = new EpollEventLoopGroup(); - logger.info("Netty using native Epoll event loop"); - } - else - { - workerGroup = new NioEventLoopGroup(); - logger.info("Netty using Java NIO event loop"); - } - + EventLoopGroup workerGroup = sharedResourceManager.getOrCreateSharedWorkerGroup(); int nativePort = DatabaseDescriptor.getNativeTransportPort(); InetAddress nativeAddr = DatabaseDescriptor.getRpcAddress(); @@ -87,6 +81,7 @@ synchronized void initialize() ClientMetrics.instance.init(server); + sharedResourceManager.registerActiveService(); initialized = true; } @@ -122,12 +117,8 @@ public void destroy() stop(); ClientMetrics.instance.release(); server = null; - - // shutdown executors used by netty for native transport server - if (workerGroup != null) - workerGroup.shutdownGracefully(3, 5, TimeUnit.SECONDS).awaitUninterruptibly(); - - Dispatcher.shutdown(); + sharedResourceManager.unregisterActiveService(); + initialized = false; } /** @@ -154,7 +145,7 @@ public boolean isRunning() @VisibleForTesting EventLoopGroup getWorkerGroup() { - return workerGroup; + return sharedResourceManager.getSharedWorkerGroup(); } @VisibleForTesting diff --git a/src/java/org/apache/cassandra/service/NativeTransportSharedResourceManager.java b/src/java/org/apache/cassandra/service/NativeTransportSharedResourceManager.java new file mode 100644 index 000000000000..855e55d06b3e --- /dev/null +++ b/src/java/org/apache/cassandra/service/NativeTransportSharedResourceManager.java @@ -0,0 +1,113 @@ +/* + * 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.cassandra.service; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.transport.Dispatcher; + +import io.netty.channel.EventLoopGroup; +import io.netty.channel.epoll.EpollEventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; + +/** + * Manages shared resources (EventLoopGroup, Dispatcher) for native transport services. + * This ensures both NativeTransportService and NativeTransportManagementService can share + * the same EventLoopGroup and properly coordinate Dispatcher shutdown. + */ +public class NativeTransportSharedResourceManager +{ + private static final NativeTransportSharedResourceManager instance = new NativeTransportSharedResourceManager(); + + private final AtomicReference sharedWorkerGroup = new AtomicReference<>(); + private final AtomicInteger activeServicesCount = new AtomicInteger(0); + private final Object mutex = new Object(); + + public static NativeTransportSharedResourceManager instance() + { + return instance; + } + + public EventLoopGroup getOrCreateSharedWorkerGroup() + { + EventLoopGroup group = sharedWorkerGroup.get(); + if (group == null) + { + synchronized (mutex) + { + group = sharedWorkerGroup.get(); + if (group == null) + { + if (NativeTransportService.useEpoll()) + group = new EpollEventLoopGroup(); + else + group = new NioEventLoopGroup(); + sharedWorkerGroup.set(group); + } + } + } + return group; + } + + public void registerActiveService() + { + activeServicesCount.incrementAndGet(); + } + + public void unregisterActiveService() + { + int remaining = activeServicesCount.decrementAndGet(); + if (remaining == 0) + shutdownSharedResources(); + } + + private void shutdownSharedResources() + { + synchronized (mutex) + { + EventLoopGroup group = sharedWorkerGroup.getAndSet(null); + if (group != null) + group.shutdownGracefully(3, 5, TimeUnit.SECONDS).awaitUninterruptibly(); + + // Dispatcher.shutdown() shuts down static executors shared by all Server instances + // Only call this when ALL native transport services are shutting down + Dispatcher.shutdown(); + } + } + + @VisibleForTesting + public EventLoopGroup getSharedWorkerGroup() + { + return sharedWorkerGroup.get(); + } + + @VisibleForTesting + public void forceShutdown() + { + synchronized (mutex) + { + shutdownSharedResources(); + activeServicesCount.set(0); + } + } +} diff --git a/src/java/org/apache/cassandra/transport/Connection.java b/src/java/org/apache/cassandra/transport/Connection.java index 074b558a10df..06bc391b6092 100644 --- a/src/java/org/apache/cassandra/transport/Connection.java +++ b/src/java/org/apache/cassandra/transport/Connection.java @@ -23,6 +23,7 @@ public class Connection { static final AttributeKey attributeKey = AttributeKey.valueOf("CONN"); + static final AttributeKey managementKey = AttributeKey.valueOf("IS_MANAGEMENT_CONN"); private final Channel channel; private final ProtocolVersion version; diff --git a/src/java/org/apache/cassandra/transport/PipelineConfigurator.java b/src/java/org/apache/cassandra/transport/PipelineConfigurator.java index 3cb88bbb85ed..6e86f61c5b11 100644 --- a/src/java/org/apache/cassandra/transport/PipelineConfigurator.java +++ b/src/java/org/apache/cassandra/transport/PipelineConfigurator.java @@ -117,17 +117,20 @@ public class PipelineConfigurator private final Dispatcher dispatcher; // Shared between pre-v5 and CQLMessage handlers private final QueueBackpressure queueBackpressure; + private final boolean isManagementConnection; public PipelineConfigurator(boolean epoll, boolean keepAlive, EncryptionOptions.TlsEncryptionPolicy encryptionPolicy, - Dispatcher dispatcher) + Dispatcher dispatcher, + boolean isManagementConnection) { this.epoll = epoll; this.keepAlive = keepAlive; this.tlsEncryptionPolicy = encryptionPolicy; this.dispatcher = dispatcher; this.queueBackpressure = QueueBackpressure.DEFAULT; + this.isManagementConnection = isManagementConnection; } @VisibleForTesting @@ -141,6 +144,7 @@ public PipelineConfigurator(boolean epoll, this.tlsEncryptionPolicy = encryptionPolicy; this.dispatcher = new Dispatcher(useLegacyFlusher); this.queueBackpressure = QueueBackpressure.DEFAULT; + this.isManagementConnection = false; } public ChannelFuture initializeChannel(final EventLoopGroup workerGroup, @@ -261,6 +265,9 @@ public void configureInitialPipeline(Channel channel, Connection.Factory connect pipeline.addFirst(CONNECTION_LIMIT_HANDLER, connectionLimitHandler); } + if (isManagementConnection) + channel.attr(Connection.managementKey).set(Boolean.TRUE); + long idleTimeout = DatabaseDescriptor.nativeTransportIdleTimeout(); if (idleTimeout > 0) { diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java index d7af40eb96f7..543446c7f2ed 100644 --- a/src/java/org/apache/cassandra/transport/Server.java +++ b/src/java/org/apache/cassandra/transport/Server.java @@ -85,7 +85,8 @@ public class Server implements CassandraDaemon.Server { public Connection newConnection(Channel channel, ProtocolVersion version) { - return new ServerConnection(channel, version, connectionTracker); + boolean isManagementConnection = Boolean.TRUE.equals(channel.attr(Connection.managementKey).get()); + return new ServerConnection(channel, version, connectionTracker, isManagementConnection); } }; @@ -117,7 +118,8 @@ private Server (Builder builder) : new PipelineConfigurator(useEpoll, DatabaseDescriptor.getRpcKeepAlive(), builder.tlsEncryptionPolicy, - dispatcher); + dispatcher, + builder.isManagementConnection); EventNotifier notifier = builder.eventNotifier != null ? builder.eventNotifier : new EventNotifier(); connectionTracker = new ConnectionTracker(isRunning::get); @@ -234,6 +236,7 @@ public static class Builder private InetSocketAddress socket; private PipelineConfigurator pipelineConfigurator; private EventNotifier eventNotifier; + private boolean isManagementConnection = false; public Builder withTlsEncryptionPolicy(EncryptionOptions.TlsEncryptionPolicy tlsEncryptionPolicy) { @@ -273,6 +276,12 @@ public Builder withEventNotifier(EventNotifier eventNotifier) return this; } + public Builder withManagementConnectionFlag(boolean management) + { + this.isManagementConnection = management; + return this; + } + public Server build() { return new Server(this); diff --git a/src/java/org/apache/cassandra/transport/ServerConnection.java b/src/java/org/apache/cassandra/transport/ServerConnection.java index 50986d1b27e6..7c06cef51f75 100644 --- a/src/java/org/apache/cassandra/transport/ServerConnection.java +++ b/src/java/org/apache/cassandra/transport/ServerConnection.java @@ -42,13 +42,20 @@ public class ServerConnection extends Connection private final ClientState clientState; private volatile ConnectionStage stage; public final Counter requests = new Counter(); + private final boolean isManagementConnection; ServerConnection(Channel channel, ProtocolVersion version, Connection.Tracker tracker) + { + this(channel, version, tracker, false); + } + + ServerConnection(Channel channel, ProtocolVersion version, Connection.Tracker tracker, boolean isManagementConnection) { super(channel, version, tracker); clientState = ClientState.forExternalCalls(channel.remoteAddress()); stage = ConnectionStage.ESTABLISHED; + this.isManagementConnection = isManagementConnection; } public ClientState getClientState() @@ -157,4 +164,9 @@ public boolean isSSL() .get("ssl"); return sslHandler != null; } + + public boolean isManagementConnection() + { + return isManagementConnection; + } } diff --git a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java index 85c531c20aa6..b19e8b88436d 100644 --- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java +++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java @@ -130,6 +130,12 @@ else if (compression.equals("lz4")) clientState.setDriverVersion(options.get(DRIVER_VERSION)); } + if (connection instanceof ServerConnection) + { + ServerConnection serverConnection = (ServerConnection) connection; + clientState.setManagement(serverConnection.isManagementConnection()); + } + IAuthenticator authenticator = DatabaseDescriptor.getAuthenticator(); if (authenticator.requireAuthentication()) { diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 8e1ca3faf720..3764bc7b0eae 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -276,12 +276,14 @@ public abstract class CQLTester */ private static boolean coordinatorExecution = false; + private static org.apache.cassandra.transport.Server managementServer; private static org.apache.cassandra.transport.Server server; private static JMXConnectorServer jmxServer; protected static String jmxHost; protected static int jmxPort; protected static MBeanServerConnection jmxConnection; + protected static int managementPort; protected static int nativePort; protected static final InetAddress nativeAddr; private static final Map clusters = new HashMap<>(); @@ -327,6 +329,7 @@ public static final ProtocolVersion getDefaultVersion() nativeAddr = InetAddress.getLoopbackAddress(); nativePort = getAutomaticallyAllocatedPort(nativeAddr); + managementPort = getAutomaticallyAllocatedPort(nativeAddr); } private List keyspaces = new ArrayList<>(); @@ -505,6 +508,9 @@ public static void tearDownClass() if (server != null) server.stop(); + if (managementServer != null) + managementServer.stop(); + // We use queryInternal for CQLTester so prepared statement will populate our internal cache (if reusePrepared is used; otherwise prepared // statements are not cached but re-prepared every time). So we clear the cache between test files to avoid accumulating too much. if (reusePrepared) @@ -609,7 +615,7 @@ public static List buildNodetoolCqlArgs(List args) List allArgs = new ArrayList<>(); allArgs.add("bin/nodetool"); allArgs.add("-p"); - allArgs.add(Integer.toString(nativePort)); + allArgs.add(Integer.toString(managementPort)); allArgs.add("-h"); allArgs.add(nativeAddr.getHostAddress()); allArgs.addAll(args); @@ -737,6 +743,7 @@ protected static void requireNetwork(Consumer serverConfigurator startServices(); startServer(serverConfigurator); + startManagementServer(serverConfigurator); } protected static void requireNetworkWithoutDriver() @@ -746,6 +753,7 @@ protected static void requireNetworkWithoutDriver() startServices(); startServer(server -> {}); + startManagementServer(server -> {}); } private static void startServices() @@ -781,6 +789,13 @@ protected static void reinitializeNetwork(Consumer serverConfigu clusterBuilderConfigurator = clusterConfigurator; startServer(serverConfigurator); + + if (managementServer != null && managementServer.isRunning()) + { + managementServer.stop(); + managementServer = null; + } + startManagementServer(serverConfigurator); } private static void startServer(Consumer decorator) @@ -793,6 +808,17 @@ private static void startServer(Consumer decorator) server.start(); } + private static void startManagementServer(Consumer decorator) + { + managementPort = getAutomaticallyAllocatedPort(nativeAddr); + Server.Builder serverBuilder = new Server.Builder().withHost(nativeAddr) + .withPort(managementPort) + .withManagementConnectionFlag(true); + decorator.accept(serverBuilder); + managementServer = serverBuilder.build(); + managementServer.start(); + } + private static Cluster initClientCluster(User user, ProtocolVersion version, boolean useEncryption, boolean useClientCert) { SocketOptions socketOptions = diff --git a/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java b/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java index 960289940062..16bcf659c73f 100644 --- a/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java +++ b/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java @@ -25,10 +25,13 @@ import org.junit.BeforeClass; import org.junit.Test; +import org.apache.cassandra.Util; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.EncryptionOptions; import org.apache.cassandra.transport.Server; +import io.netty.channel.EventLoopGroup; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -49,6 +52,7 @@ public static void setupDD() public void resetConfig() { DatabaseDescriptor.updateNativeProtocolEncryptionOptions(update -> new EncryptionOptions.ClientEncryptionOptions.Builder(defaultOptions).build()); + NativeTransportSharedResourceManager.instance().forceShutdown(); } @Test @@ -83,12 +87,14 @@ public void testIgnoresStoppedOnAlreadyStopped() @Test public void testDestroy() { + NativeTransportSharedResourceManager mgr = NativeTransportSharedResourceManager.instance(); + EventLoopGroup eventLoopGroup = mgr.getOrCreateSharedWorkerGroup(); withService((NativeTransportService service) -> { - BooleanSupplier allTerminated = () -> - service.getWorkerGroup().isShutdown() && service.getWorkerGroup().isTerminated(); + BooleanSupplier allTerminated = () -> eventLoopGroup.isShutdown() && eventLoopGroup.isTerminated(); assertFalse(allTerminated.getAsBoolean()); service.destroy(); assertTrue(allTerminated.getAsBoolean()); + Util.spinUntilTrue(() -> mgr.getSharedWorkerGroup() == null); }); } From c3f4996cbe631029bac76b9920f610220c853916 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Fri, 23 Jan 2026 23:11:46 +0100 Subject: [PATCH 06/14] add new ExceptionCode, handle exceptions for cql strategy --- .../statements/ExecuteCommandStatement.java | 68 +++++--- .../CommandRequestExecutionException.java | 43 ++++++ .../cassandra/exceptions/ExceptionCode.java | 1 + .../CommandAuthorizationException.java} | 13 +- .../management/CommandExecutionException.java | 37 +++++ .../cassandra/management/CommandInvoker.java | 107 ------------- ...ervice.java => CommandInvokerService.java} | 146 +++++++++++++++--- ...n.java => CommandInvokerServiceMBean.java} | 6 +- .../cassandra/management/CommandMBean.java | 22 +-- .../CommandValidationException.java | 27 ++++ .../picocli/PicocliCommandAdapter.java | 25 ++- .../cassandra/service/CassandraDaemon.java | 6 +- .../org/apache/cassandra/tools/NodeProbe.java | 22 ++- .../strategy/CqlCommandExecutionStrategy.java | 47 ++++-- .../ProtocolAwareExecutionStrategy.java | 1 + .../transport/messages/ErrorMessage.java | 30 ++++ .../cassandra/distributed/impl/Instance.java | 4 +- .../cql3/CQLNodetoolProtocolTester.java | 2 +- .../org/apache/cassandra/cql3/CQLTester.java | 7 +- .../management/CommandServiceTest.java | 8 +- .../cassandra/tools/nodetool/CompactTest.java | 14 +- .../cassandra/transport/ErrorMessageTest.java | 69 +++++++++ 22 files changed, 478 insertions(+), 227 deletions(-) create mode 100644 src/java/org/apache/cassandra/exceptions/CommandRequestExecutionException.java rename src/java/org/apache/cassandra/{exceptions/CommandExecutionException.java => management/CommandAuthorizationException.java} (71%) create mode 100644 src/java/org/apache/cassandra/management/CommandExecutionException.java delete mode 100644 src/java/org/apache/cassandra/management/CommandInvoker.java rename src/java/org/apache/cassandra/management/{CommandService.java => CommandInvokerService.java} (65%) rename src/java/org/apache/cassandra/management/{CommandServiceMBean.java => CommandInvokerServiceMBean.java} (93%) create mode 100644 src/java/org/apache/cassandra/management/CommandValidationException.java diff --git a/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java b/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java index 32f7228f4506..4e167abe3668 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java @@ -22,8 +22,11 @@ import java.util.List; import java.util.Map; +import com.google.common.base.Throwables; + import org.apache.cassandra.audit.AuditLogContext; import org.apache.cassandra.audit.AuditLogEntryType; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.CQLStatement; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.ColumnSpecification; @@ -31,14 +34,16 @@ import org.apache.cassandra.cql3.ResultSet; import org.apache.cassandra.cql3.terms.ArrayLiteral; import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.exceptions.CommandExecutionException; +import org.apache.cassandra.exceptions.CommandRequestExecutionException; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.exceptions.UnauthorizedException; +import org.apache.cassandra.management.CommandAuthorizationException; import org.apache.cassandra.management.CommandExecutionArgsSerde; -import org.apache.cassandra.management.CommandService; +import org.apache.cassandra.management.CommandExecutionException; +import org.apache.cassandra.management.CommandInvokerService; +import org.apache.cassandra.management.CommandValidationException; import org.apache.cassandra.management.api.Command; import org.apache.cassandra.management.api.CommandExecutionArgs; -import org.apache.cassandra.serializers.MarshalException; import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.transport.Dispatcher; @@ -66,19 +71,30 @@ public CQLStatement prepare(ClientState state) public void authorize(ClientState state) throws UnauthorizedException { + // TODO: CASSANRA-XXXXX. Restrict command execution to environments without authentication + // This is a temporary limitation until full authentication support is implemented. + if (DatabaseDescriptor.getAuthenticator().requireAuthentication()) + { + throw new UnauthorizedException("Command execution via management port is currently only supported " + + "when authentication is disabled (AllowAllAuthenticator). " + + "Full authentication and authorization support will be added in a " + + "future release."); + } + + // Validate login (will succeed with AllowAllAuthenticator) state.validateLogin(); } @Override public void validate(ClientState state) throws InvalidRequestException { - Command command = CommandService.instance.getRegistry().command(commandName); + Command command = CommandInvokerService.instance.getRegistry().command(commandName); if (command == null) throw new InvalidRequestException("Command not found: " + commandName); } @Override - public ResultMessage execute(QueryState state, QueryOptions options, Dispatcher.RequestTime requestTime) throws InvalidRequestException + public ResultMessage execute(QueryState state, QueryOptions options, Dispatcher.RequestTime requestTime) { try { @@ -86,7 +102,7 @@ public ResultMessage execute(QueryState state, QueryOptions options, Dispatcher. if (!clientState.isInternal && !clientState.isManagement()) throw new InvalidRequestException("Command execution is only allowed via native management interface"); - Command command = CommandService.instance.getRegistry().command(commandName); + Command command = CommandInvokerService.instance.getRegistry().command(commandName); if (command == null) throw new InvalidRequestException("Command not found: " + commandName); @@ -94,30 +110,42 @@ public ResultMessage execute(QueryState state, QueryOptions options, Dispatcher. CommandExecutionArgs commandArgs = CommandExecutionArgsSerde.fromMap(paramsMap, command.metadata()); - String output = CommandService.instance.executeCommand(commandName, commandArgs); - ColumnSpecification outputColumn = new ColumnSpecification("system", - "command_output", - new ColumnIdentifier("output", true), - UTF8Type.instance); + CommandInvokerService.CommandResult result = CommandInvokerService.instance.invokeCommand(commandName, commandArgs); - ResultSet resultSet = new ResultSet(new ResultSet.ResultMetadata(List.of(outputColumn))); - resultSet.addColumnValue(bytes(output)); + ResultSet resultSet = getCommandResultSet(); + resultSet.addColumnValue(bytes(result.getExecutionId())); + resultSet.addColumnValue(bytes(result.getOutput())); return new ResultMessage.Rows(resultSet); } - catch (CommandExecutionException e) + catch (CommandAuthorizationException e) { - throw new InvalidRequestException("Failed to execute command '" + commandName + "': " + e.getMessage(), e); + throw new UnauthorizedException(e.getMessage()); } - catch (MarshalException e) + catch (CommandValidationException e) { - throw new InvalidRequestException("Invalid JSON format for command arguments: " + e.getMessage(), e); + throw new InvalidRequestException(Throwables.getStackTraceAsString(e), e.getCause()); } - catch (IllegalArgumentException e) + catch (CommandExecutionException e) { - throw new InvalidRequestException("Invalid command arguments: " + e.getMessage(), e); + throw new CommandRequestExecutionException(e.getExecutionId(), + Throwables.getStackTraceAsString(e), + e.getCause()); } } + private static ResultSet getCommandResultSet() + { + ColumnSpecification executionIdColumn = new ColumnSpecification("system", + "command_output", + new ColumnIdentifier("execution_id", true), + UTF8Type.instance); + ColumnSpecification outputColumn = new ColumnSpecification("system", + "command_output", + new ColumnIdentifier("output", true), + UTF8Type.instance); + return new ResultSet(new ResultSet.ResultMetadata(List.of(executionIdColumn, outputColumn))); + } + /** Parse a CQL list literal string like "['k1', 'k2']" into a List of strings, and unquote CQL string literals. */ private static Map convertArgsMap(Map args) { @@ -203,9 +231,7 @@ private static int findClosingQuote(String content, int start, String listLitera return i; } else - { i++; - } } throw new IllegalArgumentException("Unclosed string in list literal: " + listLiteral); } diff --git a/src/java/org/apache/cassandra/exceptions/CommandRequestExecutionException.java b/src/java/org/apache/cassandra/exceptions/CommandRequestExecutionException.java new file mode 100644 index 000000000000..7a3ef01a3373 --- /dev/null +++ b/src/java/org/apache/cassandra/exceptions/CommandRequestExecutionException.java @@ -0,0 +1,43 @@ +/* + * 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.cassandra.exceptions; + +import java.util.UUID; + +public class CommandRequestExecutionException extends RequestExecutionException +{ + private final UUID commandExecutionId; + + public CommandRequestExecutionException(UUID commandExecutionId, String msg, Throwable cause) + { + super(ExceptionCode.COMMAND_FAILED, msg, cause); + this.commandExecutionId = commandExecutionId; + } + + public CommandRequestExecutionException(UUID commandExecutionId, String msg) + { + super(ExceptionCode.COMMAND_FAILED, msg); + this.commandExecutionId = commandExecutionId; + } + + public UUID getCommandExecutionId() + { + return commandExecutionId; + } +} diff --git a/src/java/org/apache/cassandra/exceptions/ExceptionCode.java b/src/java/org/apache/cassandra/exceptions/ExceptionCode.java index 8bb0cfd77953..bdc948186939 100644 --- a/src/java/org/apache/cassandra/exceptions/ExceptionCode.java +++ b/src/java/org/apache/cassandra/exceptions/ExceptionCode.java @@ -48,6 +48,7 @@ public enum ExceptionCode WRITE_FAILURE (0x1500), CDC_WRITE_FAILURE (0x1600), CAS_WRITE_UNKNOWN (0x1700), + COMMAND_FAILED (0x1800), // 2xx: problem validating the request SYNTAX_ERROR (0x2000), diff --git a/src/java/org/apache/cassandra/exceptions/CommandExecutionException.java b/src/java/org/apache/cassandra/management/CommandAuthorizationException.java similarity index 71% rename from src/java/org/apache/cassandra/exceptions/CommandExecutionException.java rename to src/java/org/apache/cassandra/management/CommandAuthorizationException.java index 2d07df846d22..c1e85bf1061d 100644 --- a/src/java/org/apache/cassandra/exceptions/CommandExecutionException.java +++ b/src/java/org/apache/cassandra/management/CommandAuthorizationException.java @@ -16,17 +16,12 @@ * limitations under the License. */ -package org.apache.cassandra.exceptions; +package org.apache.cassandra.management; -public class CommandExecutionException extends CassandraException +public class CommandAuthorizationException extends Exception { - public CommandExecutionException(String msg) + public CommandAuthorizationException(String msg) { - super(ExceptionCode.INVALID, msg); - } - - public CommandExecutionException(String msg, Throwable cause) - { - super(ExceptionCode.INVALID, msg, cause); + super(msg); } } diff --git a/src/java/org/apache/cassandra/management/CommandExecutionException.java b/src/java/org/apache/cassandra/management/CommandExecutionException.java new file mode 100644 index 000000000000..3c4a2d712f51 --- /dev/null +++ b/src/java/org/apache/cassandra/management/CommandExecutionException.java @@ -0,0 +1,37 @@ +/* + * 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.cassandra.management; + +import java.util.UUID; + +public class CommandExecutionException extends Exception +{ + private final UUID executionId; + + public CommandExecutionException(String message, Throwable cause, UUID executionId) + { + super(message, cause); + this.executionId = executionId; + } + + public UUID getExecutionId() + { + return executionId; + } +} diff --git a/src/java/org/apache/cassandra/management/CommandInvoker.java b/src/java/org/apache/cassandra/management/CommandInvoker.java deleted file mode 100644 index c522f2c84665..000000000000 --- a/src/java/org/apache/cassandra/management/CommandInvoker.java +++ /dev/null @@ -1,107 +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 org.apache.cassandra.management; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.cassandra.exceptions.CommandExecutionException; -import org.apache.cassandra.management.api.Command; -import org.apache.cassandra.management.api.CommandExecutionArgs; -import org.apache.cassandra.management.api.CommandExecutionContext; -import org.apache.cassandra.management.api.CommandMetadata; -import org.apache.cassandra.management.api.OptionMetadata; -import org.apache.cassandra.management.api.ParameterMetadata; - -/** - * Default implementation of CommandInvoker. - * - *

This invoker: - *

    - *
  • Validates command arguments against metadata
  • - *
  • Executes the command via Command.execute()
  • - *
  • Handles exceptions and provides error context
  • - *
  • Can be extended for cross-cutting concerns (logging, metrics, etc.)
  • - *
- * - *

The invoker is stateless and can be reused for multiple commands. - */ -public class CommandInvoker -{ - private static final Logger logger = LoggerFactory.getLogger(CommandInvoker.class); - - private final Command command; - private final CommandExecutionArgs arguments; - private final CommandExecutionContext executionContext; - - public CommandInvoker(Command command, CommandExecutionArgs arguments, CommandExecutionContext executionContext) - { - if (command == null) - throw new IllegalArgumentException("Command cannot be null"); - if (arguments == null) - throw new IllegalArgumentException("CommandExecutionArgs cannot be null"); - if (executionContext == null) - throw new IllegalArgumentException("CommandExecutionContext cannot be null"); - - this.command = command; - this.arguments = arguments; - this.executionContext = executionContext; - } - - public R invoke() - { - CommandMetadata metadata = command.metadata(); - String commandName = metadata.name(); - - try - { - validateArguments(arguments, metadata); - return command.execute(arguments, executionContext); - } - catch (IllegalArgumentException e) - { - logger.warn("Invalid arguments for command '{}': {}", commandName, e.getMessage()); - throw new CommandExecutionException( - String.format("Invalid arguments for command '%s': %s", commandName, e.getMessage()), e); - } - catch (Exception e) - { - logger.error("Error executing command '{}'", commandName, e); - throw new CommandExecutionException( - String.format("Failed to execute command '%s': %s", commandName, e.getMessage()), e); - } - } - - /** Validate command arguments against metadata. */ - protected void validateArguments(CommandExecutionArgs arguments, CommandMetadata metadata) - { - for (OptionMetadata option : metadata.options()) - { - if (option.required() && !arguments.hasOption(option)) - throw new IllegalArgumentException(String.format("Required option '%s' is missing", option.paramLabel())); - } - - for (ParameterMetadata param : metadata.parameters()) - { - if (param.required() && !arguments.hasParameter(param)) - throw new IllegalArgumentException(String.format("Required parameter at index %d ('%s') is missing", - param.index(), param.paramLabel())); - } - } -} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/management/CommandService.java b/src/java/org/apache/cassandra/management/CommandInvokerService.java similarity index 65% rename from src/java/org/apache/cassandra/management/CommandService.java rename to src/java/org/apache/cassandra/management/CommandInvokerService.java index 283483762570..b50ceeff7b6e 100644 --- a/src/java/org/apache/cassandra/management/CommandService.java +++ b/src/java/org/apache/cassandra/management/CommandInvokerService.java @@ -37,16 +37,20 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.exceptions.CommandExecutionException; +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.management.api.Command; import org.apache.cassandra.management.api.CommandExecutionArgs; import org.apache.cassandra.management.api.CommandExecutionContext; +import org.apache.cassandra.management.api.CommandMetadata; import org.apache.cassandra.management.api.CommandRegistry; +import org.apache.cassandra.management.api.OptionMetadata; +import org.apache.cassandra.management.api.ParameterMetadata; import org.apache.cassandra.tools.NodeProbe; import org.apache.cassandra.tools.Output; import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.MBeanWrapper; +import static java.lang.String.format; import static org.apache.cassandra.management.ManagementUtils.countCommands; /** @@ -59,15 +63,15 @@ * - Integrates with daemon startup * - Manages state (registry, MBean instances) */ -public class CommandService implements CommandServiceMBean +public class CommandInvokerService implements CommandInvokerServiceMBean { private static final String MBEAN_DOMAIN = "org.apache.cassandra.management"; private static final String MBEAN_TYPE_COMMAND = "Command"; private static final int MAX_EXECUTION_HISTORY = 100; private static final String COMMAND_NAME_DELIMITER = "."; - private static final Logger logger = LoggerFactory.getLogger(CommandService.class); + private static final Logger logger = LoggerFactory.getLogger(CommandInvokerService.class); - public static final CommandService instance = new CommandService(); + public static final CommandInvokerService instance = new CommandInvokerService(); private final BoundedExecutionHistory executionHistory = new BoundedExecutionHistory(MAX_EXECUTION_HISTORY); private final Map commandMBeanNames = new ConcurrentHashMap<>(); @@ -76,7 +80,7 @@ public class CommandService implements CommandServiceMBean private volatile boolean started = false; - private CommandService() + private CommandInvokerService() { this.registry = new CassandraCommandRegistry(); } @@ -115,7 +119,7 @@ public synchronized void stop() } catch (Exception e) { - logger.warn("Failed to unregister CommandService MBean", e); + logger.warn("Failed to unregister CommandInvokerService MBean", e); } started = false; @@ -134,16 +138,11 @@ public CommandRegistry getRegistry() * @param arguments arguments for the command execution. * @return captured output of the command execution. */ - public String executeCommand(String commandName, CommandExecutionArgs arguments) + public CommandResult invokeCommand(String commandName, CommandExecutionArgs arguments) + throws CommandExecutionException, CommandValidationException, CommandAuthorizationException { - // TODO do input arguments validation and conversion to the CommandExecutionArgs class. - // TODO this should be called via invoker that handles permissions, logging, etc. if (!started) - throw new IllegalStateException("CommandService is not started"); - - Command command = registry.command(commandName); - if (command == null) - throw new IllegalArgumentException("Command not found: " + commandName); + throw new IllegalStateException("CommandInvokerService is not started"); UUID executionId = UUID.randomUUID(); CapturingOutput captured = new CapturingOutput(); @@ -152,24 +151,65 @@ public String executeCommand(String commandName, CommandExecutionArgs arguments) try { + Command command = registry.command(commandName); + if (command == null) + throw new IllegalArgumentException("Command not found: " + commandName); + + // TODO: CASSANRA-XXXXX. Restrict command execution to environments without authentication + // This is a temporary limitation until full authentication support is implemented. + if (DatabaseDescriptor.getAuthenticator().requireAuthentication()) + { + throw new CommandAuthorizationException(format("Command execution '%s' via management port is currently " + + "only supported when authentication is disabled " + + "(AllowAllAuthenticator). Full authentication and authorization " + + "support will be added in a future release.", commandName)); + } + logger.info("Executing command '{}' with execution ID: {}", commandName, executionId); executionHistory.add(record); - CommandInvoker invoker = new CommandInvoker<>(command, arguments, executionContext); // Currently, for picocli-based commands in C*, which have no structured result, // the output is written to the Output in the context. - Object ignore = invoker.invoke(); + // TODO do input arguments validation and conversion to the CommandExecutionArgs class. + // TODO this should be called via invoker that handles permissions, logging, etc. + validateArguments(arguments, command.metadata()); + Object ignore = command.execute(arguments, executionContext); record.completed(Clock.Global.currentTimeMillis()); logger.info("Command '{}' (execution ID: {}) completed successfully", commandName, executionId); - - return captured.getCapturedOutput(); + return new CommandResult(executionId, + captured.getCapturedOutput(), + record.startTime, + record.endTime - record.startTime); + } + catch (CommandAuthorizationException e) + { + record.failed(Clock.Global.currentTimeMillis(), e); + logger.error("Command '{}' (execution ID: {}) authorization failed", commandName, executionId, e); + throw e; + } + catch (IllegalStateException | IllegalArgumentException e) + { + record.failed(Clock.Global.currentTimeMillis(), e); + String msg = format("Bad usage for command '%s' (execution ID: %s)", commandName, executionId); + logger.error(msg, e); + throw new CommandValidationException(msg, e); } catch (Exception e) { record.failed(Clock.Global.currentTimeMillis(), e); - logger.error("Unexpected error executing command '{}' (execution ID: {})", commandName, executionId, e); - throw new CommandExecutionException(String.format("Failed to execute command '%s': %s", commandName, e.getMessage()), e); + String msg = format("Command '%s' (execution ID: %s) execution failed", commandName, executionId); + logger.error(msg, e); + throw new CommandExecutionException(msg, e, executionId); + } + catch (Throwable e) + { + record.failed(Clock.Global.currentTimeMillis(), e); + logger.error("Command '{}' (execution ID: {}) unexpected error", commandName, executionId, e); + throw new CommandExecutionException(format("Unexpected error while executing '%s': %s", + commandName, e.getMessage()), + e, + executionId); } } @@ -181,6 +221,23 @@ public String[] getCommandNames() return commandNames.toArray(new String[0]); } + /** Validate command arguments against metadata. */ + protected void validateArguments(CommandExecutionArgs arguments, CommandMetadata metadata) + { + for (OptionMetadata option : metadata.options()) + { + if (option.required() && !arguments.hasOption(option)) + throw new IllegalArgumentException(String.format("Required option '%s' is missing", option.paramLabel())); + } + + for (ParameterMetadata param : metadata.parameters()) + { + if (param.required() && !arguments.hasParameter(param)) + throw new IllegalArgumentException(String.format("Required parameter at index %d ('%s') is missing", + param.index(), param.paramLabel())); + } + } + private void collectCommandNamesRecursively(CommandRegistry registry, String parentCommandName, List result) @@ -241,9 +298,9 @@ private void registerCommandMBeansRecursively(CommandRegistry registry, String p try { String escapedName = ObjectName.quote(fullCommandName); - ObjectName objectName = new ObjectName(String.format("%s:type=%s,name=%s", - MBEAN_DOMAIN, MBEAN_TYPE_COMMAND, escapedName)); - CommandMBean commandMBean = new CommandMBean(command, this::executeCommand); + ObjectName objectName = new ObjectName(format("%s:type=%s,name=%s", + MBEAN_DOMAIN, MBEAN_TYPE_COMMAND, escapedName)); + CommandMBean commandMBean = new CommandMBean(command, this::invokeCommand); MBeanWrapper.instance.registerMBean(commandMBean, objectName, MBeanWrapper.OnException.LOG); commandMBeanNames.put(fullCommandName, objectName); @@ -307,6 +364,42 @@ public Output output() } } + public static class CommandResult + { + private final UUID executionId; + private final String output; + private final long startTime; + private final long durationMillis; + + public CommandResult(UUID executionId, String output, long startTime, long durationMillis) + { + this.executionId = executionId; + this.output = output; + this.startTime = startTime; + this.durationMillis = durationMillis; + } + + public UUID getExecutionId() + { + return executionId; + } + + public String getOutput() + { + return output; + } + + public long getStartTime() + { + return startTime; + } + + public long getDurationMillis() + { + return durationMillis; + } + } + private static class ExecutionHistory { final UUID executionId; @@ -360,4 +453,11 @@ public void add(ExecutionHistory info) } } } + + @FunctionalInterface + public interface Executor + { + CommandResult execute(String commandName, CommandExecutionArgs arguments) + throws CommandExecutionException, CommandValidationException, CommandAuthorizationException; + } } diff --git a/src/java/org/apache/cassandra/management/CommandServiceMBean.java b/src/java/org/apache/cassandra/management/CommandInvokerServiceMBean.java similarity index 93% rename from src/java/org/apache/cassandra/management/CommandServiceMBean.java rename to src/java/org/apache/cassandra/management/CommandInvokerServiceMBean.java index 18572e7f780a..bc81118e01bb 100644 --- a/src/java/org/apache/cassandra/management/CommandServiceMBean.java +++ b/src/java/org/apache/cassandra/management/CommandInvokerServiceMBean.java @@ -19,12 +19,12 @@ package org.apache.cassandra.management; /** - * MBean interface for CommandService. + * MBean interface for CommandInvokerService. * Exposes registry-level operations (not individual commands). */ -public interface CommandServiceMBean +public interface CommandInvokerServiceMBean { - String MBEAN_NAME = "org.apache.cassandra.management:type=CommandService"; + String MBEAN_NAME = "org.apache.cassandra.management:type=CommandInvokerService"; /** * Get a list of all command names in the registry. diff --git a/src/java/org/apache/cassandra/management/CommandMBean.java b/src/java/org/apache/cassandra/management/CommandMBean.java index acf7101eda29..22c4f0db8edc 100644 --- a/src/java/org/apache/cassandra/management/CommandMBean.java +++ b/src/java/org/apache/cassandra/management/CommandMBean.java @@ -23,7 +23,6 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.function.BiFunction; import java.util.stream.Collectors; import javax.management.Attribute; @@ -105,9 +104,9 @@ public class CommandMBean implements DynamicMBean private final Command command; private final CommandMetadata metadata; - private final BiFunction executor; + private final CommandInvokerService.Executor executor; - public CommandMBean(Command command, BiFunction executor) + public CommandMBean(Command command, CommandInvokerService.Executor executor) { this.command = command; this.metadata = command.metadata(); @@ -164,17 +163,22 @@ public Object invoke(String actionName, Object[] params, String[] signature) thr try { CommandExecutionArgs args = CommandExecutionArgsSerde.fromJson(jsonParams, metadata); - return executor.apply(command.name(), args); + return executor.execute(command.name(), args).getOutput(); } - catch (MarshalException e) + catch (CommandAuthorizationException e) { - logger.error("Error parsing JSON parameters for command: {}", metadata.name(), e); - throw new MBeanException(e, "Invalid JSON format: " + e.getMessage()); + logger.error("Authorization error executing command: {}", metadata.name(), e); + throw new SecurityException("Access Denied", e); } - catch (Exception e) + catch (CommandValidationException | MarshalException e) + { + logger.error("Validation error for command: {}", metadata.name(), e); + throw new IllegalArgumentException("Command validation failed: " + e.getMessage(), e.getCause()); + } + catch (CommandExecutionException e) { logger.error("Error executing command: {}", metadata.name(), e); - throw new MBeanException(e, "Failed to execute command: " + metadata.name() + ": " + e.getMessage()); + throw new RuntimeException("Failed to execute command: " + metadata.name() + ": " + e.getMessage()); } } diff --git a/src/java/org/apache/cassandra/management/CommandValidationException.java b/src/java/org/apache/cassandra/management/CommandValidationException.java new file mode 100644 index 000000000000..4419c4ca7cd8 --- /dev/null +++ b/src/java/org/apache/cassandra/management/CommandValidationException.java @@ -0,0 +1,27 @@ +/* + * 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.cassandra.management; + +public class CommandValidationException extends Exception +{ + public CommandValidationException(String message, Throwable cause) + { + super(message, cause); + } +} diff --git a/src/java/org/apache/cassandra/management/picocli/PicocliCommandAdapter.java b/src/java/org/apache/cassandra/management/picocli/PicocliCommandAdapter.java index 8aa9e12c3d9e..f31a2a5010d1 100644 --- a/src/java/org/apache/cassandra/management/picocli/PicocliCommandAdapter.java +++ b/src/java/org/apache/cassandra/management/picocli/PicocliCommandAdapter.java @@ -53,25 +53,18 @@ public CommandMetadata metadata() @Override public Void execute(CommandExecutionArgs arguments, CommandExecutionContext context) { - try - { - CommandLine commandLine = new CommandLine(commandClass, new InjectCassandraContext(context.output())); - AbstractCommand userCommand = commandLine.getCommand(); + CommandLine commandLine = new CommandLine(commandClass, new InjectCassandraContext(context.output())); + AbstractCommand userCommand = commandLine.getCommand(); - userCommand.probe(context.nodeProbe()); - PicocliCommandArgsConverter.toCommand(arguments, userCommand); + userCommand.probe(context.nodeProbe()); + PicocliCommandArgsConverter.toCommand(arguments, userCommand); - // TODO utilize picocli to invoke the command with proper error handling? - userCommand.run(); + // TODO utilize picocli to invoke the command with proper error handling? + userCommand.run(); - // The picocli commands in Cassandra do not return any value, so we return null here. - // If a command has a different return type, e.g. implements Callable, we would need to handle that accordingly. - return null; - } - catch (Exception e) - { - throw new RuntimeException("Failed to execute command: " + commandClass.getSimpleName(), e); - } + // The picocli commands in Cassandra do not return any value, so we return null here. + // If a command has a different return type, e.g. implements Callable, we would need to handle that accordingly. + return null; } private static class InjectCassandraContext implements CommandLine.IFactory diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index 6053d360d9f3..7a05c9f94534 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -76,7 +76,7 @@ import org.apache.cassandra.io.util.FileUtils; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.locator.Locator; -import org.apache.cassandra.management.CommandService; +import org.apache.cassandra.management.CommandInvokerService; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.DefaultNameFactory; import org.apache.cassandra.net.StartupClusterConnectivityChecker; @@ -387,7 +387,7 @@ protected void setup() QueryProcessor.instance.preloadPreparedStatements(); // Initialize command service (after JMX, before StorageService.initServer) - CommandService.instance.start(); + CommandInvokerService.instance.start(); // start server internals StorageService.instance.registerDaemon(this); @@ -743,7 +743,7 @@ public void stop() // jsvc takes care of taking the rest down logger.info("Cassandra shutting down..."); destroyClientTransports(); - CommandService.instance.stop(); + CommandInvokerService.instance.stop(); StorageService.instance.setRpcReady(false); if (jmxServer != null) diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java index 83579e6d7a1c..21fd3f2d61ee 100644 --- a/src/java/org/apache/cassandra/tools/NodeProbe.java +++ b/src/java/org/apache/cassandra/tools/NodeProbe.java @@ -22,6 +22,7 @@ import java.lang.management.MemoryMXBean; import java.lang.management.MemoryUsage; import java.lang.management.RuntimeMXBean; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Proxy; import java.net.InetAddress; @@ -85,6 +86,7 @@ import org.apache.cassandra.locator.DynamicEndpointSnitchMBean; import org.apache.cassandra.locator.EndpointSnitchInfoMBean; import org.apache.cassandra.locator.LocationInfoMBean; +import org.apache.cassandra.management.CommandInvokerService; import org.apache.cassandra.management.MBeanAccessor; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.StorageMetrics; @@ -142,7 +144,7 @@ *

  • No Network Overhead: Direct in-process access to MBeans without JMX/RMI overhead
  • *
  • Better Performance: No serialization/deserialization of JMX calls
  • *
  • Unified API: Same NodeProbe interface works for both client and server-side execution
  • - *
  • Management API Integration: Commands can be executed via {@link org.apache.cassandra.management.CommandService} and exposed + *
  • Management API Integration: Commands can be executed via {@link CommandInvokerService} and exposed * through various protocols (native protocol, REST, etc.)
  • * * @@ -157,7 +159,7 @@ * * @see MBeanAccessor * @see RemoteJmxMBeanAccessor - * @see org.apache.cassandra.management.CommandService + * @see CommandInvokerService */ public class NodeProbe implements AutoCloseable { @@ -2508,8 +2510,7 @@ private static class LazyMBeanProxy implements java.lang.reflect.InvocationHa } @Override - public Object invoke(Object proxy, Method method, Object[] args) - throws Throwable + public Object invoke(Object proxy, Method method, Object[] args) throws Throwable { if (delegate == null) { @@ -2519,7 +2520,18 @@ public Object invoke(Object proxy, Method method, Object[] args) delegate = provider.findMBean(mbeanClass); } } - return method.invoke(delegate, args); + + try + { + return method.invoke(delegate, args); + } + catch (InvocationTargetException e) + { + Throwable cause = e.getCause(); + if (cause == null) + throw e; + throw cause; + } } @SuppressWarnings("unchecked") diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java index 93010779d8c6..c22982b11c9d 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java @@ -29,6 +29,9 @@ import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.exceptions.CommandRequestExecutionException; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.UnauthorizedException; import org.apache.cassandra.management.api.CommandExecutionArgs; import org.apache.cassandra.management.api.OptionMetadata; import org.apache.cassandra.management.api.ParameterMetadata; @@ -74,7 +77,7 @@ public int execute(CommandLine.ParseResult parseResult) throws CommandLine.Execu if (commandName == null || commandName.isEmpty()) return new CommandLine.RunLast().execute(parseResult); - // Command is already populated with args from CommandLine.parseArgs(), convert to CommandExecutionArgs + // Command is already populated with args from CommandLine.parseArgs(), converting it to CommandExecutionArgs CommandExecutionArgs args = PicocliCommandArgsConverter.fromCommand(userObject); try @@ -82,27 +85,47 @@ public int execute(CommandLine.ParseResult parseResult) throws CommandLine.Execu String cqlCommand = buildCqlCommandString(commandName, args); ResultMessage result = connect.client().execute(cqlCommand, ConsistencyLevel.ONE); - if (result instanceof ResultMessage.Rows) - { + if (result instanceof ResultMessage.Rows) { ResultMessage.Rows rows = (ResultMessage.Rows) result; - if (!rows.result.isEmpty() && !rows.result.metadata.names.isEmpty()) - { + assert rows.result.size() == 2 : "Command execution result schema has been changed. " + + "Expected 2 columns in result, got " + rows.result.size(); + if (!rows.result.isEmpty() && rows.result.metadata.names.size() >= 2) { List firstRow = rows.result.rows.get(0); - if (!firstRow.isEmpty()) - { - String output = UTF8Type.instance.getSerializer().deserialize(firstRow.get(0)); - // For CQL strategy, we don't have a NodeProbe, so use CommandLine's output directly + if (firstRow.size() >= 2) { + String executionId = UTF8Type.instance.getSerializer().deserialize(firstRow.get(0)); + String output = UTF8Type.instance.getSerializer().deserialize(firstRow.get(1)); + // NodeProbe instance is not available here, so print directly to the command output. parseResult.commandSpec().commandLine().getOut().println(output); + parseResult.commandSpec().commandLine().getOut().println(executionId); } } } return 0; } - catch (Exception e) - { + catch (RuntimeException e) { + Throwable cause = e.getCause(); + if (cause instanceof UnauthorizedException) { + throw new CommandLine.ExecutionException(parseResult.commandSpec().commandLine(), + "Unauthorized: " + cause.getMessage()); + } + else if (cause instanceof InvalidRequestException) { + throw new CommandLine.ExecutionException(parseResult.commandSpec().commandLine(), + "Invalid request: " + cause.getMessage()); + } + else if (cause instanceof CommandRequestExecutionException) { + CommandRequestExecutionException cree = (CommandRequestExecutionException) cause; + String msg = String.format("Command execution failed (executionId: %s): %s", + cree.getCommandExecutionId(), cree.getMessage()); + throw new CommandLine.ExecutionException(parseResult.commandSpec().commandLine(), msg); + } + throw new CommandLine.ExecutionException(parseResult.commandSpec().commandLine(), + "Unknown command execution exception via CQL: " + e.getMessage(), e); + } + catch (Exception e) { + // Catch-all for checked exceptions thrown during command execution. throw new CommandLine.ExecutionException(parseResult.commandSpec().commandLine(), - "Failed to execute command via CQL: " + e.getMessage(), e); + "Unknown command execution exception via CQL: " + e.getMessage(), e); } } diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java index a8f43b3d052b..884240438f15 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/ProtocolAwareExecutionStrategy.java @@ -99,6 +99,7 @@ private static void createCqlConnectFromJmxConnect() // Initialize DatabaseDescriptor before creating CqlConnect to ensure it's ready // when transport classes (like Envelope.Decoder) are loaded. This is critical // for forked nodetool processes where DatabaseDescriptor might not be initialized. + // TODO this initialization should ideally be handled in a more centralized manner or don't needed at all. if (!DatabaseDescriptor.isClientOrToolInitialized()) DatabaseDescriptor.clientInitialization(false); } diff --git a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java index 36b7f34eb38b..a0347b786262 100644 --- a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java +++ b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.UUID; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicate; @@ -37,6 +38,7 @@ import org.apache.cassandra.exceptions.CDCWriteException; import org.apache.cassandra.exceptions.CasWriteTimeoutException; import org.apache.cassandra.exceptions.CasWriteUnknownResultException; +import org.apache.cassandra.exceptions.CommandRequestExecutionException; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.exceptions.ExceptionCode; import org.apache.cassandra.exceptions.FunctionExecutionException; @@ -217,6 +219,18 @@ public ErrorMessage decode(ByteBuf body, ProtocolVersion version) int blockFor = body.readInt(); te = new CasWriteUnknownResultException(cl, received, blockFor); break; + case COMMAND_FAILED: + if (version.isSmallerThan(ProtocolVersion.V5)) + { + // Fallback for older clients - they'll get SERVER_ERROR instead + te = new ServerError(msg); + } + else + { + UUID executionId = CBUtil.readUUID(body); + te = new CommandRequestExecutionException(executionId, msg, null); + } + break; } return new ErrorMessage(te); } @@ -302,6 +316,11 @@ public void encode(ErrorMessage msg, ByteBuf dest, ProtocolVersion version) CBUtil.writeConsistencyLevel(cwue.consistency, dest); dest.writeInt(cwue.received); dest.writeInt(cwue.blockFor); + break; + case COMMAND_FAILED: + CommandRequestExecutionException cree = (CommandRequestExecutionException)err; + CBUtil.writeUUID(cree.getCommandExecutionId(), dest); + break; } } @@ -371,6 +390,10 @@ public int encodedSize(ErrorMessage msg, ProtocolVersion version) CasWriteUnknownResultException cwue = (CasWriteUnknownResultException)err; size += CBUtil.sizeOfConsistencyLevel(cwue.consistency) + 4 + 4; // receivedFor: 4, blockFor: 4 break; + case COMMAND_FAILED: + CommandRequestExecutionException cree = (CommandRequestExecutionException)err; + size += CBUtil.sizeOfUUID(cree.getCommandExecutionId()); // 16 bytes + break; } return size; } @@ -408,6 +431,13 @@ private static TransportException getBackwardsCompatibleException(ErrorMessage m case CAS_WRITE_UNKNOWN: CasWriteUnknownResultException cwue = (CasWriteUnknownResultException) msg.error; return new WriteTimeoutException(WriteType.CAS, cwue.consistency, cwue.received, cwue.blockFor); + case COMMAND_FAILED: + // For older clients, the executionId is lost, but the message should contain it. + CommandRequestExecutionException cree = (CommandRequestExecutionException) msg.error; + String msgWithId = String.format("Command execution failed (executionId: %s): %s", + cree.getCommandExecutionId(), + cree.getMessage()); + return new ServerError(msgWithId); } } diff --git a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java index 5496839c484c..50443a5bc952 100644 --- a/test/distributed/org/apache/cassandra/distributed/impl/Instance.java +++ b/test/distributed/org/apache/cassandra/distributed/impl/Instance.java @@ -118,7 +118,7 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.PathUtils; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.management.CommandService; +import org.apache.cassandra.management.CommandInvokerService; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.Sampler; import org.apache.cassandra.metrics.ThreadLocalMetrics; @@ -1024,7 +1024,7 @@ public Future shutdown(boolean runOnExitThreads, boolean shutdownMessaging SnapshotManager.instance::close, () -> IndexStatusManager.instance.shutdownAndWait(1L, MINUTES), DiskErrorsHandlerService::close, - CommandService::shutdown, + CommandInvokerService::shutdown, () -> ThreadLocalMetrics.shutdownCleaner(1L, MINUTES) ); diff --git a/test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java b/test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java index c8d695e6d6d0..300c5516bc4b 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLNodetoolProtocolTester.java @@ -50,7 +50,7 @@ public ToolRunner.ToolResult invokeNodetool(String... args) { // Use invokeNodetoolInJvm for faster execution of the command operations and // enabling easier debugging when running in debug mode from IDE. There is also - // no need to build the jars to run nodetool commands in this test. + // no need to run 'ant jars' to run nodetool commands in this test after code changes. try (WithProperties with = new WithProperties().set(CassandraRelevantProperties.CASSANDRA_CLI_EXECUTION_PROTOCOL, strategy.name().toLowerCase())) { diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 3764bc7b0eae..69a5f1c61e0d 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -172,7 +172,7 @@ import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileSystems; import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.management.CommandService; +import org.apache.cassandra.management.CommandInvokerService; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.ClientMetrics; import org.apache.cassandra.net.MessagingService; @@ -488,7 +488,7 @@ protected static void prePrepareServer() StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance); SnapshotManager.instance.registerMBean(); SYSTEM_DISTRIBUTED_DEFAULT_RF.setInt(1); - CommandService.instance.start(); + CommandInvokerService.instance.start(); } // So derived classes can get enough intialization to start setting DatabaseDescriptor options @@ -527,7 +527,7 @@ public static void tearDownClass() logger.warn("Error shutting down jmx", e); } - CommandService.instance.stop(); + CommandInvokerService.instance.stop(); } } @@ -619,7 +619,6 @@ public static List buildNodetoolCqlArgs(List args) allArgs.add("-h"); allArgs.add(nativeAddr.getHostAddress()); allArgs.addAll(args); - System.out.println(">>>> " + allArgs); return allArgs; } diff --git a/test/unit/org/apache/cassandra/management/CommandServiceTest.java b/test/unit/org/apache/cassandra/management/CommandServiceTest.java index 71e8f50b55c3..fad48d19d5c3 100644 --- a/test/unit/org/apache/cassandra/management/CommandServiceTest.java +++ b/test/unit/org/apache/cassandra/management/CommandServiceTest.java @@ -40,8 +40,8 @@ public void testCommandServiceMBeanRegistered() { try { - ObjectName serviceName = new ObjectName(CommandServiceMBean.MBEAN_NAME); - assertThat(MBeanWrapper.instance.isRegistered(serviceName)).as("CommandService MBean should be registered after start()").isTrue(); + ObjectName serviceName = new ObjectName(CommandInvokerServiceMBean.MBEAN_NAME); + assertThat(MBeanWrapper.instance.isRegistered(serviceName)).as("CommandInvokerService MBean should be registered after start()").isTrue(); } catch (Exception e) { @@ -52,7 +52,7 @@ public void testCommandServiceMBeanRegistered() @Test public void testCommandMBeansRegistered() { - CommandService service = CommandService.instance; + CommandInvokerService service = CommandInvokerService.instance; try { ObjectName pattern = new ObjectName(COMMAND_MBEAN_PATTERN); @@ -81,7 +81,7 @@ public void testCommandMBeansRegistered() @Test public void testCommandMBeanInvoke() { - CommandService service = CommandService.instance; + CommandInvokerService service = CommandInvokerService.instance; try { diff --git a/test/unit/org/apache/cassandra/tools/nodetool/CompactTest.java b/test/unit/org/apache/cassandra/tools/nodetool/CompactTest.java index 84e22c3ea516..f163e2193b95 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/CompactTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/CompactTest.java @@ -23,13 +23,11 @@ import org.junit.BeforeClass; import org.junit.Test; -import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.CQLNodetoolProtocolTester; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.Keyspace; -import static org.apache.cassandra.tools.ToolRunner.invokeNodetoolInJvm; - -public class CompactTest extends CQLTester +public class CompactTest extends CQLNodetoolProtocolTester { @BeforeClass public static void setup() throws Throwable @@ -52,7 +50,7 @@ public void keyPresent() throws Throwable flush(keyspace()); } Assertions.assertThat(cfs.getTracker().getView().liveSSTables()).hasSize(10); - invokeNodetoolInJvm("compact", "--partition", Long.toString(key), keyspace(), currentTable()).assertOnCleanExit(); + invokeNodetool("compact", "--partition", Long.toString(key), keyspace(), currentTable()).assertOnCleanExit(); // only 1 SSTable should exist Assertions.assertThat(cfs.getTracker().getView().liveSSTables()).hasSize(1); @@ -75,7 +73,7 @@ public void keyNotPresent() throws Throwable for (long keyNotFound : Arrays.asList(key - 1, key + 1)) { - invokeNodetoolInJvm("compact", "--partition", Long.toString(keyNotFound), keyspace(), currentTable()).assertOnCleanExit(); + invokeNodetool("compact", "--partition", Long.toString(keyNotFound), keyspace(), currentTable()).assertOnCleanExit(); // only 1 SSTable should exist Assertions.assertThat(cfs.getTracker().getView().liveSSTables()).hasSize(10); @@ -85,7 +83,7 @@ public void keyNotPresent() throws Throwable @Test public void tableNotFound() { - invokeNodetoolInJvm("compact", "--partition", Long.toString(42), keyspace(), "doesnotexist") + invokeNodetool("compact", "--partition", Long.toString(42), keyspace(), "doesnotexist") .asserts() .failure() .errorContains(String.format("java.lang.IllegalArgumentException: Unknown keyspace/cf pair (%s.doesnotexist)", keyspace())); @@ -96,7 +94,7 @@ public void keyWrongType() { createTable("CREATE TABLE %s (id bigint, value text, PRIMARY KEY ((id)))"); - invokeNodetoolInJvm("compact", "--partition", "this_will_not_work", keyspace(), currentTable()) + invokeNodetool("compact", "--partition", "this_will_not_work", keyspace(), currentTable()) .asserts() .failure() .errorContains(String.format("Unable to parse partition key 'this_will_not_work' for table %s.%s; Unable to make long from 'this_will_not_work'", keyspace(), currentTable())); diff --git a/test/unit/org/apache/cassandra/transport/ErrorMessageTest.java b/test/unit/org/apache/cassandra/transport/ErrorMessageTest.java index c0e3f5ed91ef..bc1d489bd1b9 100644 --- a/test/unit/org/apache/cassandra/transport/ErrorMessageTest.java +++ b/test/unit/org/apache/cassandra/transport/ErrorMessageTest.java @@ -21,6 +21,7 @@ import java.net.UnknownHostException; import java.util.HashMap; import java.util.Map; +import java.util.UUID; import org.junit.BeforeClass; import org.junit.Test; @@ -29,6 +30,7 @@ import org.apache.cassandra.db.WriteType; import org.apache.cassandra.exceptions.CasWriteTimeoutException; import org.apache.cassandra.exceptions.CasWriteUnknownResultException; +import org.apache.cassandra.exceptions.CommandRequestExecutionException; import org.apache.cassandra.exceptions.ReadFailureException; import org.apache.cassandra.exceptions.RequestFailureReason; import org.apache.cassandra.exceptions.WriteFailureException; @@ -39,6 +41,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; public class ErrorMessageTest extends EncodeAndDecodeTestBase @@ -191,6 +194,72 @@ public void testRequestFailureExceptionMakesCopy() throws UnknownHostException assertEquals(failureReasonMap1, wfe.failureReasonByEndpoint); } + @Test + public void testV5CommandFailedSerDeser() + { + UUID executionId = UUID.randomUUID(); + String errorMessage = "Command execution failed: test command"; + Throwable cause = new RuntimeException("Underlying cause"); + CommandRequestExecutionException ex = new CommandRequestExecutionException(executionId, errorMessage, cause); + + ErrorMessage deserialized = encodeThenDecode(ErrorMessage.fromException(ex), ProtocolVersion.V5); + assertTrue(deserialized.error instanceof CommandRequestExecutionException); + CommandRequestExecutionException deserializedEx = (CommandRequestExecutionException) deserialized.error; + + assertEquals(executionId, deserializedEx.getCommandExecutionId()); + assertEquals(errorMessage, deserializedEx.getMessage()); + assertNull(deserializedEx.getCause()); + } + + @Test + public void testV4CommandFailedSerDeser() + { + UUID executionId = UUID.randomUUID(); + String errorMessage = "Command execution failed: test command"; + Throwable cause = new RuntimeException("Underlying cause"); + CommandRequestExecutionException ex = new CommandRequestExecutionException(executionId, errorMessage, cause); + + ErrorMessage deserialized = encodeThenDecode(ErrorMessage.fromException(ex), ProtocolVersion.V4); + assertTrue(deserialized.error instanceof ServerError); + ServerError deserializedEx = (ServerError) deserialized.error; + + // The executionId should be included in the message for backward compatibility. + assertTrue(deserializedEx.getMessage().contains(executionId.toString())); + assertTrue(deserializedEx.getMessage().contains(errorMessage)); + } + + @Test + public void testV5CommandFailedWithoutCause() + { + UUID executionId = UUID.randomUUID(); + String errorMessage = "Command execution failed: test command"; + CommandRequestExecutionException ex = new CommandRequestExecutionException(executionId, errorMessage); + + ErrorMessage deserialized = encodeThenDecode(ErrorMessage.fromException(ex), ProtocolVersion.V5); + assertTrue(deserialized.error instanceof CommandRequestExecutionException); + CommandRequestExecutionException deserializedEx = (CommandRequestExecutionException) deserialized.error; + + assertEquals(executionId, deserializedEx.getCommandExecutionId()); + assertEquals(errorMessage, deserializedEx.getMessage()); + assertNull(deserializedEx.getCause()); + } + + @Test + public void testV3CommandFailedSerDeser() + { + UUID executionId = UUID.randomUUID(); + String errorMessage = "Command execution failed: test command"; + CommandRequestExecutionException ex = new CommandRequestExecutionException(executionId, errorMessage); + + ErrorMessage deserialized = encodeThenDecode(ErrorMessage.fromException(ex), ProtocolVersion.V3); + assertTrue(deserialized.error instanceof ServerError); + ServerError deserializedEx = (ServerError) deserialized.error; + + // The executionId should be included in the message for backward compatibility. + assertTrue(deserializedEx.getMessage().contains(executionId.toString())); + assertTrue(deserializedEx.getMessage().contains(errorMessage)); + } + protected Message.Codec getCodec() { return ErrorMessage.codec; From f6b28f7d705f15b526df8f99cd44f463dbc6fde0 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 25 Jan 2026 16:12:15 +0100 Subject: [PATCH 07/14] native management transport should use it's own executor --- conf/cassandra.yaml | 6 + conf/cassandra_latest.yaml | 6 + .../org/apache/cassandra/config/Config.java | 1 + .../cassandra/config/DatabaseDescriptor.java | 19 +- .../statements/ExecuteCommandStatement.java | 3 +- .../cassandra/management/CommandMBean.java | 4 +- .../cassandra/service/CassandraDaemon.java | 4 + .../NativeTransportManagementService.java | 62 ++--- .../service/NativeTransportService.java | 38 +-- .../NativeTransportSharedResourceManager.java | 113 -------- .../strategy/CqlCommandExecutionStrategy.java | 12 +- .../cassandra/transport/Dispatcher.java | 57 ++-- .../transport/DispatcherExecutors.java | 124 +++++++++ .../NativeTransportManagementServiceTest.java | 79 ++++++ .../service/NativeTransportServiceTest.java | 22 +- .../transport/MessageDispatcherTest.java | 4 +- .../MessageManagementDispatcherTest.java | 252 ++++++++++++++++++ 17 files changed, 602 insertions(+), 204 deletions(-) delete mode 100644 src/java/org/apache/cassandra/service/NativeTransportSharedResourceManager.java create mode 100644 src/java/org/apache/cassandra/transport/DispatcherExecutors.java create mode 100644 test/unit/org/apache/cassandra/service/NativeTransportManagementServiceTest.java create mode 100644 test/unit/org/apache/cassandra/transport/MessageManagementDispatcherTest.java diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index c23e62987ffa..f6e509451905 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1153,6 +1153,12 @@ start_native_management_transport: false # For security reasons, you should not expose this port to the internet. # Firewall it if needed. native_management_transport_port: 11211 +# The maximum threads for handling management requests are set +# separately from regular requests to allow better isolation and +# prioritization of management operations. This is important to +# ensure that management operations can proceed even under a high +# load of regular client requests. Defaults to 2. +# native_transport_management_max_threads: 2 # Uncomment to set socket buffer size for internode communication # Note that when setting this, the buffer size is limited by net.core.wmem_max diff --git a/conf/cassandra_latest.yaml b/conf/cassandra_latest.yaml index a89d7fb2a99a..dcbdaf590f49 100644 --- a/conf/cassandra_latest.yaml +++ b/conf/cassandra_latest.yaml @@ -1142,6 +1142,12 @@ start_native_management_transport: false # For security reasons, you should not expose this port to the internet. # Firewall it if needed. native_management_transport_port: 11211 +# The maximum threads for handling management requests are set +# separately from regular requests to allow better isolation and +# prioritization of management operations. This is important to +# ensure that management operations can proceed even under a high +# load of regular client requests. Defaults to 2. +# native_transport_management_max_threads: 2 # Uncomment to set socket buffer size for internode communication # Note that when setting this, the buffer size is limited by net.core.wmem_max diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index b3424e427963..c36e16970894 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -339,6 +339,7 @@ public MemtableOptions() public boolean rpc_management_interface_prefer_ipv6 = false; public boolean start_native_transport_management = false; public int native_transport_management_port = 11211; + public int native_transport_management_max_threads = 2; /** * Max size of values in SSTables, in MebiBytes. diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java index bd718e433326..728ddda6db51 100644 --- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java +++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java @@ -3678,9 +3678,12 @@ public static boolean startNativeTransportManagement() return conf.start_native_transport_management; } - /** - * This is the port used for the native management protocol to communicate with clients. - */ + @VisibleForTesting + public static void setStartNativeTransportManagement(boolean start) + { + conf.start_native_transport_management = start; + } + public static int getNativeTransportManagementPort() { return NATIVE_TRANSPORT_MANAGEMENT_PORT.getInt(conf.native_transport_management_port); @@ -3692,6 +3695,16 @@ public static void setNativeTransportPortManagement(int port) conf.native_transport_management_port = port; } + public static int getNativeTransportManagementMaxThreads() + { + return conf.native_transport_management_max_threads; + } + + public static void setNativeTransportManagementMaxThreads(int max_threads) + { + conf.native_transport_management_max_threads = max_threads; + } + public static Config.PaxosVariant getPaxosVariant() { return conf.paxos_variant; diff --git a/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java b/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java index 4e167abe3668..842ae8d8f2a3 100644 --- a/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/ExecuteCommandStatement.java @@ -34,6 +34,7 @@ import org.apache.cassandra.cql3.ResultSet; import org.apache.cassandra.cql3.terms.ArrayLiteral; import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; import org.apache.cassandra.exceptions.CommandRequestExecutionException; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.exceptions.UnauthorizedException; @@ -138,7 +139,7 @@ private static ResultSet getCommandResultSet() ColumnSpecification executionIdColumn = new ColumnSpecification("system", "command_output", new ColumnIdentifier("execution_id", true), - UTF8Type.instance); + UUIDType.instance); ColumnSpecification outputColumn = new ColumnSpecification("system", "command_output", new ColumnIdentifier("output", true), diff --git a/src/java/org/apache/cassandra/management/CommandMBean.java b/src/java/org/apache/cassandra/management/CommandMBean.java index 22c4f0db8edc..0572ed2e4f99 100644 --- a/src/java/org/apache/cassandra/management/CommandMBean.java +++ b/src/java/org/apache/cassandra/management/CommandMBean.java @@ -173,12 +173,12 @@ public Object invoke(String actionName, Object[] params, String[] signature) thr catch (CommandValidationException | MarshalException e) { logger.error("Validation error for command: {}", metadata.name(), e); - throw new IllegalArgumentException("Command validation failed: " + e.getMessage(), e.getCause()); + throw new IllegalArgumentException("Command validation failed: " + e.getMessage(), e); } catch (CommandExecutionException e) { logger.error("Error executing command: {}", metadata.name(), e); - throw new RuntimeException("Failed to execute command: " + metadata.name() + ": " + e.getMessage()); + throw new RuntimeException("Failed to execute command: " + metadata.name(), e); } } diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index 7a05c9f94534..9acc82ca6382 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -96,6 +96,7 @@ import org.apache.cassandra.tcm.Startup; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.membership.NodeState; +import org.apache.cassandra.transport.DispatcherExecutors; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JMXServerUtils; import org.apache.cassandra.utils.JVMStabilityInspector; @@ -765,6 +766,9 @@ public void destroyClientTransports() stopNativeTransport(); if (nativeTransportService != null) nativeTransportService.destroy(); + if (nativeTransportManagementService != null) + nativeTransportManagementService.destroy(); + DispatcherExecutors.instance().shutdown(); } /** diff --git a/src/java/org/apache/cassandra/service/NativeTransportManagementService.java b/src/java/org/apache/cassandra/service/NativeTransportManagementService.java index 61092b9151c6..01758b526799 100644 --- a/src/java/org/apache/cassandra/service/NativeTransportManagementService.java +++ b/src/java/org/apache/cassandra/service/NativeTransportManagementService.java @@ -19,6 +19,7 @@ package org.apache.cassandra.service; import java.net.InetAddress; +import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import com.google.common.annotations.VisibleForTesting; @@ -32,31 +33,25 @@ import org.apache.cassandra.transport.Server; import io.netty.channel.EventLoopGroup; +import io.netty.channel.epoll.EpollEventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.Version; +import static org.apache.cassandra.service.NativeTransportService.useEpoll; + /** * Manages the native management transport server on port {@code 11211}. * This service is independent of NativeTransportService and can only be * enabled/disabled via configuration (not at runtime). */ -public class NativeTransportManagementService +public class NativeTransportManagementService implements CassandraDaemon.Server { private static final Logger logger = LoggerFactory.getLogger(NativeTransportManagementService.class); private Server server = null; - private boolean initialized = false; - private final NativeTransportSharedResourceManager resourceManager; + private EventLoopGroup workerGroup; - public NativeTransportManagementService() - { - this(NativeTransportSharedResourceManager.instance()); - } - - @VisibleForTesting - NativeTransportManagementService(NativeTransportSharedResourceManager resourceManager) - { - this.resourceManager = resourceManager; - } + private volatile boolean initialized = false; @VisibleForTesting synchronized void initialize() @@ -64,13 +59,19 @@ synchronized void initialize() if (initialized) return; - if (!DatabaseDescriptor.startNativeTransportManagement()) + if (useEpoll()) { - logger.debug("Management transport is disabled via configuration"); - return; + workerGroup = new EpollEventLoopGroup(); + logger.info("Management transport Netty using native Epoll event loop"); + } + else + { + workerGroup = new NioEventLoopGroup(); + logger.info("Management transport Netty using Java NIO event loop"); } - EventLoopGroup workerGroup = resourceManager.getOrCreateSharedWorkerGroup(); + logger.info("Management transport Netty Version: {}", Version.identify().entrySet()); + int managementPort = DatabaseDescriptor.getNativeTransportManagementPort(); InetAddress addr = DatabaseDescriptor.getRpcManagementAddress(); @@ -85,7 +86,6 @@ synchronized void initialize() .withManagementConnectionFlag(true) .build(); - resourceManager.registerActiveService(); initialized = true; } @@ -93,29 +93,19 @@ public void start() { if (!DatabaseDescriptor.startNativeTransportManagement()) { - logger.info("Management transport is disabled via configuration"); + logger.info("Management transport is disabled via configuration and will not start."); return; } - logger.info("Using Netty Version: {}", Version.identify().entrySet()); initialize(); - - if (server != null) - { - server.start(); - logger.info("Management transport started on port {}", DatabaseDescriptor.getNativeTransportManagementPort()); - } + server.start(); + logger.info("Management transport started on port: {}", server.socket.getPort()); } public void stop() - { - stop(false); - } - - public void stop(boolean force) { if (server != null) - server.stop(force); + server.stop(false); } public void destroy() @@ -125,7 +115,9 @@ public void destroy() return; server = null; - resourceManager.unregisterActiveService(); + if (workerGroup != null) + workerGroup.shutdownGracefully(3, 5, TimeUnit.SECONDS).awaitUninterruptibly(); + initialized = false; } @@ -150,8 +142,8 @@ public void disconnect(Predicate userPredicate) } @VisibleForTesting - Server getServer() + public EventLoopGroup getWorkerGroup() { - return server; + return workerGroup; } } diff --git a/src/java/org/apache/cassandra/service/NativeTransportService.java b/src/java/org/apache/cassandra/service/NativeTransportService.java index 92d2dc4db40f..c23917478e49 100644 --- a/src/java/org/apache/cassandra/service/NativeTransportService.java +++ b/src/java/org/apache/cassandra/service/NativeTransportService.java @@ -18,6 +18,7 @@ package org.apache.cassandra.service; import java.net.InetAddress; +import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import com.google.common.annotations.VisibleForTesting; @@ -34,6 +35,8 @@ import io.netty.channel.EventLoopGroup; import io.netty.channel.epoll.Epoll; +import io.netty.channel.epoll.EpollEventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.Version; import static org.apache.cassandra.config.CassandraRelevantProperties.NATIVE_EPOLL_ENABLED; @@ -41,23 +44,15 @@ /** * Handles native transport server lifecycle and associated resources. Lazily initialized. */ -public class NativeTransportService +public class NativeTransportService implements CassandraDaemon.Server { private static final Logger logger = LoggerFactory.getLogger(NativeTransportService.class); private Server server = null; - private boolean initialized = false; - private final NativeTransportSharedResourceManager sharedResourceManager; - - public NativeTransportService() { - this(NativeTransportSharedResourceManager.instance()); - } - @VisibleForTesting - NativeTransportService(NativeTransportSharedResourceManager sharedResourceManager) { - this.sharedResourceManager = sharedResourceManager; - } + private boolean initialized = false; + private EventLoopGroup workerGroup; /** * Creates netty thread pools and event loops. @@ -68,7 +63,17 @@ synchronized void initialize() if (initialized) return; - EventLoopGroup workerGroup = sharedResourceManager.getOrCreateSharedWorkerGroup(); + if (useEpoll()) + { + workerGroup = new EpollEventLoopGroup(); + logger.info("Netty using native Epoll event loop"); + } + else + { + workerGroup = new NioEventLoopGroup(); + logger.info("Netty using Java NIO event loop"); + } + int nativePort = DatabaseDescriptor.getNativeTransportPort(); InetAddress nativeAddr = DatabaseDescriptor.getRpcAddress(); @@ -81,7 +86,6 @@ synchronized void initialize() ClientMetrics.instance.init(server); - sharedResourceManager.registerActiveService(); initialized = true; } @@ -117,8 +121,10 @@ public void destroy() stop(); ClientMetrics.instance.release(); server = null; - sharedResourceManager.unregisterActiveService(); - initialized = false; + + // shutdown executors used by netty for native transport server + if (workerGroup != null) + workerGroup.shutdownGracefully(3, 5, TimeUnit.SECONDS).awaitUninterruptibly(); } /** @@ -145,7 +151,7 @@ public boolean isRunning() @VisibleForTesting EventLoopGroup getWorkerGroup() { - return sharedResourceManager.getSharedWorkerGroup(); + return workerGroup; } @VisibleForTesting diff --git a/src/java/org/apache/cassandra/service/NativeTransportSharedResourceManager.java b/src/java/org/apache/cassandra/service/NativeTransportSharedResourceManager.java deleted file mode 100644 index 855e55d06b3e..000000000000 --- a/src/java/org/apache/cassandra/service/NativeTransportSharedResourceManager.java +++ /dev/null @@ -1,113 +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 org.apache.cassandra.service; - -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; - -import com.google.common.annotations.VisibleForTesting; - -import org.apache.cassandra.transport.Dispatcher; - -import io.netty.channel.EventLoopGroup; -import io.netty.channel.epoll.EpollEventLoopGroup; -import io.netty.channel.nio.NioEventLoopGroup; - -/** - * Manages shared resources (EventLoopGroup, Dispatcher) for native transport services. - * This ensures both NativeTransportService and NativeTransportManagementService can share - * the same EventLoopGroup and properly coordinate Dispatcher shutdown. - */ -public class NativeTransportSharedResourceManager -{ - private static final NativeTransportSharedResourceManager instance = new NativeTransportSharedResourceManager(); - - private final AtomicReference sharedWorkerGroup = new AtomicReference<>(); - private final AtomicInteger activeServicesCount = new AtomicInteger(0); - private final Object mutex = new Object(); - - public static NativeTransportSharedResourceManager instance() - { - return instance; - } - - public EventLoopGroup getOrCreateSharedWorkerGroup() - { - EventLoopGroup group = sharedWorkerGroup.get(); - if (group == null) - { - synchronized (mutex) - { - group = sharedWorkerGroup.get(); - if (group == null) - { - if (NativeTransportService.useEpoll()) - group = new EpollEventLoopGroup(); - else - group = new NioEventLoopGroup(); - sharedWorkerGroup.set(group); - } - } - } - return group; - } - - public void registerActiveService() - { - activeServicesCount.incrementAndGet(); - } - - public void unregisterActiveService() - { - int remaining = activeServicesCount.decrementAndGet(); - if (remaining == 0) - shutdownSharedResources(); - } - - private void shutdownSharedResources() - { - synchronized (mutex) - { - EventLoopGroup group = sharedWorkerGroup.getAndSet(null); - if (group != null) - group.shutdownGracefully(3, 5, TimeUnit.SECONDS).awaitUninterruptibly(); - - // Dispatcher.shutdown() shuts down static executors shared by all Server instances - // Only call this when ALL native transport services are shutting down - Dispatcher.shutdown(); - } - } - - @VisibleForTesting - public EventLoopGroup getSharedWorkerGroup() - { - return sharedWorkerGroup.get(); - } - - @VisibleForTesting - public void forceShutdown() - { - synchronized (mutex) - { - shutdownSharedResources(); - activeServicesCount.set(0); - } - } -} diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java index c22982b11c9d..ac785ef84b62 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java @@ -24,11 +24,13 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import org.apache.cassandra.cql3.ColumnIdentifier; import org.apache.cassandra.cql3.CqlBuilder; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; import org.apache.cassandra.exceptions.CommandRequestExecutionException; import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.exceptions.UnauthorizedException; @@ -87,16 +89,18 @@ public int execute(CommandLine.ParseResult parseResult) throws CommandLine.Execu if (result instanceof ResultMessage.Rows) { ResultMessage.Rows rows = (ResultMessage.Rows) result; - assert rows.result.size() == 2 : "Command execution result schema has been changed. " + - "Expected 2 columns in result, got " + rows.result.size(); + assert rows.result.size() == 1 : "Command execution result should have exactly 1 row, got " + rows.result.size(); + assert rows.result.metadata.getColumnCount() == 2 : "Command execution result schema has been changed. " + + "Expected 2 columns in result, got " + rows.result.metadata.getColumnCount(); + if (!rows.result.isEmpty() && rows.result.metadata.names.size() >= 2) { List firstRow = rows.result.rows.get(0); if (firstRow.size() >= 2) { - String executionId = UTF8Type.instance.getSerializer().deserialize(firstRow.get(0)); + UUID executionId = UUIDType.instance.getSerializer().deserialize(firstRow.get(0)); String output = UTF8Type.instance.getSerializer().deserialize(firstRow.get(1)); // NodeProbe instance is not available here, so print directly to the command output. parseResult.commandSpec().commandLine().getOut().println(output); - parseResult.commandSpec().commandLine().getOut().println(executionId); + parseResult.commandSpec().commandLine().getOut().println(executionId.toString()); } } } diff --git a/src/java/org/apache/cassandra/transport/Dispatcher.java b/src/java/org/apache/cassandra/transport/Dispatcher.java index ccec6817c388..810e835b66d4 100644 --- a/src/java/org/apache/cassandra/transport/Dispatcher.java +++ b/src/java/org/apache/cassandra/transport/Dispatcher.java @@ -23,7 +23,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.Consumer; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; @@ -51,17 +50,11 @@ import io.netty.channel.EventLoop; import io.netty.util.AttributeKey; -import static org.apache.cassandra.concurrent.SharedExecutorPool.SHARED; - public class Dispatcher implements CQLMessageHandler.MessageConsumer { private static final Logger logger = LoggerFactory.getLogger(Dispatcher.class); - @VisibleForTesting - static final LocalAwareExecutorPlus requestExecutor = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(), - DatabaseDescriptor::setNativeTransportMaxThreads, - "transport", - "Native-Transport-Requests"); + private final LocalAwareExecutorPlus requestExecutor; /** CASSANDRA-17812: Rate-limit new client connection setup to avoid overwhelming during bcrypt * @@ -76,11 +69,26 @@ public class Dispatcher implements CQLMessageHandler.MessageConsumerManagement connections are identified via Connection's flag set by the management + * transport server at initial connection setup. Request then are routed to a dedicated + * executor instead of the standard {@link #requestExecutor}. This provides isolation and + * prioritization of management operations, ensuring they can proceed even under + * a high load of regular client requests. + * + *

    The executor is configured separately via + * {@link DatabaseDescriptor#getNativeTransportManagementMaxThreads()} to allow + * independent tuning of management operation throughput. + * + *

    Management connections are established through the management transport server + * (see {@link org.apache.cassandra.service.NativeTransportManagementService}), which listens + * on a separate port from the regular native transport. + */ + private final LocalAwareExecutorPlus managementExecutor; private static final ConcurrentMap flusherLookup = new ConcurrentHashMap<>(); private final boolean useLegacyFlusher; @@ -101,6 +109,9 @@ interface FlushItemConverter public Dispatcher(boolean useLegacyFlusher) { this.useLegacyFlusher = useLegacyFlusher; + this.requestExecutor = DispatcherExecutors.instance().getRequestExecutor(); + this.authExecutor = DispatcherExecutors.instance().getAuthExecutor(); + this.managementExecutor = DispatcherExecutors.instance().getManagementExecutor(); } @Override @@ -120,6 +131,20 @@ public void dispatch(Channel channel, Message.Request request, FlushItemConverte return; } + // Use connection object to check for management connections, this could be faster than checking + // channel attributies directly every time. For management connections, we route requests to + // the management executor. + Connection connection = request.connection(); + if (connection instanceof ServerConnection) + { + ServerConnection serverConnection = (ServerConnection) connection; + if (serverConnection.isManagementConnection()) + { + managementExecutor.submit(new RequestProcessor(channel, request, forFlusher, backpressure)); + return; + } + } + // if native_transport_max_auth_threads is < 1, don't delegate to new pool on auth messages boolean isAuthQuery = DatabaseDescriptor.getNativeTransportMaxAuthThreads() > 0 && (request.type == Message.Type.AUTH_RESPONSE || request.type == Message.Type.CREDENTIALS); @@ -495,12 +520,6 @@ public boolean isDone() return requestExecutor.getPendingTaskCount() == 0 && requestExecutor.getActiveTaskCount() == 0; } - public static void shutdown() - { - requestExecutor.shutdown(); - authExecutor.shutdown(); - } - /** * Dispatcher for EventMessages. In {@link Server.ConnectionTracker#send(Event)}, the strategy * for delivering events to registered clients is dependent on protocol version and the configuration diff --git a/src/java/org/apache/cassandra/transport/DispatcherExecutors.java b/src/java/org/apache/cassandra/transport/DispatcherExecutors.java new file mode 100644 index 000000000000..7fbbbc33339e --- /dev/null +++ b/src/java/org/apache/cassandra/transport/DispatcherExecutors.java @@ -0,0 +1,124 @@ +/* + * 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.cassandra.transport; + +import org.apache.cassandra.concurrent.LocalAwareExecutorPlus; +import org.apache.cassandra.config.DatabaseDescriptor; + +import static org.apache.cassandra.concurrent.SharedExecutorPool.SHARED; + +/** + * Manages the lifecycle of executors used by the native transport dispatcher. + * These executors are shared across all Server instances and tied to the cassandra daemon lifecycle. + */ +public class DispatcherExecutors +{ + private static final DispatcherExecutors instance = new DispatcherExecutors(); + + private LocalAwareExecutorPlus requestExecutor; + private LocalAwareExecutorPlus authExecutor; + private LocalAwareExecutorPlus managementExecutor; + + private final Object mutex = new Object(); + private volatile boolean initialized = false; + + public static DispatcherExecutors instance() + { + return instance; + } + + /** Ensures all executors are initialized. All executors are initialized together. */ + private void initialize() + { + if (initialized) + return; + + synchronized (mutex) + { + if (initialized) + return; + + if (requestExecutor == null) + { + requestExecutor = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(), + DatabaseDescriptor::setNativeTransportMaxThreads, + "transport", + "Native-Transport-Requests"); + } + if (authExecutor == null) + { + authExecutor = SHARED.newExecutor(Math.max(1, DatabaseDescriptor.getNativeTransportMaxAuthThreads()), + DatabaseDescriptor::setNativeTransportMaxAuthThreads, + "transport", + "Native-Transport-Auth-Requests"); + } + if (managementExecutor == null) + { + managementExecutor = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportManagementMaxThreads(), + DatabaseDescriptor::setNativeTransportManagementMaxThreads, + "transport", + "Native-Transport-Management-Tasks"); + } + + initialized = true; + } + } + + public LocalAwareExecutorPlus getRequestExecutor() + { + initialize(); + return requestExecutor; + } + + public LocalAwareExecutorPlus getAuthExecutor() + { + initialize(); + return authExecutor; + } + + public LocalAwareExecutorPlus getManagementExecutor() + { + initialize(); + return managementExecutor; + } + + /** Shuts down node-level executors. Should only be called during node shutdown. */ + public void shutdown() + { + synchronized (mutex) + { + if (requestExecutor != null) + { + requestExecutor.shutdown(); + requestExecutor = null; + } + if (authExecutor != null) + { + authExecutor.shutdown(); + authExecutor = null; + } + if (managementExecutor != null) + { + managementExecutor.shutdown(); + managementExecutor = null; + } + initialized = false; + } + } +} diff --git a/test/unit/org/apache/cassandra/service/NativeTransportManagementServiceTest.java b/test/unit/org/apache/cassandra/service/NativeTransportManagementServiceTest.java new file mode 100644 index 000000000000..60fb947c93d9 --- /dev/null +++ b/test/unit/org/apache/cassandra/service/NativeTransportManagementServiceTest.java @@ -0,0 +1,79 @@ +/* + * 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.cassandra.service; + +import java.util.function.BooleanSupplier; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.cassandra.config.DatabaseDescriptor; + +import io.netty.channel.EventLoopGroup; + +import static org.apache.cassandra.service.NativeTransportServiceTest.withService; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class NativeTransportManagementServiceTest +{ + @BeforeClass + public static void setupTransport() + { + DatabaseDescriptor.daemonInitialization(); + DatabaseDescriptor.setStartNativeTransportManagement(true); + } + + @AfterClass + public static void cleanupManagementConfig() + { + DatabaseDescriptor.setStartNativeTransportManagement(false); + } + + @Test + public void testStart() + { + withService((CassandraDaemon.Server service) -> assertTrue(service.isRunning()), + NativeTransportManagementService::new, true, 1); + } + + @Test + public void testDestroy() + { + withService((CassandraDaemon.Server srv) -> { + NativeTransportManagementService service = (NativeTransportManagementService) srv; + EventLoopGroup workerGroup = service.getWorkerGroup(); + BooleanSupplier allTerminated = () -> workerGroup != null + && workerGroup.isShutdown() + && workerGroup.isTerminated(); + + assertFalse(allTerminated.getAsBoolean()); + service.destroy(); + assertTrue(allTerminated.getAsBoolean()); + }, NativeTransportManagementService::new, true, 1); + } + + @Test + public void testConcurrentDestroys() + { + withService(srv -> ((NativeTransportManagementService) srv).destroy(), + NativeTransportManagementService::new, true, 20); + } +} diff --git a/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java b/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java index 16bcf659c73f..1f8deebe5635 100644 --- a/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java +++ b/test/unit/org/apache/cassandra/service/NativeTransportServiceTest.java @@ -19,19 +19,17 @@ import java.util.function.BooleanSupplier; import java.util.function.Consumer; +import java.util.function.Supplier; import java.util.stream.IntStream; import org.junit.After; import org.junit.BeforeClass; import org.junit.Test; -import org.apache.cassandra.Util; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.EncryptionOptions; import org.apache.cassandra.transport.Server; -import io.netty.channel.EventLoopGroup; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -52,7 +50,6 @@ public static void setupDD() public void resetConfig() { DatabaseDescriptor.updateNativeProtocolEncryptionOptions(update -> new EncryptionOptions.ClientEncryptionOptions.Builder(defaultOptions).build()); - NativeTransportSharedResourceManager.instance().forceShutdown(); } @Test @@ -87,14 +84,12 @@ public void testIgnoresStoppedOnAlreadyStopped() @Test public void testDestroy() { - NativeTransportSharedResourceManager mgr = NativeTransportSharedResourceManager.instance(); - EventLoopGroup eventLoopGroup = mgr.getOrCreateSharedWorkerGroup(); withService((NativeTransportService service) -> { - BooleanSupplier allTerminated = () -> eventLoopGroup.isShutdown() && eventLoopGroup.isTerminated(); + BooleanSupplier allTerminated = () -> + service.getWorkerGroup().isShutdown() && service.getWorkerGroup().isTerminated(); assertFalse(allTerminated.getAsBoolean()); service.destroy(); assertTrue(allTerminated.getAsBoolean()); - Util.spinUntilTrue(() -> mgr.getSharedWorkerGroup() == null); }); } @@ -176,7 +171,16 @@ private static void withService(Consumer f) private static void withService(Consumer f, boolean start, int concurrently) { - NativeTransportService service = new NativeTransportService(); + withService(srv -> f.accept((NativeTransportService) srv), + NativeTransportService::new, start, concurrently); + } + + static void withService(Consumer f, + Supplier provider, + boolean start, + int concurrently) + { + CassandraDaemon.Server service = provider.get(); assertFalse(service.isRunning()); if (start) { diff --git a/test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java b/test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java index 601a1b1cdbc4..ec2b51e01cf8 100644 --- a/test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java +++ b/test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java @@ -140,12 +140,12 @@ public void testAuthRateLimiterDisabled() throws Exception private long completedRequests() { - return Dispatcher.requestExecutor.getCompletedTaskCount(); + return DispatcherExecutors.instance().getRequestExecutor().getCompletedTaskCount(); } private long completedAuth() { - return Dispatcher.authExecutor.getCompletedTaskCount(); + return DispatcherExecutors.instance().getAuthExecutor().getCompletedTaskCount(); } public long tryAuth(Callable check) throws Exception diff --git a/test/unit/org/apache/cassandra/transport/MessageManagementDispatcherTest.java b/test/unit/org/apache/cassandra/transport/MessageManagementDispatcherTest.java new file mode 100644 index 000000000000..616a5395b3fe --- /dev/null +++ b/test/unit/org/apache/cassandra/transport/MessageManagementDispatcherTest.java @@ -0,0 +1,252 @@ +/* + * 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.cassandra.transport; + +import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; + +import com.google.common.util.concurrent.Uninterruptibles; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.Mockito; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.metrics.ClientMetrics; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.utils.Clock; + +import io.netty.channel.Channel; + +import static java.lang.String.format; +import static org.junit.Assert.assertEquals; + +public class MessageManagementDispatcherTest +{ + private static ManagementTestDispatcher dispatch; + private static int maxManagementThreadsBeforeTests; + + @BeforeClass + public static void init() throws Exception + { + DatabaseDescriptor.daemonInitialization(); + ClientMetrics.instance.init(null); + maxManagementThreadsBeforeTests = DatabaseDescriptor.getNativeTransportManagementMaxThreads(); + dispatch = new ManagementTestDispatcher(); + } + + @AfterClass + public static void restoreManagementSize() + { + DatabaseDescriptor.setNativeTransportManagementMaxThreads(maxManagementThreadsBeforeTests); + } + + @Test + public void testManagementExecutorRouting() throws Exception + { + long startRequests = completedRequests(); + long startAuth = completedAuth(); + + DatabaseDescriptor.setNativeTransportManagementMaxThreads(1); + Message.Request managementRequest = createManagementRequest(managementConnectionMock(), Message.Type.QUERY); + + long managementTasks = tryRequest(this::completedManagement, managementRequest); + assertEquals("Management request should be routed to management executor", + 1, managementTasks); + assertEquals("No auth requests should be processed", startAuth, completedAuth()); + assertEquals("Regular requests should not increase", startRequests, completedRequests()); + } + + @Test + public void testManagementExecutorIsolation() throws Exception + { + long startManagement = completedManagement(); + + DatabaseDescriptor.setNativeTransportManagementMaxThreads(1); + + // Test that regular (non-management) requests don't use management executor + for (Message.Type type : Message.Type.values()) + { + if (type.direction != Message.Direction.REQUEST) + continue; + + long requests = tryRequest(() -> Message.Type.CREDENTIALS == type || Message.Type.AUTH_RESPONSE == type + ? completedAuth() + : completedRequests(), + createRegularRequest(type)); + + assertEquals("No management tasks should be processed", startManagement, completedManagement()); + assertEquals(format("Request should be processed for type: %s", type), 1, requests); + } + } + + @Test + public void testManagementConnectionAllMessageTypes() throws Exception + { + DatabaseDescriptor.setNativeTransportManagementMaxThreads(1); + + // Test that all request types from management connections go to the management executor. + for (Message.Type type : Message.Type.values()) + { + if (type.direction != Message.Direction.REQUEST) + continue; + + Message.Request request = createManagementRequest(managementConnectionMock(), type); + long managementTasks = tryRequest(this::completedManagement, request); + assertEquals(format("Management %s request should route to management executor", type), + 1, managementTasks); + } + } + + @Test + public void testNonServerConnectionNotRoutedToManagement() throws Exception + { + DatabaseDescriptor.setNativeTransportManagementMaxThreads(1); + + // Create a connection that is not a ServerConnection + Connection nonServerConnection = connectionMock(); + + Message.Request request = new Message.Request(Message.Type.QUERY) + { + @Override + public Connection connection() + { + return nonServerConnection; + } + + @Override + public Response execute(QueryState queryState, Dispatcher.RequestTime requestTime, boolean traceRequest) + { + return null; + } + }; + + long startManagement = completedManagement(); + long regularTasks = tryRequest(this::completedRequests, request); + + assertEquals("Non-server connection should use regular executor", 1, regularTasks); + assertEquals("Non-server connection should not use management executor", + startManagement, completedManagement()); + } + + private long completedRequests() + { + return DispatcherExecutors.instance().getRequestExecutor().getCompletedTaskCount(); + } + + private long completedAuth() + { + return DispatcherExecutors.instance().getAuthExecutor().getCompletedTaskCount(); + } + + private long completedManagement() + { + return DispatcherExecutors.instance().getManagementExecutor().getCompletedTaskCount(); + } + + private long tryRequest(Callable check, Message.Request request) throws Exception + { + long start = check.call(); + dispatch.dispatch(null, request, (channel, req, response) -> null, ClientResourceLimits.Overload.NONE); + + long timeout = Clock.Global.currentTimeMillis(); + while (start == check.call() && Clock.Global.currentTimeMillis() - timeout < 1000) + Uninterruptibles.sleepUninterruptibly(10, TimeUnit.MILLISECONDS); + return check.call() - start; + } + + private static ServerConnection managementConnectionMock() + { + Connection.Tracker tracker = Mockito.mock(Connection.Tracker.class); + Mockito.when(tracker.isRunning()).thenAnswer(invocation -> true); + + Channel channel = Mockito.mock(Channel.class); + ServerConnection connection = Mockito.mock(ServerConnection.class); + Mockito.when(connection.getTracker()).thenAnswer(invocation -> tracker); + Mockito.when(connection.isManagementConnection()).thenReturn(true); + Mockito.when(connection.getVersion()).thenReturn(ProtocolVersion.CURRENT); + Mockito.when(connection.channel()).thenReturn(channel); + + return connection; + } + + private static Connection connectionMock() + { + Connection.Tracker tracker = Mockito.mock(Connection.Tracker.class); + Mockito.when(tracker.isRunning()).thenAnswer(invocation -> true); + Connection c = Mockito.mock(Connection.class); + Mockito.when(c.getTracker()).thenAnswer(invocation -> tracker); + return c; + } + + private Message.Request createManagementRequest(ServerConnection conn, Message.Type type) + { + return new Message.Request(type) + { + @Override + public Connection connection() + { + return conn; + } + + @Override + public Response execute(QueryState queryState, Dispatcher.RequestTime requestTime, boolean traceRequest) + { + return null; + } + }; + } + + private Message.Request createRegularRequest(Message.Type type) + { + return new Message.Request(type) + { + @Override + public Connection connection() + { + return connectionMock(); + } + + @Override + public Response execute(QueryState queryState, Dispatcher.RequestTime requestTime, boolean traceRequest) + { + return null; + } + }; + } + + public static class ManagementTestDispatcher extends Dispatcher + { + public ManagementTestDispatcher() + { + super(false); + } + + @Override + void processRequest(Channel channel, + Message.Request request, + FlushItemConverter forFlusher, + ClientResourceLimits.Overload backpressure, + RequestTime requestTime) + { + // noop - just for testing routing + } + } +} \ No newline at end of file From 38d0d5a2c141fb02be710c79a92096b1daa63dc6 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 25 Jan 2026 17:17:02 +0100 Subject: [PATCH 08/14] fix protocol doc, code cleanup --- doc/native_protocol_v5.spec | 9 +++++++++ .../apache/cassandra/transport/PipelineConfigurator.java | 5 ----- .../apache/cassandra/management/CommandServiceTest.java | 1 - 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/doc/native_protocol_v5.spec b/doc/native_protocol_v5.spec index 88d6a948a709..6881bfe187b2 100644 --- a/doc/native_protocol_v5.spec +++ b/doc/native_protocol_v5.spec @@ -1419,6 +1419,15 @@ Table of Contents acknowledged the request. is an [int] representing the number of replicas whose acknowledgement is required to achieve . + 0x1800 COMMAND_FAILED: An exception occurred during command execution. This error is returned + when a command executed via the native management interface fails during execution. + The exception may or may not include more detail in the accompanying error message. + The rest of the ERROR message body will be + + where: + is a [uuid] representing the unique identifier for the command + execution that failed. This identifier can be used to correlate the + error with command execution logs on the server. 0x2000 Syntax_error: The submitted query has a syntax error. 0x2100 Unauthorized: The logged user doesn't have the right to perform diff --git a/src/java/org/apache/cassandra/transport/PipelineConfigurator.java b/src/java/org/apache/cassandra/transport/PipelineConfigurator.java index 6e86f61c5b11..4c5faf3c914e 100644 --- a/src/java/org/apache/cassandra/transport/PipelineConfigurator.java +++ b/src/java/org/apache/cassandra/transport/PipelineConfigurator.java @@ -376,11 +376,6 @@ protected ClientResourceLimits.ResourceProvider resourceProvider(ClientResourceL return new ClientResourceLimits.ResourceProvider.Default(allocator); } - protected Dispatcher dispatcher(boolean useLegacyFlusher) - { - return new Dispatcher(useLegacyFlusher); - } - protected CQLMessageHandler.MessageConsumer messageConsumer() { return dispatcher; diff --git a/test/unit/org/apache/cassandra/management/CommandServiceTest.java b/test/unit/org/apache/cassandra/management/CommandServiceTest.java index fad48d19d5c3..22d512200ca8 100644 --- a/test/unit/org/apache/cassandra/management/CommandServiceTest.java +++ b/test/unit/org/apache/cassandra/management/CommandServiceTest.java @@ -93,7 +93,6 @@ public void testCommandMBeanInvoke() MBeanServer mbs = MBeanWrapper.instance.getMBeanServer(); String schema = (String) mbs.invoke(commandObjectName, "getJsonSchema", null, null); - System.out.println(">>> JSON Schema for command " + testCommand + ":\n" + schema); assertThat(schema).as("getJsonSchema() should return non-null JSON string").isNotNull().isNotEmpty(); assertThat(schema.trim()).as("Schema should start with '{'").startsWith("{"); From 36597473e7f1156fe02824c6c9036f339e8b4f85 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 25 Jan 2026 20:41:46 +0100 Subject: [PATCH 09/14] restric operations on management port except command execution --- .../cassandra/service/CassandraDaemon.java | 4 +- .../cassandra/transport/Dispatcher.java | 143 +++++++++++++++--- .../transport/DispatcherExecutors.java | 124 --------------- .../management/CqlExecuteCommandTest.java | 13 +- .../transport/MessageDispatcherTest.java | 4 +- .../MessageManagementDispatcherTest.java | 11 +- 6 files changed, 137 insertions(+), 162 deletions(-) delete mode 100644 src/java/org/apache/cassandra/transport/DispatcherExecutors.java diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index 9acc82ca6382..7f85ef518409 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -96,7 +96,7 @@ import org.apache.cassandra.tcm.Startup; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.membership.NodeState; -import org.apache.cassandra.transport.DispatcherExecutors; +import org.apache.cassandra.transport.Dispatcher; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JMXServerUtils; import org.apache.cassandra.utils.JVMStabilityInspector; @@ -768,7 +768,7 @@ public void destroyClientTransports() nativeTransportService.destroy(); if (nativeTransportManagementService != null) nativeTransportManagementService.destroy(); - DispatcherExecutors.instance().shutdown(); + Dispatcher.shutdown(); } /** diff --git a/src/java/org/apache/cassandra/transport/Dispatcher.java b/src/java/org/apache/cassandra/transport/Dispatcher.java index 810e835b66d4..4e8362b381fe 100644 --- a/src/java/org/apache/cassandra/transport/Dispatcher.java +++ b/src/java/org/apache/cassandra/transport/Dispatcher.java @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.Consumer; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; @@ -32,6 +33,10 @@ import org.apache.cassandra.concurrent.DebuggableTask; import org.apache.cassandra.concurrent.LocalAwareExecutorPlus; import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.CQLStatement; +import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.cql3.statements.ExecuteCommandStatement; +import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.exceptions.OverloadedException; import org.apache.cassandra.metrics.ClientMetrics; import org.apache.cassandra.net.FrameEncoder; @@ -42,6 +47,7 @@ import org.apache.cassandra.transport.Flusher.FlushItem; import org.apache.cassandra.transport.messages.ErrorMessage; import org.apache.cassandra.transport.messages.EventMessage; +import org.apache.cassandra.transport.messages.QueryMessage; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.MonotonicClock; import org.apache.cassandra.utils.NoSpamLogger; @@ -50,11 +56,21 @@ import io.netty.channel.EventLoop; import io.netty.util.AttributeKey; +import static org.apache.cassandra.concurrent.SharedExecutorPool.SHARED; + +/** + * TODO: a few disributed tests must be added in order to verify that the Dispatcher behaves correctly, + * when management connections are used (CEP-38). + */ public class Dispatcher implements CQLMessageHandler.MessageConsumer { private static final Logger logger = LoggerFactory.getLogger(Dispatcher.class); - private final LocalAwareExecutorPlus requestExecutor; + @VisibleForTesting + static final LocalAwareExecutorPlus requestExecutor = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(), + DatabaseDescriptor::setNativeTransportMaxThreads, + "transport", + "Native-Transport-Requests"); /** CASSANDRA-17812: Rate-limit new client connection setup to avoid overwhelming during bcrypt * @@ -69,7 +85,11 @@ public class Dispatcher implements CQLMessageHandler.MessageConsumer flusherLookup = new ConcurrentHashMap<>(); private final boolean useLegacyFlusher; @@ -109,9 +133,6 @@ interface FlushItemConverter public Dispatcher(boolean useLegacyFlusher) { this.useLegacyFlusher = useLegacyFlusher; - this.requestExecutor = DispatcherExecutors.instance().getRequestExecutor(); - this.authExecutor = DispatcherExecutors.instance().getAuthExecutor(); - this.managementExecutor = DispatcherExecutors.instance().getManagementExecutor(); } @Override @@ -131,6 +152,18 @@ public void dispatch(Channel channel, Message.Request request, FlushItemConverte return; } + // if native_transport_max_auth_threads is < 1, don't delegate to new pool on auth messages + boolean isAuthQuery = DatabaseDescriptor.getNativeTransportMaxAuthThreads() > 0 && + (request.type == Message.Type.AUTH_RESPONSE || request.type == Message.Type.CREDENTIALS); + + if (isAuthQuery) + { + // Importantly, the authExecutor will handle the AUTHENTICATE message which may be CPU intensive. + authExecutor.submit(new RequestProcessor(channel, request, forFlusher, backpressure)); + ClientMetrics.instance.markRequestDispatched(); + return; + } + // Use connection object to check for management connections, this could be faster than checking // channel attributies directly every time. For management connections, we route requests to // the management executor. @@ -140,19 +173,12 @@ public void dispatch(Channel channel, Message.Request request, FlushItemConverte ServerConnection serverConnection = (ServerConnection) connection; if (serverConnection.isManagementConnection()) { - managementExecutor.submit(new RequestProcessor(channel, request, forFlusher, backpressure)); + managementExecutor.submit(new ManagementRequestProcessor(channel, request, forFlusher, backpressure)); return; } } - // if native_transport_max_auth_threads is < 1, don't delegate to new pool on auth messages - boolean isAuthQuery = DatabaseDescriptor.getNativeTransportMaxAuthThreads() > 0 && - (request.type == Message.Type.AUTH_RESPONSE || request.type == Message.Type.CREDENTIALS); - - // Importantly, the authExecutor will handle the AUTHENTICATE message which may be CPU intensive. - LocalAwareExecutorPlus executor = isAuthQuery ? authExecutor : requestExecutor; - - executor.submit(new RequestProcessor(channel, request, forFlusher, backpressure)); + requestExecutor.submit(new RequestProcessor(channel, request, forFlusher, backpressure)); ClientMetrics.instance.markRequestDispatched(); } @@ -319,12 +345,12 @@ public long timeSpentInQueueNanos() */ public class RequestProcessor implements DebuggableTask.RunnableDebuggableTask { - private final Channel channel; - private final Message.Request request; - private final FlushItemConverter forFlusher; - private final Overload backpressure; + protected final Channel channel; + protected final Message.Request request; + protected final FlushItemConverter forFlusher; + protected final Overload backpressure; - private volatile long startTimeNanos; + protected volatile long startTimeNanos; public RequestProcessor(Channel channel, Message.Request request, FlushItemConverter forFlusher, Overload backpressure) { @@ -369,6 +395,76 @@ public String toString() } } + /** RequestProcessor for management connections that validates before executing. */ + private class ManagementRequestProcessor extends RequestProcessor { + + public ManagementRequestProcessor(Channel channel, + Message.Request request, + FlushItemConverter forFlusher, + Overload backpressure) { + super(channel, request, forFlusher, backpressure); + } + + @Override + public void run() { + startTimeNanos = MonotonicClock.Global.preciseTime.now(); + RequestTime requestTime = new RequestTime(request.createdAtNanos, startTimeNanos); + + // Validate management request BEFORE executing + Connection connection = request.connection(); + if (connection instanceof ServerConnection) { + ServerConnection serverConnection = (ServerConnection) connection; + if (serverConnection.isManagementConnection()) { + if (!isManagementRequestAllowed(request)) { + Message.Response response = ErrorMessage.fromException( + new InvalidRequestException( + "Only executions of the COMMAND statements are allowed on the management port.")); + response.setStreamId(request.getStreamId()); + response.attach(connection); + FlushItem toFlush = forFlusher.toFlushItem(channel, request, response); + flush(toFlush); + return; + } + } + } + + // If validation passes, call the normal processRequest to execute + // This calls the instance method processRequest() which does all the work + processRequest(channel, request, forFlusher, backpressure, requestTime); + } + + private boolean isManagementRequestAllowed(Message.Request request) + { + switch (request.type) + { + case QUERY: + try + { + // Early parse the query to check if it's a COMMAND statement. + // For management non-intensive operations double parsing is acceptable. + CQLStatement.Raw rawStatement = QueryProcessor.parseStatement(((QueryMessage) request).query); + return rawStatement instanceof ExecuteCommandStatement.Raw; + } + catch (Exception e) + { + // If parsing fails (syntax error, etc.), it's not a valid command statement; + // this is expected for non-command queries. + return false; + } + case STARTUP: + case CREDENTIALS: + case AUTH_RESPONSE: + return true; // Protocol messages are always allowed. + case EXECUTE: + case PREPARE: + case BATCH: + case OPTIONS: + default: + return false; // Not supported and not allowed on management connections. + } + } + } + /** * Checks if the item in the head of the queue has spent more than allowed time in the queue. */ @@ -520,6 +616,13 @@ public boolean isDone() return requestExecutor.getPendingTaskCount() == 0 && requestExecutor.getActiveTaskCount() == 0; } + public static void shutdown() + { + requestExecutor.shutdown(); + authExecutor.shutdown(); + managementExecutor.shutdown(); + } + /** * Dispatcher for EventMessages. In {@link Server.ConnectionTracker#send(Event)}, the strategy * for delivering events to registered clients is dependent on protocol version and the configuration diff --git a/src/java/org/apache/cassandra/transport/DispatcherExecutors.java b/src/java/org/apache/cassandra/transport/DispatcherExecutors.java deleted file mode 100644 index 7fbbbc33339e..000000000000 --- a/src/java/org/apache/cassandra/transport/DispatcherExecutors.java +++ /dev/null @@ -1,124 +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 org.apache.cassandra.transport; - -import org.apache.cassandra.concurrent.LocalAwareExecutorPlus; -import org.apache.cassandra.config.DatabaseDescriptor; - -import static org.apache.cassandra.concurrent.SharedExecutorPool.SHARED; - -/** - * Manages the lifecycle of executors used by the native transport dispatcher. - * These executors are shared across all Server instances and tied to the cassandra daemon lifecycle. - */ -public class DispatcherExecutors -{ - private static final DispatcherExecutors instance = new DispatcherExecutors(); - - private LocalAwareExecutorPlus requestExecutor; - private LocalAwareExecutorPlus authExecutor; - private LocalAwareExecutorPlus managementExecutor; - - private final Object mutex = new Object(); - private volatile boolean initialized = false; - - public static DispatcherExecutors instance() - { - return instance; - } - - /** Ensures all executors are initialized. All executors are initialized together. */ - private void initialize() - { - if (initialized) - return; - - synchronized (mutex) - { - if (initialized) - return; - - if (requestExecutor == null) - { - requestExecutor = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(), - DatabaseDescriptor::setNativeTransportMaxThreads, - "transport", - "Native-Transport-Requests"); - } - if (authExecutor == null) - { - authExecutor = SHARED.newExecutor(Math.max(1, DatabaseDescriptor.getNativeTransportMaxAuthThreads()), - DatabaseDescriptor::setNativeTransportMaxAuthThreads, - "transport", - "Native-Transport-Auth-Requests"); - } - if (managementExecutor == null) - { - managementExecutor = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportManagementMaxThreads(), - DatabaseDescriptor::setNativeTransportManagementMaxThreads, - "transport", - "Native-Transport-Management-Tasks"); - } - - initialized = true; - } - } - - public LocalAwareExecutorPlus getRequestExecutor() - { - initialize(); - return requestExecutor; - } - - public LocalAwareExecutorPlus getAuthExecutor() - { - initialize(); - return authExecutor; - } - - public LocalAwareExecutorPlus getManagementExecutor() - { - initialize(); - return managementExecutor; - } - - /** Shuts down node-level executors. Should only be called during node shutdown. */ - public void shutdown() - { - synchronized (mutex) - { - if (requestExecutor != null) - { - requestExecutor.shutdown(); - requestExecutor = null; - } - if (authExecutor != null) - { - authExecutor.shutdown(); - authExecutor = null; - } - if (managementExecutor != null) - { - managementExecutor.shutdown(); - managementExecutor = null; - } - initialized = false; - } - } -} diff --git a/test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java b/test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java index 575c7fad70ec..b38d3e84b328 100644 --- a/test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java +++ b/test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java @@ -24,6 +24,7 @@ import org.apache.cassandra.exceptions.InvalidRequestException; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -42,36 +43,28 @@ public void testExecuteCommandForcecompact() throws Throwable flush(keyspaceName, tableName); - // Execute the COMMAND statement using CQL-style syntax + // Execute the COMMAND statement using CQL-style syntax. String command = String.format("COMMAND forcecompact WITH \"keyspace\" = '%s' AND \"table\" = '%s' AND keys = ['k4', 'k2', 'k7'];", keyspaceName, tableName); UntypedResultSet result = execute(command); - // Verify that we got a result assertNotNull("Result should not be null", result); - - // Verify that the result has one row with one column (the output) assertEquals("Result should have one row", 1, result.size()); UntypedResultSet.Row row = result.one(); assertNotNull("Row should not be null", row); - // Verify that the output column exists and has a value assertTrue("Result should have 'output' column", row.has("output")); String output = row.getString("output"); assertNotNull("Output should not be null", output); - - // The output should be non-empty (command execution output) - assertTrue("Output should not be empty", !output.trim().isEmpty()); + assertFalse("Output should not be empty", output.trim().isEmpty()); } @Test public void testExecuteCommandWithInvalidCommand() throws Throwable { - // Test that an invalid command name throws an exception String command = "COMMAND nonexistentcommand WITH key = 'value';"; - assertInvalidThrow(InvalidRequestException.class, command); } } diff --git a/test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java b/test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java index ec2b51e01cf8..601a1b1cdbc4 100644 --- a/test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java +++ b/test/unit/org/apache/cassandra/transport/MessageDispatcherTest.java @@ -140,12 +140,12 @@ public void testAuthRateLimiterDisabled() throws Exception private long completedRequests() { - return DispatcherExecutors.instance().getRequestExecutor().getCompletedTaskCount(); + return Dispatcher.requestExecutor.getCompletedTaskCount(); } private long completedAuth() { - return DispatcherExecutors.instance().getAuthExecutor().getCompletedTaskCount(); + return Dispatcher.authExecutor.getCompletedTaskCount(); } public long tryAuth(Callable check) throws Exception diff --git a/test/unit/org/apache/cassandra/transport/MessageManagementDispatcherTest.java b/test/unit/org/apache/cassandra/transport/MessageManagementDispatcherTest.java index 616a5395b3fe..3ade14073773 100644 --- a/test/unit/org/apache/cassandra/transport/MessageManagementDispatcherTest.java +++ b/test/unit/org/apache/cassandra/transport/MessageManagementDispatcherTest.java @@ -109,7 +109,10 @@ public void testManagementConnectionAllMessageTypes() throws Exception continue; Message.Request request = createManagementRequest(managementConnectionMock(), type); - long managementTasks = tryRequest(this::completedManagement, request); + long managementTasks = tryRequest(() -> Message.Type.CREDENTIALS == type || Message.Type.AUTH_RESPONSE == type + ? completedAuth() + : completedManagement(), + request); assertEquals(format("Management %s request should route to management executor", type), 1, managementTasks); } @@ -148,17 +151,17 @@ public Response execute(QueryState queryState, Dispatcher.RequestTime requestTim private long completedRequests() { - return DispatcherExecutors.instance().getRequestExecutor().getCompletedTaskCount(); + return Dispatcher.requestExecutor.getCompletedTaskCount(); } private long completedAuth() { - return DispatcherExecutors.instance().getAuthExecutor().getCompletedTaskCount(); + return Dispatcher.authExecutor.getCompletedTaskCount(); } private long completedManagement() { - return DispatcherExecutors.instance().getManagementExecutor().getCompletedTaskCount(); + return Dispatcher.managementExecutor.getCompletedTaskCount(); } private long tryRequest(Callable check, Message.Request request) throws Exception From 73a5e966806ec6a0cbbc4f6934e43a0ba318dd2a Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 25 Jan 2026 23:10:55 +0100 Subject: [PATCH 10/14] fix configuration yaml --- conf/cassandra.yaml | 4 ++-- conf/cassandra_latest.yaml | 4 ++-- .../apache/cassandra/tools/nodetool/CompactionStatsTest.java | 5 +++-- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index f6e509451905..2f73886a748f 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -1114,7 +1114,7 @@ rpc_keepalive: true # The address on which the native management transport is bound can be configured # using rpc_management_address or rpc_management_interface. If neither is set, # it defaults to rpc_address (or rpc_interface if that was used). -start_native_management_transport: false +start_native_transport_management: false # The address or interface to bind the native management transport server to. # This allows you to bind management operations to a different network interface @@ -1152,7 +1152,7 @@ start_native_management_transport: false # Port for the management CQL native transport to listen for clients on. # For security reasons, you should not expose this port to the internet. # Firewall it if needed. -native_management_transport_port: 11211 +native_transport_management_port: 11211 # The maximum threads for handling management requests are set # separately from regular requests to allow better isolation and # prioritization of management operations. This is important to diff --git a/conf/cassandra_latest.yaml b/conf/cassandra_latest.yaml index dcbdaf590f49..ce0a533aff90 100644 --- a/conf/cassandra_latest.yaml +++ b/conf/cassandra_latest.yaml @@ -1103,7 +1103,7 @@ rpc_keepalive: true # The address on which the native management transport is bound can be configured # using rpc_management_address or rpc_management_interface. If neither is set, # it defaults to rpc_address (or rpc_interface if that was used). -start_native_management_transport: false +start_native_transport_management: false # The address or interface to bind the native management transport server to. # This allows you to bind management operations to a different network interface @@ -1141,7 +1141,7 @@ start_native_management_transport: false # Port for the management CQL native transport to listen for clients on. # For security reasons, you should not expose this port to the internet. # Firewall it if needed. -native_management_transport_port: 11211 +native_transport_management_port: 11211 # The maximum threads for handling management requests are set # separately from regular requests to allow better isolation and # prioritization of management operations. This is important to diff --git a/test/unit/org/apache/cassandra/tools/nodetool/CompactionStatsTest.java b/test/unit/org/apache/cassandra/tools/nodetool/CompactionStatsTest.java index 64ab8fe34bb3..e5bff1c0ad97 100644 --- a/test/unit/org/apache/cassandra/tools/nodetool/CompactionStatsTest.java +++ b/test/unit/org/apache/cassandra/tools/nodetool/CompactionStatsTest.java @@ -26,6 +26,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import org.apache.cassandra.cql3.CQLNodetoolProtocolTester; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.compaction.CompactionInfo; @@ -40,7 +41,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; -public class CompactionStatsTest extends CQLTester +public class CompactionStatsTest extends CQLNodetoolProtocolTester { @BeforeClass public static void setup() throws Exception @@ -259,7 +260,7 @@ private String waitForNumberOfPendingTasks(int pendingTasksToWaitFor, String... { AtomicReference stdout = new AtomicReference<>(); await().until(() -> { - ToolRunner.ToolResult tool = ToolRunner.invokeNodetool(args); + ToolRunner.ToolResult tool = invokeNodetool(args); tool.assertOnCleanExit(); String output = tool.getStdout(); stdout.set(output); From f1759559520ba2001729c2459e86917ce5e30b4a Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 25 Jan 2026 23:22:05 +0100 Subject: [PATCH 11/14] change default port for cql management connect command --- src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java b/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java index 7d758796954f..4fe3f40bbdb8 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java +++ b/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java @@ -39,7 +39,7 @@ @Command(name = "cqlconnect", description = "Connect to a Cassandra node via CQL") public class CqlConnect extends AbstractCommand implements AutoCloseable { - private static final int DEFAULT_CQL_PORT = 9042; + private static final int DEFAULT_CQL_PORT = 11211; /** The command specification, used to access command-specific properties. */ @Spec From 125bb8ba665bbaecfc0ee3b36e18406d9152d855 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 25 Jan 2026 23:52:30 +0100 Subject: [PATCH 12/14] fix parser to respect command without args --- src/antlr/Parser.g | 4 ++-- .../config/CassandraRelevantProperties.java | 2 +- .../cassandra/tools/nodetool/CqlConnect.java | 12 ++++++++---- .../strategy/CqlCommandExecutionStrategy.java | 2 +- .../apache/cassandra/transport/Dispatcher.java | 1 + .../management/CqlExecuteCommandTest.java | 18 ++++++++++++++++++ 6 files changed, 31 insertions(+), 8 deletions(-) diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g index b380da2d46f9..b4b290e808ef 100644 --- a/src/antlr/Parser.g +++ b/src/antlr/Parser.g @@ -1372,7 +1372,7 @@ securityLabelOnUserTypeFieldStatement returns [SecurityLabelOnUserTypeFieldState ; /** - * COMMAND WITH key1 = value1 AND key2 = value2; + * COMMAND [WITH key1 = value1 AND key2 = value2]; */ executeCommandStatement returns [ExecuteCommandStatement.Raw stmt] @init { @@ -1380,7 +1380,7 @@ executeCommandStatement returns [ExecuteCommandStatement.Raw stmt] java.util.Map args = new java.util.LinkedHashMap<>(); } : K_COMMAND cmdName=noncol_ident - K_WITH commandProperties[args] + (K_WITH commandProperties[args])? { $stmt = new ExecuteCommandStatement.Raw(cmdName.toString(), args); } diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index ae63668836d3..61492bfd28d0 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -92,7 +92,7 @@ public enum CassandraRelevantProperties CASSANDRA_AVAILABLE_PROCESSORS("cassandra.available_processors"), /** * Defines the protocol used by the Cassandra CLI to connect to Cassandra nodes. - * Possible values are {@code "static_mbeans"}, {@code "command_mbeans"} or {@code cql}. + * Possible values are {@code "static_mbean"}, {@code "command_mbean"} or {@code cql}. * By default, the Cassandra CLI uses the JMX protocol via static MBeans. */ CASSANDRA_CLI_EXECUTION_PROTOCOL("cassandra.cli.execution.protocol", "static_mbean"), diff --git a/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java b/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java index 4fe3f40bbdb8..56703af58403 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java +++ b/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java @@ -39,7 +39,7 @@ @Command(name = "cqlconnect", description = "Connect to a Cassandra node via CQL") public class CqlConnect extends AbstractCommand implements AutoCloseable { - private static final int DEFAULT_CQL_PORT = 11211; + private static final String DEFAULT_CQL_PORT = 11211; /** The command specification, used to access command-specific properties. */ @Spec @@ -48,8 +48,11 @@ public class CqlConnect extends AbstractCommand implements AutoCloseable @Option(names = { "-h", "--host" }, description = "Node hostname or ip address", arity = "0..1") private String host = "127.0.0.1"; - @Option(names = { "-p", "--port" }, description = "Remote CQL native transport port number", arity = "0..1") - private String port = String.valueOf(DEFAULT_CQL_PORT); + @Option(names = { "-p", "--port" }, + description = "Remote CQL native transport port number", + arity = "0..1", + defaultValue = DEFAULT_CQL_PORT) + private String port; private volatile SimpleClient client; @@ -63,7 +66,8 @@ public void run() try { - SimpleClient.Builder builder = SimpleClient.builder(host, parseInt(port)) + String portToUse = port == null || port.isEmpty() ? DEFAULT_CQL_PORT : port; + SimpleClient.Builder builder = SimpleClient.builder(host, parseInt(portToUse)) .protocolVersion(ProtocolVersion.V5); client = builder.build(); client.connect(false); diff --git a/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java b/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java index ac785ef84b62..8c2f6eb6c894 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java +++ b/src/java/org/apache/cassandra/tools/nodetool/strategy/CqlCommandExecutionStrategy.java @@ -100,7 +100,7 @@ public int execute(CommandLine.ParseResult parseResult) throws CommandLine.Execu String output = UTF8Type.instance.getSerializer().deserialize(firstRow.get(1)); // NodeProbe instance is not available here, so print directly to the command output. parseResult.commandSpec().commandLine().getOut().println(output); - parseResult.commandSpec().commandLine().getOut().println(executionId.toString()); + parseResult.commandSpec().commandLine().getOut().println("Command execution id: " + executionId.toString()); } } } diff --git a/src/java/org/apache/cassandra/transport/Dispatcher.java b/src/java/org/apache/cassandra/transport/Dispatcher.java index 4e8362b381fe..9dd56c63a9ad 100644 --- a/src/java/org/apache/cassandra/transport/Dispatcher.java +++ b/src/java/org/apache/cassandra/transport/Dispatcher.java @@ -447,6 +447,7 @@ private boolean isManagementRequestAllowed(Message.Request request) } catch (Exception e) { + logger.warn("The command request parsing failed. The command will not be executed: {}", e.getMessage()); // If parsing fails (syntax error, etc.), it's not a valid command statement; // this is expected for non-command queries. return false; diff --git a/test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java b/test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java index b38d3e84b328..d90ea9f8d81a 100644 --- a/test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java +++ b/test/unit/org/apache/cassandra/management/CqlExecuteCommandTest.java @@ -67,4 +67,22 @@ public void testExecuteCommandWithInvalidCommand() throws Throwable String command = "COMMAND nonexistentcommand WITH key = 'value';"; assertInvalidThrow(InvalidRequestException.class, command); } + + @Test + public void testExecuteCommandWithoutParameters() throws Throwable + { + // Test that COMMAND statements without WITH clause parse correctly + String command = "COMMAND status;"; + + UntypedResultSet result = execute(command); + + assertNotNull("Result should not be null", result); + assertEquals("Result should have one row", 1, result.size()); + + UntypedResultSet.Row row = result.one(); + assertNotNull("Row should not be null", row); + assertTrue("Result should have 'output' column", row.has("output")); + String output = row.getString("output"); + assertNotNull("Output should not be null", output); + } } From 6433c6e108b82b5d0c6c7dcab033eb4e214d1a6b Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Sun, 25 Jan 2026 23:53:08 +0100 Subject: [PATCH 13/14] fix compile --- src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java b/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java index 56703af58403..4470772cffed 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java +++ b/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java @@ -39,7 +39,7 @@ @Command(name = "cqlconnect", description = "Connect to a Cassandra node via CQL") public class CqlConnect extends AbstractCommand implements AutoCloseable { - private static final String DEFAULT_CQL_PORT = 11211; + private static final String DEFAULT_CQL_PORT = "11211"; /** The command specification, used to access command-specific properties. */ @Spec From 80a9de69ca967678837277ba90226136b7f51b82 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 26 Jan 2026 00:25:21 +0100 Subject: [PATCH 14/14] nodetool shell script should respect management cql port --- bin/nodetool | 71 +++++++++++++------ .../cassandra/tools/nodetool/CqlConnect.java | 16 ++--- 2 files changed, 55 insertions(+), 32 deletions(-) diff --git a/bin/nodetool b/bin/nodetool index 4ca496b4ead5..4e3062d4fd4c 100755 --- a/bin/nodetool +++ b/bin/nodetool @@ -43,30 +43,51 @@ if [ -z "$CASSANDRA_CONF" -o -z "$CLASSPATH" ]; then exit 1 fi -JMX_PORT="" - -# Try to parse port from configure_jmx method call, when not commented out -if [ -f "$CASSANDRA_CONF/cassandra-env.sh" ]; then - jmx_method_call=$(grep "^configure_jmx \+[0-9]\+$" "$CASSANDRA_CONF/cassandra-env.sh") - if [ ! "x${jmx_method_call}" = "x" ]; then - JMX_PORT=$(echo "${jmx_method_call}" | tr -s " " | cut -d " " -f2) +# Check if protocol is cql (from environment variable) +PROTOCOL_IS_CQL="" +if [ "x$CASSANDRA_CLI_EXECUTION_PROTOCOL" != "x" ]; then + protocol_lower=$(echo "$CASSANDRA_CLI_EXECUTION_PROTOCOL" | tr '[:upper:]' '[:lower:]') + if [ "$protocol_lower" = "cql" ]; then + PROTOCOL_IS_CQL="true" fi fi -# In case JMX_PORT is not set (when configure_jmx in cassandra-env.sh is commented out), -# try to parse it from cassandra.yaml. -if [ "x$JMX_PORT" = "x" ]; then +# Check for cql protocol in system properties to determine if we should parse CQL or JMX port +for arg in "$@"; do + case "$arg" in + -Dcassandra.cli.execution.protocol=cql|-Dcassandra.cli.execution.protocol=CQL) + PROTOCOL_IS_CQL="true" + break + ;; + esac +done + +# Parse port from config files based on protocol +CONNECTION_PORT="" +if [ "$PROTOCOL_IS_CQL" = "true" ]; then + # For CQL, parse native_transport_management_port from cassandra.yaml if [ -f "$CASSANDRA_CONF/cassandra.yaml" ]; then - JMX_PORT=$(grep jmx_port "$CASSANDRA_CONF/cassandra.yaml" | cut -d ':' -f 2 | tr -d '[[:space:]]') + CONNECTION_PORT=$(grep native_transport_management_port "$CASSANDRA_CONF/cassandra.yaml" | cut -d ':' -f 2 | tr -d '[[:space:]]') + fi +else + # For JMX, parse port from configure_jmx method call, when not commented out + if [ -f "$CASSANDRA_CONF/cassandra-env.sh" ]; then + jmx_method_call=$(grep "^configure_jmx \+[0-9]\+$" "$CASSANDRA_CONF/cassandra-env.sh") + if [ ! "x${jmx_method_call}" = "x" ]; then + CONNECTION_PORT=$(echo "${jmx_method_call}" | tr -s " " | cut -d " " -f2) + fi fi -fi -# If, by any chance, it is not there either, set it to default. -if [ "x$JMX_PORT" = "x" ]; then - JMX_PORT=7199 + # In case CONNECTION_PORT is not set (when configure_jmx in cassandra-env.sh is commented out), + # try to parse it from cassandra.yaml. + if [ "x$CONNECTION_PORT" = "x" ]; then + if [ -f "$CASSANDRA_CONF/cassandra.yaml" ]; then + CONNECTION_PORT=$(grep jmx_port "$CASSANDRA_CONF/cassandra.yaml" | cut -d ':' -f 2 | tr -d '[[:space:]]') + fi + fi fi -# JMX Port passed via cmd line args (-p 9999 / --port 9999 / --port=9999) +# Connection Port passed via cmd line args (-p 9999 / --port 9999 / --port=9999) # should override the value from cassandra-env.sh ARGS="" JVM_ARGS="" @@ -76,19 +97,19 @@ do if [ ! $1 ]; then break; fi case $1 in -p) - JMX_PORT=$2 + CONNECTION_PORT=$2 shift ;; --port=*) - JMX_PORT=$(echo $1 | cut -d '=' -f 2) + CONNECTION_PORT=$(echo $1 | cut -d '=' -f 2) ;; --port) - JMX_PORT=$2 + CONNECTION_PORT=$2 shift ;; --ssl) if [ -f $SSL_FILE ] - then + then SSL_ARGS=$(cat $SSL_FILE | tr '\n' ' ') fi JVM_ARGS="$JVM_ARGS -Dssl.enable=true $SSL_ARGS" @@ -108,6 +129,14 @@ do shift done +if [ "x$CONNECTION_PORT" = "x" ]; then + if [ "$PROTOCOL_IS_CQL" = "true" ]; then + CONNECTION_PORT=11211 + else + CONNECTION_PORT=7199 + fi +fi + if [ "x$MAX_HEAP_SIZE" = "x" ]; then MAX_HEAP_SIZE="128m" fi @@ -123,7 +152,7 @@ CMD=$(echo "$JAVA" $JAVA_AGENT -ea -cp "$CLASSPATH" $JVM_OPTS -Xmx$MAX_HEAP_SIZE -Dcassandra.logdir="$CASSANDRA_LOG_DIR" \ -Dlogback.configurationFile=logback-tools.xml \ $JVM_ARGS \ - org.apache.cassandra.tools.NodeTool -p $JMX_PORT $ARGS) + org.apache.cassandra.tools.NodeTool -p $CONNECTION_PORT $ARGS) if [ "x$ARCHIVE_COMMAND" != "x" ] then diff --git a/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java b/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java index 4470772cffed..d60d0a7cc150 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java +++ b/src/java/org/apache/cassandra/tools/nodetool/CqlConnect.java @@ -31,15 +31,13 @@ import picocli.CommandLine.Option; import picocli.CommandLine.Spec; -import static java.lang.Integer.parseInt; - /** * Command options for NodeTool commands that are executed via CQL. */ @Command(name = "cqlconnect", description = "Connect to a Cassandra node via CQL") public class CqlConnect extends AbstractCommand implements AutoCloseable { - private static final String DEFAULT_CQL_PORT = "11211"; + private static final int DEFAULT_CQL_PORT = 11211; /** The command specification, used to access command-specific properties. */ @Spec @@ -48,11 +46,8 @@ public class CqlConnect extends AbstractCommand implements AutoCloseable @Option(names = { "-h", "--host" }, description = "Node hostname or ip address", arity = "0..1") private String host = "127.0.0.1"; - @Option(names = { "-p", "--port" }, - description = "Remote CQL native transport port number", - arity = "0..1", - defaultValue = DEFAULT_CQL_PORT) - private String port; + @Option(names = { "-p", "--port" }, description = "Remote CQL native transport port number", arity = "0..1") + private int port = DEFAULT_CQL_PORT; private volatile SimpleClient client; @@ -66,9 +61,8 @@ public void run() try { - String portToUse = port == null || port.isEmpty() ? DEFAULT_CQL_PORT : port; - SimpleClient.Builder builder = SimpleClient.builder(host, parseInt(portToUse)) - .protocolVersion(ProtocolVersion.V5); + output.printInfo("Connecting to %s:%s via CQL...%n", host, port); + SimpleClient.Builder builder = SimpleClient.builder(host, port).protocolVersion(ProtocolVersion.V5); client = builder.build(); client.connect(false); }