You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by dc...@apache.org on 2020/09/16 00:01:27 UTC

[cassandra] branch trunk updated: Add a new jmxtool which can dump what JMX objects exist and diff

This is an automated email from the ASF dual-hosted git repository.

dcapwell pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 91bcbb2  Add a new jmxtool which can dump what JMX objects exist and diff
91bcbb2 is described below

commit 91bcbb28733437a7a5fedb16984d4de2fa70b049
Author: David Capwell <dc...@apache.org>
AuthorDate: Tue Sep 15 15:16:39 2020 -0700

    Add a new jmxtool which can dump what JMX objects exist and diff
    
    patch by David Capwell, Stephen Mallette; reviewed by Berenguer Blasi, Jon Meredith for CASSANDRA-16082
---
 .../apache/cassandra/io/util/DataOutputBuffer.java |      2 +-
 .../apache/cassandra/service/CassandraDaemon.java  |      8 +-
 src/java/org/apache/cassandra/tools/JMXTool.java   |    848 +
 test/data/jmxdump/cassandra-3.0-jmx.yaml           |  48799 ++++++++
 test/data/jmxdump/cassandra-3.11-jmx.yaml          |  57780 ++++++++++
 test/data/jmxdump/cassandra-4.0-jmx.yaml           | 107557 ++++++++++++++++++
 test/unit/org/apache/cassandra/cql3/CQLTester.java |     13 +
 .../cassandra/tools/JMXCompatabilityTest.java      |    171 +
 .../org/apache/cassandra/tools/JMXToolTest.java    |    171 +
 .../org/apache/cassandra/tools/ToolRunner.java     |    214 +-
 tools/bin/jmxtool                                  |     76 +
 tools/bin/jmxtool.bat                              |     36 +
 12 files changed, 215580 insertions(+), 95 deletions(-)

diff --git a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java
index a6c7086..ac8bc2f 100644
--- a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java
+++ b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java
@@ -90,7 +90,7 @@ public class DataOutputBuffer extends BufferedDataOutputStreamPlus
     @Override
     public void flush() throws IOException
     {
-        throw new UnsupportedOperationException();
+
     }
 
     //The actual value observed in Hotspot is only -2
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index c8e1e57..e1a254a 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -645,7 +645,7 @@ public class CassandraDaemon
         {
             applyConfig();
 
-            MBeanWrapper.instance.registerMBean(new StandardMBean(new NativeAccess(), NativeAccessMBean.class), MBEAN_NAME, MBeanWrapper.OnException.LOG);
+            registerNativeAccess();
 
             if (FBUtilities.isWindows)
             {
@@ -697,6 +697,12 @@ public class CassandraDaemon
         }
     }
 
+    @VisibleForTesting
+    public static void registerNativeAccess() throws javax.management.NotCompliantMBeanException
+    {
+        MBeanWrapper.instance.registerMBean(new StandardMBean(new NativeAccess(), NativeAccessMBean.class), MBEAN_NAME, MBeanWrapper.OnException.LOG);
+    }
+
     public void applyConfig()
     {
         DatabaseDescriptor.daemonInitialization();
diff --git a/src/java/org/apache/cassandra/tools/JMXTool.java b/src/java/org/apache/cassandra/tools/JMXTool.java
new file mode 100644
index 0000000..30d3d22
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/JMXTool.java
@@ -0,0 +1,848 @@
+package org.apache.cassandra.tools;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.Callable;
+import java.util.function.BiConsumer;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import javax.inject.Inject;
+import javax.management.InstanceNotFoundException;
+import javax.management.IntrospectionException;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanFeatureInfo;
+import javax.management.MBeanInfo;
+import javax.management.MBeanOperationInfo;
+import javax.management.MBeanParameterInfo;
+import javax.management.MBeanServerConnection;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXServiceURL;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Sets;
+import com.google.common.collect.Sets.SetView;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.airlift.airline.Arguments;
+import io.airlift.airline.Cli;
+import io.airlift.airline.Command;
+import io.airlift.airline.Help;
+import io.airlift.airline.HelpOption;
+import io.airlift.airline.Option;
+import org.yaml.snakeyaml.TypeDescription;
+import org.yaml.snakeyaml.Yaml;
+import org.yaml.snakeyaml.constructor.Constructor;
+import org.yaml.snakeyaml.nodes.MappingNode;
+import org.yaml.snakeyaml.nodes.Node;
+import org.yaml.snakeyaml.nodes.Tag;
+import org.yaml.snakeyaml.representer.Representer;
+
+public class JMXTool
+{
+    private static final List<String> METRIC_PACKAGES = Arrays.asList("org.apache.cassandra.metrics",
+                                                                      "org.apache.cassandra.db",
+                                                                      "org.apache.cassandra.hints",
+                                                                      "org.apache.cassandra.internal",
+                                                                      "org.apache.cassandra.net",
+                                                                      "org.apache.cassandra.request",
+                                                                      "org.apache.cassandra.service");
+
+    private static final Comparator<MBeanOperationInfo> OPERATOR_COMPARATOR = (a, b) -> {
+        int rc = a.getName().compareTo(b.getName());
+        if (rc != 0)
+            return rc;
+        String[] aSig = Stream.of(a.getSignature()).map(MBeanParameterInfo::getName).toArray(String[]::new);
+        String[] bSig = Stream.of(b.getSignature()).map(MBeanParameterInfo::getName).toArray(String[]::new);
+        rc = Integer.compare(aSig.length, bSig.length);
+        if (rc != 0)
+            return rc;
+        for (int i = 0; i < aSig.length; i++)
+        {
+            rc = aSig[i].compareTo(bSig[i]);
+            if (rc != 0)
+                return rc;
+        }
+        return rc;
+    };
+
+    @Command(name = "dump", description = "Dump the Apache Cassandra JMX objects and metadata.")
+    public static final class Dump implements Callable<Void>
+    {
+        @Inject
+        private HelpOption helpOption;
+
+        @Option(title = "url", name = { "-u", "--url" }, description = "JMX url to target")
+        private String targetUrl = "service:jmx:rmi:///jndi/rmi://localhost:7199/jmxrmi";
+
+        @Option(title = "format", name = { "-f", "--format" }, description = "What format to dump content as; supported values are console (default), json, and yaml")
+        private Format format = Format.console;
+
+        public Void call() throws Exception
+        {
+            Map<String, Info> map = load(new JMXServiceURL(targetUrl));
+            format.dump(System.out, map);
+            return null;
+        }
+
+        public enum Format
+        {
+            console
+            {
+                void dump(OutputStream output, Map<String, Info> map)
+                {
+                    @SuppressWarnings("resource")
+                    // output should be released by caller
+                    PrintStream out = toPrintStream(output);
+                    for (Map.Entry<String, Info> e : map.entrySet())
+                    {
+                        String name = e.getKey();
+                        Info info = e.getValue();
+
+                        out.println(name);
+                        out.println("\tAttributes");
+                        Stream.of(info.attributes).forEach(a -> printRow(out, a.name, a.type, a.access));
+                        out.println("\tOperations");
+                        Stream.of(info.operations).forEach(o -> {
+                            String args = Stream.of(o.parameters)
+                                                .map(i -> i.name + ": " + i.type)
+                                                .collect(Collectors.joining(",", "(", ")"));
+                            printRow(out, o.name, o.returnType, args);
+                        });
+                    }
+                }
+            },
+            json
+            {
+                void dump(OutputStream output, Map<String, Info> map) throws IOException
+                {
+                    ObjectMapper mapper = new ObjectMapper();
+                    mapper.writeValue(output, map);
+                }
+            },
+            yaml
+            {
+                void dump(OutputStream output, Map<String, Info> map) throws IOException
+                {
+                    Representer representer = new Representer();
+                    representer.addClassTag(Info.class, Tag.MAP); // avoid the auto added tag
+                    Yaml yaml = new Yaml(representer);
+                    yaml.dump(map, new OutputStreamWriter(output));
+                }
+            };
+
+            private static PrintStream toPrintStream(OutputStream output)
+            {
+                try
+                {
+                    return output instanceof PrintStream ? (PrintStream) output : new PrintStream(output, true, "UTF-8");
+                }
+                catch (UnsupportedEncodingException e)
+                {
+                    throw new AssertionError(e); // utf-8 is a required charset for the JVM
+                }
+            }
+
+            abstract void dump(OutputStream output, Map<String, Info> map) throws IOException;
+        }
+    }
+
+    @Command(name = "diff", description = "Diff two jmx dump files and report their differences")
+    public static final class Diff implements Callable<Void>
+    {
+        @Inject
+        private HelpOption helpOption;
+
+        @Arguments(title = "files", usage = "<left> <right>", description = "Files to diff")
+        private List<File> files;
+
+        @Option(title = "format", name = { "-f", "--format" }, description = "What format the files are in; only support json and yaml as format")
+        private Format format = Format.yaml;
+
+        @Option(title = "ignore left", name = { "--ignore-missing-on-left" }, description = "Ignore results missing on the left")
+        private boolean ignoreMissingLeft;
+
+        @Option(title = "ignore right", name = { "--ignore-missing-on-right" }, description = "Ignore results missing on the right")
+        private boolean ignoreMissingRight;
+
+        @Option(title = "exclude objects", name = "--exclude-object", description
+                                                                      = "Ignores processing specific objects. " +
+                                                                        "Each usage should take a single object, " +
+                                                                        "but can use this flag multiple times.")
+        private List<CliPattern> excludeObjects = new ArrayList<>();
+
+        @Option(title = "exclude attributes", name = "--exclude-attribute", description
+                                                                            = "Ignores processing specific attributes. " +
+                                                                              "Each usage should take a single attribute, " +
+                                                                              "but can use this flag multiple times.")
+        private List<CliPattern> excludeAttributes = new ArrayList<>();
+
+        @Option(title = "exclude operations", name = "--exclude-operation", description
+                                                                            = "Ignores processing specific operations. " +
+                                                                              "Each usage should take a single operation, " +
+                                                                              "but can use this flag multiple times.")
+        private List<CliPattern> excludeOperations = new ArrayList<>();
+
+        public Void call() throws Exception
+        {
+            Preconditions.checkArgument(files.size() == 2, "files requires 2 arguments but given %s", files);
+            Map<String, Info> left;
+            Map<String, Info> right;
+            try (FileInputStream leftStream = new FileInputStream(files.get(0));
+                 FileInputStream rightStream = new FileInputStream(files.get(1)))
+            {
+                left = format.load(leftStream);
+                right = format.load(rightStream);
+            }
+
+            diff(left, right);
+            return null;
+        }
+
+        private void diff(Map<String, Info> left, Map<String, Info> right)
+        {
+            DiffResult<String> objectNames = diff(left.keySet(), right.keySet(), name -> {
+                for (CliPattern p : excludeObjects)
+                {
+                    if (p.pattern.matcher(name).matches())
+                        return false;
+                }
+                return true;
+            });
+
+            if (!ignoreMissingRight && !objectNames.notInRight.isEmpty())
+            {
+                System.out.println("Objects not in right:");
+                printSet(0, objectNames.notInRight);
+            }
+            if (!ignoreMissingLeft && !objectNames.notInLeft.isEmpty())
+            {
+                System.out.println("Objects not in left: ");
+                printSet(0, objectNames.notInLeft);
+            }
+            Runnable printHeader = new Runnable()
+            {
+                boolean printedHeader = false;
+
+                public void run()
+                {
+                    if (!printedHeader)
+                    {
+                        System.out.println("Difference found in attribute or operation");
+                        printedHeader = true;
+                    }
+                }
+            };
+
+            for (String key : objectNames.shared)
+            {
+                Info leftInfo = left.get(key);
+                Info rightInfo = right.get(key);
+                DiffResult<Attribute> attributes = diff(leftInfo.attributeSet(), rightInfo.attributeSet(), attribute -> {
+                    for (CliPattern p : excludeAttributes)
+                    {
+                        if (p.pattern.matcher(attribute.name).matches())
+                            return false;
+                    }
+                    return true;
+                });
+                if (!ignoreMissingRight && !attributes.notInRight.isEmpty())
+                {
+                    printHeader.run();
+                    System.out.println(key + "\tattribute not in right:");
+                    printSet(1, attributes.notInRight);
+                }
+                if (!ignoreMissingLeft && !attributes.notInLeft.isEmpty())
+                {
+                    printHeader.run();
+                    System.out.println(key + "\tattribute not in left:");
+                    printSet(1, attributes.notInLeft);
+                }
+
+                DiffResult<Operation> operations = diff(leftInfo.operationSet(), rightInfo.operationSet(), operation -> {
+                    for (CliPattern p : excludeOperations)
+                    {
+                        if (p.pattern.matcher(operation.name).matches())
+                            return false;
+                    }
+                    return true;
+                });
+                if (!ignoreMissingRight && !operations.notInRight.isEmpty())
+                {
+                    printHeader.run();
+                    System.out.println(key + "\toperation not in right:");
+                    printSet(1, operations.notInRight, (sb, o) ->
+                                                       rightInfo.getOperation(o.name).ifPresent(match ->
+                                                                                                sb.append("\t").append("similar in right: ").append(match)));
+                }
+                if (!ignoreMissingLeft && !operations.notInLeft.isEmpty())
+                {
+                    printHeader.run();
+                    System.out.println(key + "\toperation not in left:");
+                    printSet(1, operations.notInLeft, (sb, o) ->
+                                                      leftInfo.getOperation(o.name).ifPresent(match ->
+                                                                                              sb.append("\t").append("similar in left: ").append(match)));
+                }
+            }
+        }
+
+        private static <T extends Comparable<T>> void printSet(int indent, Set<T> set)
+        {
+            printSet(indent, set, (i1, i2) -> {});
+        }
+
+        private static <T extends Comparable<T>> void printSet(int indent, Set<T> set, BiConsumer<StringBuilder, T> fn)
+        {
+            StringBuilder sb = new StringBuilder();
+            for (T t : new TreeSet<>(set))
+            {
+                sb.setLength(0);
+                for (int i = 0; i < indent; i++)
+                    sb.append('\t');
+                sb.append(t);
+                fn.accept(sb, t);
+                System.out.println(sb);
+            }
+        }
+
+        private static <T> DiffResult<T> diff(Set<T> left, Set<T> right, Predicate<T> fn)
+        {
+            left = Sets.filter(left, fn);
+            right = Sets.filter(right, fn);
+            return new DiffResult<>(Sets.difference(left, right), Sets.difference(right, left), Sets.intersection(left, right));
+        }
+
+        private static final class DiffResult<T>
+        {
+            private final SetView<T> notInRight;
+            private final SetView<T> notInLeft;
+            private final SetView<T> shared;
+
+            private DiffResult(SetView<T> notInRight, SetView<T> notInLeft, SetView<T> shared)
+            {
+                this.notInRight = notInRight;
+                this.notInLeft = notInLeft;
+                this.shared = shared;
+            }
+        }
+
+        public enum Format
+        {
+            json
+            {
+                Map<String, Info> load(InputStream input) throws IOException
+                {
+                    ObjectMapper mapper = new ObjectMapper();
+                    return mapper.readValue(input, new TypeReference<Map<String, Info>>() {});
+                }
+            },
+            yaml
+            {
+                Map<String, Info> load(InputStream input) throws IOException
+                {
+                    Yaml yaml = new Yaml(new CustomConstructor());
+                    return (Map<String, Info>) yaml.load(input);
+                }
+            };
+
+            abstract Map<String, Info> load(InputStream input) throws IOException;
+        }
+
+        private static final class CustomConstructor extends Constructor
+        {
+            private static final String ROOT = "__root__";
+            private static final TypeDescription INFO_TYPE = new TypeDescription(Info.class);
+
+            public CustomConstructor()
+            {
+                this.rootTag = new Tag(ROOT);
+                this.addTypeDescription(INFO_TYPE);
+            }
+
+            protected Object constructObject(Node node)
+            {
+                if (ROOT.equals(node.getTag().getValue()) && node instanceof MappingNode)
+                {
+                    MappingNode mn = (MappingNode) node;
+                    return mn.getValue().stream()
+                                .collect(Collectors.toMap(t -> super.constructObject(t.getKeyNode()),
+                                                          t -> {
+                                                              Node child = t.getValueNode();
+                                                              child.setType(INFO_TYPE.getType());
+                                                              return super.constructObject(child);
+                                                          }));
+                }
+                else
+                {
+                    return super.constructObject(node);
+                }
+            }
+        }
+    }
+
+    private static Map<String, Info> load(JMXServiceURL url) throws IOException, MalformedObjectNameException, IntrospectionException, InstanceNotFoundException, ReflectionException
+    {
+        try (JMXConnector jmxc = JMXConnectorFactory.connect(url, null))
+        {
+            MBeanServerConnection mbsc = jmxc.getMBeanServerConnection();
+
+            Map<String, Info> map = new TreeMap<>();
+            for (String pkg : new TreeSet<>(METRIC_PACKAGES))
+            {
+                Set<ObjectName> metricNames = new TreeSet<>(mbsc.queryNames(new ObjectName(pkg + ":*"), null));
+                for (ObjectName name : metricNames)
+                {
+                    if (mbsc.isRegistered(name))
+                    {
+                        MBeanInfo info = mbsc.getMBeanInfo(name);
+                        map.put(name.toString(), Info.from(info));
+                    }
+                }
+            }
+            return map;
+        }
+    }
+
+    private static String getAccess(MBeanAttributeInfo a)
+    {
+        String access;
+        if (a.isReadable())
+        {
+            if (a.isWritable())
+                access = "read/write";
+            else
+                access = "read-only";
+        }
+        else if (a.isWritable())
+            access = "write-only";
+        else
+            access = "no-access";
+        return access;
+    }
+
+    private static String normalizeType(String type)
+    {
+        switch (type)
+        {
+            case "[Z":
+                return "boolean[]";
+            case "[B":
+                return "byte[]";
+            case "[S":
+                return "short[]";
+            case "[I":
+                return "int[]";
+            case "[J":
+                return "long[]";
+            case "[F":
+                return "float[]";
+            case "[D":
+                return "double[]";
+            case "[C":
+                return "char[]";
+        }
+        if (type.startsWith("[L"))
+            return type.substring(2, type.length() - 1) + "[]"; // -1 will remove the ; at the end
+        return type;
+    }
+
+    private static final StringBuilder ROW_BUFFER = new StringBuilder();
+
+    private static void printRow(PrintStream out, String... args)
+    {
+        ROW_BUFFER.setLength(0);
+        ROW_BUFFER.append("\t\t");
+        for (String a : args)
+            ROW_BUFFER.append(a).append("\t");
+        out.println(ROW_BUFFER);
+    }
+
+    public static final class Info
+    {
+        private Attribute[] attributes;
+        private Operation[] operations;
+
+        public Info()
+        {
+        }
+
+        public Info(Attribute[] attributes, Operation[] operations)
+        {
+            this.attributes = attributes;
+            this.operations = operations;
+        }
+
+        private static Info from(MBeanInfo info)
+        {
+            Attribute[] attributes = Stream.of(info.getAttributes())
+                                           .sorted(Comparator.comparing(MBeanFeatureInfo::getName))
+                                           .map(Attribute::from)
+                                           .toArray(Attribute[]::new);
+
+            Operation[] operations = Stream.of(info.getOperations())
+                                           .sorted(OPERATOR_COMPARATOR)
+                                           .map(Operation::from)
+                                           .toArray(Operation[]::new);
+            return new Info(attributes, operations);
+        }
+
+        public Attribute[] getAttributes()
+        {
+            return attributes;
+        }
+
+        public void setAttributes(Attribute[] attributes)
+        {
+            this.attributes = attributes;
+        }
+
+        public Set<String> attributeNames()
+        {
+            return Stream.of(attributes).map(a -> a.name).collect(Collectors.toSet());
+        }
+
+        public Set<Attribute> attributeSet()
+        {
+            return new HashSet<>(Arrays.asList(attributes));
+        }
+
+        public Operation[] getOperations()
+        {
+            return operations;
+        }
+
+        public void setOperations(Operation[] operations)
+        {
+            this.operations = operations;
+        }
+
+        public Set<String> operationNames()
+        {
+            return Stream.of(operations).map(o -> o.name).collect(Collectors.toSet());
+        }
+
+        public Set<Operation> operationSet()
+        {
+            return new HashSet<>(Arrays.asList(operations));
+        }
+
+        public Optional<Attribute> getAttribute(String name)
+        {
+            return Stream.of(attributes).filter(a -> a.name.equals(name)).findFirst();
+        }
+
+        public Attribute getAttributePresent(String name)
+        {
+            return getAttribute(name).orElseThrow(AssertionError::new);
+        }
+
+        public Optional<Operation> getOperation(String name)
+        {
+            return Stream.of(operations).filter(o -> o.name.equals(name)).findFirst();
+        }
+
+        public Operation getOperationPresent(String name)
+        {
+            return getOperation(name).orElseThrow(AssertionError::new);
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            Info info = (Info) o;
+            return Arrays.equals(attributes, info.attributes) &&
+                   Arrays.equals(operations, info.operations);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            int result = Arrays.hashCode(attributes);
+            result = 31 * result + Arrays.hashCode(operations);
+            return result;
+        }
+    }
+
+    public static final class Attribute implements Comparable<Attribute>
+    {
+        private String name;
+        private String type;
+        private String access;
+
+        public Attribute()
+        {
+        }
+
+        public Attribute(String name, String type, String access)
+        {
+            this.name = name;
+            this.type = type;
+            this.access = access;
+        }
+
+        private static Attribute from(MBeanAttributeInfo info)
+        {
+            return new Attribute(info.getName(), normalizeType(info.getType()), JMXTool.getAccess(info));
+        }
+
+        public String getName()
+        {
+            return name;
+        }
+
+        public void setName(String name)
+        {
+            this.name = name;
+        }
+
+        public String getType()
+        {
+            return type;
+        }
+
+        public void setType(String type)
+        {
+            this.type = type;
+        }
+
+        public String getAccess()
+        {
+            return access;
+        }
+
+        public void setAccess(String access)
+        {
+            this.access = access;
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            Attribute attribute = (Attribute) o;
+            return Objects.equals(name, attribute.name) &&
+                   Objects.equals(type, attribute.type);
+        }
+
+        public int hashCode()
+        {
+            return Objects.hash(name, type);
+        }
+
+        public String toString()
+        {
+            return name + ": " + type;
+        }
+
+        public int compareTo(Attribute o)
+        {
+            int rc = name.compareTo(o.name);
+            if (rc != 0)
+                return rc;
+            return type.compareTo(o.type);
+        }
+    }
+
+    public static final class Operation implements Comparable<Operation>
+    {
+        private String name;
+        private Parameter[] parameters;
+        private String returnType;
+
+        public Operation()
+        {
+        }
+
+        public Operation(String name, Parameter[] parameters, String returnType)
+        {
+            this.name = name;
+            this.parameters = parameters;
+            this.returnType = returnType;
+        }
+
+        private static Operation from(MBeanOperationInfo info)
+        {
+            Parameter[] params = Stream.of(info.getSignature()).map(Parameter::from).toArray(Parameter[]::new);
+            return new Operation(info.getName(), params, normalizeType(info.getReturnType()));
+        }
+
+        public String getName()
+        {
+            return name;
+        }
+
+        public void setName(String name)
+        {
+            this.name = name;
+        }
+
+        public Parameter[] getParameters()
+        {
+            return parameters;
+        }
+
+        public void setParameters(Parameter[] parameters)
+        {
+            this.parameters = parameters;
+        }
+
+        public List<String> parameterTypes()
+        {
+            return Stream.of(parameters).map(p -> p.type).collect(Collectors.toList());
+        }
+
+        public String getReturnType()
+        {
+            return returnType;
+        }
+
+        public void setReturnType(String returnType)
+        {
+            this.returnType = returnType;
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            Operation operation = (Operation) o;
+            return Objects.equals(name, operation.name) &&
+                   Arrays.equals(parameters, operation.parameters) &&
+                   Objects.equals(returnType, operation.returnType);
+        }
+
+        public int hashCode()
+        {
+            int result = Objects.hash(name, returnType);
+            result = 31 * result + Arrays.hashCode(parameters);
+            return result;
+        }
+
+        public String toString()
+        {
+            return name + Stream.of(parameters).map(Parameter::toString).collect(Collectors.joining(", ", "(", ")")) + ": " + returnType;
+        }
+
+        public int compareTo(Operation o)
+        {
+            int rc = name.compareTo(o.name);
+            if (rc != 0)
+                return rc;
+            rc = Integer.compare(parameters.length, o.parameters.length);
+            if (rc != 0)
+                return rc;
+            for (int i = 0; i < parameters.length; i++)
+            {
+                rc = parameters[i].type.compareTo(o.parameters[i].type);
+                if (rc != 0)
+                    return rc;
+            }
+            return returnType.compareTo(o.returnType);
+        }
+    }
+
+    public static final class Parameter
+    {
+        private String name;
+        private String type;
+
+        public Parameter()
+        {
+        }
+
+        public Parameter(String name, String type)
+        {
+            this.name = name;
+            this.type = type;
+        }
+
+        private static Parameter from(MBeanParameterInfo info)
+        {
+            return new Parameter(info.getName(), normalizeType(info.getType()));
+        }
+
+        public String getName()
+        {
+            return name;
+        }
+
+        public void setName(String name)
+        {
+            this.name = name;
+        }
+
+        public String getType()
+        {
+            return type;
+        }
+
+        public void setType(String type)
+        {
+            this.type = type;
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            Parameter parameter = (Parameter) o;
+            return Objects.equals(type, parameter.type);
+        }
+
+        public int hashCode()
+        {
+            return Objects.hash(type);
+        }
+
+        public String toString()
+        {
+            return name + ": " + type;
+        }
+    }
+
+    public static final class CliPattern
+    {
+        private final Pattern pattern;
+
+        public CliPattern(String pattern)
+        {
+            this.pattern = Pattern.compile(pattern);
+        }
+    }
+
+    public static void main(String[] args) throws Exception
+    {
+        Cli.CliBuilder<Callable<Void>> builder = Cli.builder("jmxtool");
+        builder.withDefaultCommand(Help.class);
+        builder.withCommands(Help.class, Dump.class, Diff.class);
+
+        Cli<Callable<Void>> parser = builder.build();
+        Callable<Void> command = parser.parse(args);
+        command.call();
+    }
+}
diff --git a/test/data/jmxdump/cassandra-3.0-jmx.yaml b/test/data/jmxdump/cassandra-3.0-jmx.yaml
new file mode 100644
index 0000000..0a76b55
--- /dev/null
+++ b/test/data/jmxdump/cassandra-3.0-jmx.yaml
@@ -0,0 +1,48799 @@
+org.apache.cassandra.db:type=BatchlogManager:
+  attributes:
+  - {access: read-only, name: TotalBatchesReplayed, type: long}
+  operations:
+  - name: countAllBatches
+    parameters: []
+    returnType: int
+  - name: forceBatchlogReplay
+    parameters: []
+    returnType: void
+org.apache.cassandra.db:type=BlacklistedDirectories:
+  attributes:
+  - {access: read-only, name: UnreadableDirectories, type: java.util.Set}
+  - {access: read-only, name: UnwritableDirectories, type: java.util.Set}
+  operations: []
+org.apache.cassandra.db:type=Caches:
+  attributes:
+  - {access: write-only, name: CounterCacheCapacityInMB, type: long}
+  - {access: read/write, name: CounterCacheKeysToSave, type: int}
+  - {access: read/write, name: CounterCacheSavePeriodInSeconds, type: int}
+  - {access: write-only, name: KeyCacheCapacityInMB, type: long}
+  - {access: read/write, name: KeyCacheKeysToSave, type: int}
+  - {access: read/write, name: KeyCacheSavePeriodInSeconds, type: int}
+  - {access: write-only, name: RowCacheCapacityInMB, type: long}
+  - {access: read/write, name: RowCacheKeysToSave, type: int}
+  - {access: read/write, name: RowCacheSavePeriodInSeconds, type: int}
+  operations:
+  - name: invalidateCounterCache
+    parameters: []
+    returnType: void
+  - name: invalidateKeyCache
+    parameters: []
+    returnType: void
+  - name: invalidateRowCache
+    parameters: []
+    returnType: void
+  - name: saveCaches
+    parameters: []
+    returnType: void
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=cql_test_keyspace,columnfamily=table_00:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=IndexInfo:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=available_ranges:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=batches:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=batchlog:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=built_views:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=compaction_history:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=hints:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=local:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=paxos:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=peer_events:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=peers:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=range_xfers:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=size_estimates:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=sstable_activity:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system,columnfamily=views_builds_in_progress:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_auth,columnfamily=resource_role_permissons_index:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_auth,columnfamily=role_members:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_auth,columnfamily=role_permissions:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_auth,columnfamily=roles:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_distributed,columnfamily=parent_repair_history:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_distributed,columnfamily=repair_history:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_schema,columnfamily=aggregates:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_schema,columnfamily=columns:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_schema,columnfamily=dropped_columns:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_schema,columnfamily=functions:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_schema,columnfamily=indexes:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_schema,columnfamily=keyspaces:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_schema,columnfamily=tables:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_schema,columnfamily=triggers:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_schema,columnfamily=types:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_schema,columnfamily=views:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_traces,columnfamily=events:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=ColumnFamilies,keyspace=system_traces,columnfamily=sessions:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Commitlog:
+  attributes:
+  - {access: read-only, name: ActiveContentSize, type: long}
+  - {access: read-only, name: ActiveOnDiskSize, type: long}
+  - {access: read-only, name: ActiveSegmentCompressionRatios, type: java.util.Map}
+  - {access: read-only, name: ActiveSegmentNames, type: java.util.List}
+  - {access: read-only, name: ArchiveCommand, type: java.lang.String}
+  - {access: read-only, name: ArchivingSegmentNames, type: java.util.List}
+  - {access: read-only, name: RestoreCommand, type: java.lang.String}
+  - {access: read-only, name: RestoreDirectories, type: java.lang.String}
+  - {access: read-only, name: RestorePointInTime, type: long}
+  - {access: read-only, name: RestorePrecision, type: java.lang.String}
+  operations:
+  - name: recover
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+org.apache.cassandra.db:type=CompactionManager:
+  attributes:
+  - {access: read-only, name: CompactionHistory, type: javax.management.openmbean.TabularData}
+  - {access: read-only, name: CompactionSummary, type: java.util.List}
+  - {access: read-only, name: Compactions, type: java.util.List}
+  - {access: read/write, name: CoreCompactorThreads, type: int}
+  - {access: read/write, name: CoreValidationThreads, type: int}
+  - {access: read/write, name: MaximumCompactorThreads, type: int}
+  - {access: read/write, name: MaximumValidatorThreads, type: int}
+  operations:
+  - name: forceUserDefinedCompaction
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: stopCompaction
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: stopCompactionById
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+org.apache.cassandra.db:type=DisallowedDirectories:
+  attributes:
+  - {access: read-only, name: UnreadableDirectories, type: java.util.Set}
+  - {access: read-only, name: UnwritableDirectories, type: java.util.Set}
+  operations: []
+org.apache.cassandra.db:type=DynamicEndpointSnitch:
+  attributes:
+  - {access: read-only, name: BadnessThreshold, type: double}
+  - {access: read-only, name: ResetInterval, type: int}
+  - {access: read-only, name: Scores, type: java.util.Map}
+  - {access: read/write, name: Severity, type: double}
+  - {access: read-only, name: SubsnitchClassName, type: java.lang.String}
+  - {access: read-only, name: UpdateInterval, type: int}
+  operations:
+  - name: dumpTimings
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+org.apache.cassandra.db:type=EndpointSnitchInfo:
+  attributes:
+  - {access: read-only, name: Datacenter, type: java.lang.String}
+  - {access: read-only, name: Rack, type: java.lang.String}
+  - {access: read-only, name: SnitchName, type: java.lang.String}
+  operations:
+  - name: getDatacenter
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.lang.String
+  - name: getRack
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.lang.String
+org.apache.cassandra.db:type=HintedHandoffManager:
+  attributes: []
+  operations:
+  - name: deleteHintsForEndpoint
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: listEndpointsPendingHints
+    parameters: []
+    returnType: java.util.List
+  - name: pauseHintsDelivery
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: scheduleHintDelivery
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: truncateAllHints
+    parameters: []
+    returnType: void
+org.apache.cassandra.db:type=IndexSummaries:
+  attributes:
+  - {access: read-only, name: AverageIndexInterval, type: double}
+  - {access: read-only, name: IndexIntervals, type: java.util.Map}
+  - {access: read/write, name: MemoryPoolCapacityInMB, type: long}
+  - {access: read-only, name: MemoryPoolSizeInMB, type: double}
+  - {access: read/write, name: ResizeIntervalInMinutes, type: int}
+  operations:
+  - name: redistributeSummaries
+    parameters: []
+    returnType: void
+org.apache.cassandra.db:type=NativeAccess:
+  attributes:
+  - {access: read-only, name: Available, type: boolean}
+  - {access: read-only, name: MemoryLockable, type: boolean}
+  operations: []
+org.apache.cassandra.db:type=StorageProxy:
+  attributes:
+  - {access: read/write, name: CasContentionTimeout, type: java.lang.Long}
+  - {access: read-only, name: CheckForDuplicateRowsDuringCompaction, type: boolean}
+  - {access: read-only, name: CheckForDuplicateRowsDuringReads, type: boolean}
+  - {access: read/write, name: CounterWriteRpcTimeout, type: java.lang.Long}
+  - {access: read-only, name: HintedHandoffDisabledDCs, type: java.util.Set}
+  - {access: read/write, name: HintedHandoffEnabled, type: boolean}
+  - {access: read-only, name: HintsInProgress, type: int}
+  - {access: read/write, name: MaxHintWindow, type: int}
+  - {access: read/write, name: MaxHintsInProgress, type: int}
+  - {access: read/write, name: NativeTransportMaxConcurrentConnections, type: java.lang.Long}
+  - {access: read/write, name: OtcBacklogExpirationInterval, type: int}
+  - {access: read/write, name: RangeRpcTimeout, type: java.lang.Long}
+  - {access: read-only, name: ReadRepairAttempted, type: long}
+  - {access: read-only, name: ReadRepairRepairedBackground, type: long}
+  - {access: read-only, name: ReadRepairRepairedBlocking, type: long}
+  - {access: read/write, name: ReadRpcTimeout, type: java.lang.Long}
+  - {access: read/write, name: RpcTimeout, type: java.lang.Long}
+  - {access: read-only, name: SchemaVersions, type: java.util.Map}
+  - {access: read-only, name: SnapshotOnDuplicateRowDetectionEnabled, type: boolean}
+  - {access: read-only, name: TotalHints, type: long}
+  - {access: read/write, name: TruncateRpcTimeout, type: java.lang.Long}
+  - {access: read/write, name: WriteRpcTimeout, type: java.lang.Long}
+  operations:
+  - name: disableCheckForDuplicateRowsDuringCompaction
+    parameters: []
+    returnType: void
+  - name: disableCheckForDuplicateRowsDuringReads
+    parameters: []
+    returnType: void
+  - name: disableHintsForDC
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: disableSnapshotOnDuplicateRowDetection
+    parameters: []
+    returnType: void
+  - name: enableCheckForDuplicateRowsDuringCompaction
+    parameters: []
+    returnType: void
+  - name: enableCheckForDuplicateRowsDuringReads
+    parameters: []
+    returnType: void
+  - name: enableHintsForDC
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: enableSnapshotOnDuplicateRowDetection
+    parameters: []
+    returnType: void
+  - name: reloadTriggerClasses
+    parameters: []
+    returnType: void
+org.apache.cassandra.db:type=StorageService:
+  attributes:
+  - {access: read-only, name: AllDataFileLocations, type: 'java.lang.String[]'}
+  - {access: read/write, name: BatchSizeFailureThreshold, type: int}
+  - {access: read-only, name: BootstrapMode, type: boolean}
+  - {access: read/write, name: CachedReplicaRowsFailThreshold, type: int}
+  - {access: read/write, name: CachedReplicaRowsWarnThreshold, type: int}
+  - {access: read-only, name: ClusterName, type: java.lang.String}
+  - {access: read-only, name: CommitLogLocation, type: java.lang.String}
+  - {access: read/write, name: CompactionThroughputMbPerSec, type: int}
+  - {access: read-only, name: CurrentGenerationNumber, type: int}
+  - {access: read-only, name: DrainProgress, type: java.lang.String}
+  - {access: read-only, name: Drained, type: boolean}
+  - {access: read-only, name: Draining, type: boolean}
+  - {access: read-only, name: EndpointToHostId, type: java.util.Map}
+  - {access: read-only, name: GossipRunning, type: boolean}
+  - {access: write-only, name: HintedHandoffThrottleInKB, type: int}
+  - {access: read-only, name: HostIdMap, type: java.util.Map}
+  - {access: read-only, name: HostIdToEndpoint, type: java.util.Map}
+  - {access: read/write, name: IncrementalBackupsEnabled, type: boolean}
+  - {access: read-only, name: Initialized, type: boolean}
+  - {access: read/write, name: InterDCStreamThroughputMbPerSec, type: int}
+  - {access: read-only, name: Joined, type: boolean}
+  - {access: read-only, name: JoiningNodes, type: java.util.List}
+  - {access: read-only, name: Keyspaces, type: java.util.List}
+  - {access: read-only, name: LeavingNodes, type: java.util.List}
+  - {access: read-only, name: LiveNodes, type: java.util.List}
+  - {access: read-only, name: LoadMap, type: java.util.Map}
+  - {access: read-only, name: LoadString, type: java.lang.String}
+  - {access: read-only, name: LocalHostId, type: java.lang.String}
+  - {access: read-only, name: LoggingLevels, type: java.util.Map}
+  - {access: read-only, name: MaxNativeProtocolVersion, type: int}
+  - {access: read-only, name: MovingNodes, type: java.util.List}
+  - {access: read-only, name: NativeTransportRunning, type: boolean}
+  - {access: read-only, name: NonLocalStrategyKeyspaces, type: java.util.List}
+  - {access: read-only, name: NonSystemKeyspaces, type: java.util.List}
+  - {access: read-only, name: NotificationInfo, type: 'javax.management.MBeanNotificationInfo[]'}
+  - {access: read-only, name: OperationMode, type: java.lang.String}
+  - {access: read-only, name: Ownership, type: java.util.Map}
+  - {access: read-only, name: PartitionerName, type: java.lang.String}
+  - {access: read-only, name: RPCServerRunning, type: boolean}
+  - {access: read-only, name: ReleaseVersion, type: java.lang.String}
+  - {access: read-only, name: RemovalStatus, type: java.lang.String}
+  - {access: read/write, name: RepairSessionMaxTreeDepth, type: int}
+  - {access: read-only, name: SavedCachesLocation, type: java.lang.String}
+  - {access: read-only, name: SchemaVersion, type: java.lang.String}
+  - {access: read-only, name: SnapshotDetails, type: java.util.Map}
+  - {access: read-only, name: Starting, type: boolean}
+  - {access: read/write, name: StreamThroughputMbPerSec, type: int}
+  - {access: read-only, name: TokenToEndpointMap, type: java.util.Map}
+  - {access: read-only, name: Tokens, type: java.util.List}
+  - {access: read/write, name: TombstoneFailureThreshold, type: int}
+  - {access: read/write, name: TombstoneWarnThreshold, type: int}
+  - {access: read/write, name: TraceProbability, type: double}
+  - {access: read-only, name: UnreachableNodes, type: java.util.List}
+  operations:
+  - name: addNotificationListener
+    parameters:
+    - {name: p1, type: javax.management.NotificationListener}
+    - {name: p2, type: javax.management.NotificationFilter}
+    - {name: p3, type: java.lang.Object}
+    returnType: void
+  - name: bulkLoad
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: bulkLoadAsync
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.lang.String
+  - name: cleanupSizeEstimates
+    parameters: []
+    returnType: void
+  - name: clearSnapshot
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: 'java.lang.String[]'}
+    returnType: void
+  - name: decommission
+    parameters: []
+    returnType: void
+  - name: deliverHints
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: describeRingJMX
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: disableAutoCompaction
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: 'java.lang.String[]'}
+    returnType: void
+  - name: drain
+    parameters: []
+    returnType: void
+  - name: effectiveOwnership
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.Map
+  - name: enableAutoCompaction
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: 'java.lang.String[]'}
+    returnType: void
+  - name: forceKeyspaceCleanup
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: 'java.lang.String[]'}
+    returnType: int
+  - name: forceKeyspaceCleanup
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: java.lang.String}
+    - {name: p3, type: 'java.lang.String[]'}
+    returnType: int
+  - name: forceKeyspaceCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    - {name: p2, type: java.lang.String}
+    - {name: p3, type: 'java.lang.String[]'}
+    returnType: void
+  - name: forceKeyspaceFlush
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: 'java.lang.String[]'}
+    returnType: void
+  - name: forceRemoveCompletion
+    parameters: []
+    returnType: void
+  - name: forceRepairAsync
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: boolean}
+    - {name: p3, type: boolean}
+    - {name: p4, type: boolean}
+    - {name: p5, type: boolean}
+    - {name: p6, type: 'java.lang.String[]'}
+    returnType: int
+  - name: forceRepairAsync
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: boolean}
+    - {name: p3, type: java.util.Collection}
+    - {name: p4, type: java.util.Collection}
+    - {name: p5, type: boolean}
+    - {name: p6, type: boolean}
+    - {name: p7, type: 'java.lang.String[]'}
+    returnType: int
+  - name: forceRepairAsync
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    - {name: p3, type: java.util.Collection}
+    - {name: p4, type: java.util.Collection}
+    - {name: p5, type: boolean}
+    - {name: p6, type: boolean}
+    - {name: p7, type: 'java.lang.String[]'}
+    returnType: int
+  - name: forceRepairRangeAsync
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.lang.String}
+    - {name: p3, type: java.lang.String}
+    - {name: p4, type: boolean}
+    - {name: p5, type: boolean}
+    - {name: p6, type: boolean}
+    - {name: p7, type: 'java.lang.String[]'}
+    returnType: int
+  - name: forceRepairRangeAsync
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.lang.String}
+    - {name: p3, type: java.lang.String}
+    - {name: p4, type: int}
+    - {name: p5, type: java.util.Collection}
+    - {name: p6, type: java.util.Collection}
+    - {name: p7, type: boolean}
+    - {name: p8, type: 'java.lang.String[]'}
+    returnType: int
+  - name: forceRepairRangeAsync
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.lang.String}
+    - {name: p3, type: java.lang.String}
+    - {name: p4, type: boolean}
+    - {name: p5, type: java.util.Collection}
+    - {name: p6, type: java.util.Collection}
+    - {name: p7, type: boolean}
+    - {name: p8, type: 'java.lang.String[]'}
+    returnType: int
+  - name: forceTerminateAllRepairSessions
+    parameters: []
+    returnType: void
+  - name: getNaturalEndpoints
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.nio.ByteBuffer}
+    returnType: java.util.List
+  - name: getNaturalEndpoints
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.lang.String}
+    - {name: p3, type: java.lang.String}
+    returnType: java.util.List
+  - name: getPendingRangeToEndpointMap
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.Map
+  - name: getRangeToEndpointMap
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.Map
+  - name: getRangeToRpcaddressMap
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.Map
+  - name: getTokens
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: joinRing
+    parameters: []
+    returnType: void
+  - name: loadNewSSTables
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.lang.String}
+    returnType: void
+  - name: move
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: rebuild
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: rebuildSecondaryIndex
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.lang.String}
+    - {name: p3, type: 'java.lang.String[]'}
+    returnType: void
+  - name: refreshSizeEstimates
+    parameters: []
+    returnType: void
+  - name: reloadLocalSchema
+    parameters: []
+    returnType: void
+  - name: removeNode
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: removeNotificationListener
+    parameters:
+    - {name: p1, type: javax.management.NotificationListener}
+    returnType: void
+  - name: removeNotificationListener
+    parameters:
+    - {name: p1, type: javax.management.NotificationListener}
+    - {name: p2, type: javax.management.NotificationFilter}
+    - {name: p3, type: java.lang.Object}
+    returnType: void
+  - name: repairAsync
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.util.Map}
+    returnType: int
+  - name: rescheduleFailedDeletions
+    parameters: []
+    returnType: void
+  - name: resetLocalSchema
+    parameters: []
+    returnType: void
+  - name: resumeBootstrap
+    parameters: []
+    returnType: boolean
+  - name: sampleKeyRange
+    parameters: []
+    returnType: java.util.List
+  - name: scrub
+    parameters:
+    - {name: p1, type: boolean}
+    - {name: p2, type: boolean}
+    - {name: p3, type: java.lang.String}
+    - {name: p4, type: 'java.lang.String[]'}
+    returnType: int
+  - name: scrub
+    parameters:
+    - {name: p1, type: boolean}
+    - {name: p2, type: boolean}
+    - {name: p3, type: boolean}
+    - {name: p4, type: java.lang.String}
+    - {name: p5, type: 'java.lang.String[]'}
+    returnType: int
+  - name: scrub
+    parameters:
+    - {name: p1, type: boolean}
+    - {name: p2, type: boolean}
+    - {name: p3, type: boolean}
+    - {name: p4, type: int}
+    - {name: p5, type: java.lang.String}
+    - {name: p6, type: 'java.lang.String[]'}
+    returnType: int
+  - name: scrub
+    parameters:
+    - {name: p1, type: boolean}
+    - {name: p2, type: boolean}
+    - {name: p3, type: boolean}
+    - {name: p4, type: boolean}
+    - {name: p5, type: int}
+    - {name: p6, type: java.lang.String}
+    - {name: p7, type: 'java.lang.String[]'}
+    returnType: int
+  - name: setLoggingLevel
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.lang.String}
+    returnType: void
+  - name: startGossiping
+    parameters: []
+    returnType: void
+  - name: startNativeTransport
+    parameters: []
+    returnType: void
+  - name: startRPCServer
+    parameters: []
+    returnType: void
+  - name: stopDaemon
+    parameters: []
+    returnType: void
+  - name: stopGossiping
+    parameters: []
+    returnType: void
+  - name: stopNativeTransport
+    parameters: []
+    returnType: void
+  - name: stopRPCServer
+    parameters: []
+    returnType: void
+  - name: takeMultipleTableSnapshot
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: 'java.lang.String[]'}
+    returnType: void
+  - name: takeSnapshot
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: 'java.lang.String[]'}
+    returnType: void
+  - name: takeTableSnapshot
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.lang.String}
+    - {name: p3, type: java.lang.String}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+  - name: truncate
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.lang.String}
+    returnType: void
+  - name: updateSnitch
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: java.lang.Boolean}
+    - {name: p3, type: java.lang.Integer}
+    - {name: p4, type: java.lang.Integer}
+    - {name: p5, type: java.lang.Double}
+    returnType: void
+  - name: upgradeSSTables
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: boolean}
+    - {name: p3, type: 'java.lang.String[]'}
+    returnType: int
+  - name: upgradeSSTables
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: boolean}
+    - {name: p3, type: int}
+    - {name: p4, type: 'java.lang.String[]'}
+    returnType: int
+  - name: verify
+    parameters:
+    - {name: p1, type: boolean}
+    - {name: p2, type: java.lang.String}
+    - {name: p3, type: 'java.lang.String[]'}
+    returnType: int
+org.apache.cassandra.db:type=Tables,keyspace=cql_test_keyspace,table=table_00:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=IndexInfo:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=available_ranges:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=batches:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=batchlog:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=built_views:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=compaction_history:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=hints:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=local:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=paxos:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=peer_events:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=peers:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=range_xfers:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=size_estimates:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=sstable_activity:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system,table=views_builds_in_progress:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_auth,table=resource_role_permissons_index:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_auth,table=role_members:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_auth,table=role_permissions:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_auth,table=roles:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_distributed,table=parent_repair_history:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_distributed,table=repair_history:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_schema,table=aggregates:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_schema,table=columns:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_schema,table=dropped_columns:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_schema,table=functions:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_schema,table=indexes:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_schema,table=keyspaces:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_schema,table=tables:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_schema,table=triggers:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_schema,table=types:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_schema,table=views:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_traces,table=events:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.db:type=Tables,keyspace=system_traces,table=sessions:
+  attributes:
+  - {access: read-only, name: AutoCompactionDisabled, type: boolean}
+  - {access: read-only, name: BuiltIndexes, type: java.util.List}
+  - {access: read-only, name: ColumnFamilyName, type: java.lang.String}
+  - {access: read-only, name: CompactionDiskSpaceCheckEnabled, type: boolean}
+  - {access: read/write, name: CompactionParameters, type: java.util.Map}
+  - {access: read/write, name: CompactionParametersJson, type: java.lang.String}
+  - {access: read/write, name: CompressionParameters, type: java.util.Map}
+  - {access: write-only, name: CrcCheckChance, type: double}
+  - {access: read-only, name: DroppableTombstoneRatio, type: double}
+  - {access: read/write, name: MaximumCompactionThreshold, type: int}
+  - {access: read/write, name: MinimumCompactionThreshold, type: int}
+  - {access: read-only, name: SSTableCountPerLevel, type: 'int[]'}
+  - {access: read-only, name: TableName, type: java.lang.String}
+  - {access: read-only, name: UnleveledSSTables, type: int}
+  operations:
+  - name: beginLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: void
+  - name: compactionDiskSpaceCheck
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: estimateKeys
+    parameters: []
+    returnType: long
+  - name: finishLocalSampling
+    parameters:
+    - {name: p1, type: java.lang.String}
+    - {name: p2, type: int}
+    returnType: javax.management.openmbean.CompositeData
+  - name: forceMajorCompaction
+    parameters:
+    - {name: p1, type: boolean}
+    returnType: void
+  - name: getSSTablesForKey
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: java.util.List
+  - name: loadNewSSTables
+    parameters: []
+    returnType: void
+  - name: setCompactionThresholds
+    parameters:
+    - {name: p1, type: int}
+    - {name: p2, type: int}
+    returnType: void
+  - name: trueSnapshotsSize
+    parameters: []
+    returnType: long
+org.apache.cassandra.hints:type=HintsService:
+  attributes: []
+  operations:
+  - name: deleteAllHints
+    parameters: []
+    returnType: void
+  - name: deleteAllHintsForEndpoint
+    parameters:
+    - {name: p1, type: java.lang.String}
+    returnType: void
+  - name: pauseDispatch
+    parameters: []
+    returnType: void
+  - name: resumeDispatch
+    parameters: []
+    returnType: void
+org.apache.cassandra.internal:type=AntiEntropyStage:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=CacheCleanupExecutor:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=CompactionExecutor:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=GossipStage:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=HintsDispatcher:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=InternalResponseStage:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=MemtableFlushWriter:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=MemtablePostFlush:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=MemtableReclaimMemory:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=MigrationStage:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=MiscStage:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=PendingRangeCalculator:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=Sampler:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=SecondaryIndexManagement:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.internal:type=ValidationExecutor:
+  attributes:
+  - {access: read/write, name: CoreThreads, type: int}
+  - {access: read/write, name: MaximumThreads, type: int}
+  operations: []
+org.apache.cassandra.metrics:type=BufferPool,name=Misses:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=BufferPool,name=Size:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=CQL,name=PreparedStatementsCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=CQL,name=PreparedStatementsEvicted:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=CQL,name=PreparedStatementsExecuted:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=CQL,name=PreparedStatementsRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=CQL,name=RegularStatementsExecuted:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=CounterCache,name=Capacity:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=CounterCache,name=Entries:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=CounterCache,name=FifteenMinuteHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=CounterCache,name=FiveMinuteHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=CounterCache,name=HitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=CounterCache,name=Hits:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=CounterCache,name=OneMinuteHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=CounterCache,name=Requests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=CounterCache,name=Size:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=KeyCache,name=Capacity:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=KeyCache,name=Entries:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=KeyCache,name=FifteenMinuteHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=KeyCache,name=FiveMinuteHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=KeyCache,name=HitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=KeyCache,name=Hits:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=KeyCache,name=OneMinuteHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=KeyCache,name=Requests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=KeyCache,name=Size:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=RowCache,name=Capacity:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=RowCache,name=Entries:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=RowCache,name=FifteenMinuteHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=RowCache,name=FiveMinuteHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=RowCache,name=HitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=RowCache,name=Hits:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=RowCache,name=OneMinuteHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=RowCache,name=Requests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Cache,scope=RowCache,name=Size:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Client,name=PausedConnections:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Client,name=RequestDiscarded:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Client,name=connectedNativeClients:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=Client,name=connectedThriftClients:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASRead,name=ConditionNotMet:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASRead,name=ContentionHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASRead,name=Failures:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASRead,name=Latency:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASRead,name=Timeouts:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASRead,name=TotalLatency:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASRead,name=Unavailables:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASRead,name=UnfinishedCommit:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASWrite,name=ConditionNotMet:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASWrite,name=ContentionHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASWrite,name=Failures:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASWrite,name=Latency:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASWrite,name=Timeouts:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASWrite,name=TotalLatency:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASWrite,name=Unavailables:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=CASWrite,name=UnfinishedCommit:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=RangeSlice,name=Failures:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=RangeSlice,name=Latency:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ClientRequest,scope=RangeSlice,name=Timeouts:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=RangeSlice,name=TotalLatency:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=RangeSlice,name=Unavailables:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=Read,name=Failures:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=Read,name=Latency:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ClientRequest,scope=Read,name=Timeouts:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=Read,name=TotalLatency:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=Read,name=Unavailables:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=ViewWrite,name=Failures:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=ViewWrite,name=Latency:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ClientRequest,scope=ViewWrite,name=Timeouts:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=ViewWrite,name=TotalLatency:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=ViewWrite,name=Unavailables:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=ViewWrite,name=ViewPendingMutations:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=ViewWrite,name=ViewReplicasAttempted:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=ViewWrite,name=ViewReplicasSuccess:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=ViewWrite,name=ViewWriteLatency:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ClientRequest,scope=Write,name=Failures:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=Write,name=Latency:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ClientRequest,scope=Write,name=Timeouts:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=Write,name=TotalLatency:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ClientRequest,scope=Write,name=Unavailables:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=AllMemtablesHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=AllMemtablesLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=AllMemtablesOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=BloomFilterDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=BloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=BloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=BloomFilterOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=ColUpdateTimeDeltaHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=CompressionMetadataOffHeapMemoryUsed
+: attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=CompressionRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=EstimatedColumnCountHistogram
+: attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=EstimatedRowCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=EstimatedRowSizeHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=IndexSummaryOffHeapMemoryUsed
+: attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=KeyCacheHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=LiveDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=LiveSSTableCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=LiveScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=MaxRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=MeanRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=MemtableColumnsCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=MemtableLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=MemtableOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=MemtableOnHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=MemtableSwitchCount:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=MinRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=PendingCompactions:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=PendingFlushes:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=ReadRepairRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=RecentBloomFilterFalsePositives
+: attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=RecentBloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=ReplicaFilteringProtectionRequests
+: attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=ReplicaFilteringProtectionRowsCachedPerQuery
+: attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=RowCacheHit:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=RowCacheHitOutOfRange:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=RowCacheMiss:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=SSTablesPerReadHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=ShortReadProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=SnapshotsSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=SpeculativeRetries:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=TombstoneScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=TotalDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=ViewLockAcquireTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=cql_test_keyspace,scope=table_00,name=ViewReadTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=AllMemtablesHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=AllMemtablesLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=AllMemtablesOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BloomFilterDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=BloomFilterOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=ColUpdateTimeDeltaHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=CompressionMetadataOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=CompressionRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=EstimatedColumnCountHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=EstimatedRowCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=EstimatedRowSizeHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=IndexSummaryOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=KeyCacheHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=LiveDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=LiveSSTableCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=LiveScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=MaxRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=MeanRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=MemtableColumnsCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=MemtableLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=MemtableOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=MemtableOnHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=MemtableSwitchCount:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=MinRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=PendingCompactions:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=PendingFlushes:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=ReadRepairRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=RecentBloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=RecentBloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=ReplicaFilteringProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=ReplicaFilteringProtectionRowsCachedPerQuery
+: attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=RowCacheHit:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=RowCacheHitOutOfRange:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=RowCacheMiss:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=SSTablesPerReadHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=ShortReadProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=SnapshotsSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=SpeculativeRetries:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=TombstoneScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=TotalDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=ViewLockAcquireTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=IndexInfo,name=ViewReadTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=AllMemtablesHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=AllMemtablesLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=AllMemtablesOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BloomFilterDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=BloomFilterOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=ColUpdateTimeDeltaHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=CompressionMetadataOffHeapMemoryUsed
+: attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=CompressionRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=EstimatedColumnCountHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=EstimatedRowCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=EstimatedRowSizeHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=IndexSummaryOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=KeyCacheHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=LiveDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=LiveSSTableCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=LiveScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=MaxRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=MeanRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=MemtableColumnsCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=MemtableLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=MemtableOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=MemtableOnHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=MemtableSwitchCount:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=MinRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=PendingCompactions:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=PendingFlushes:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=ReadRepairRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=RecentBloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=RecentBloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=ReplicaFilteringProtectionRequests
+: attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=ReplicaFilteringProtectionRowsCachedPerQuery
+: attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=RowCacheHit:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=RowCacheHitOutOfRange:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=RowCacheMiss:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=SSTablesPerReadHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=ShortReadProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=SnapshotsSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=SpeculativeRetries:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=TombstoneScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=TotalDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=ViewLockAcquireTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=available_ranges,name=ViewReadTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=AllMemtablesHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=AllMemtablesLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=AllMemtablesOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BloomFilterDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=BloomFilterOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=ColUpdateTimeDeltaHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=CompressionMetadataOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=CompressionRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=EstimatedColumnCountHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=EstimatedRowCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=EstimatedRowSizeHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=IndexSummaryOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=KeyCacheHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=LiveDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=LiveSSTableCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=LiveScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=MaxRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=MeanRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=MemtableColumnsCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=MemtableLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=MemtableOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=MemtableOnHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=MemtableSwitchCount:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=MinRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=PendingCompactions:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=PendingFlushes:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=ReadRepairRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=RecentBloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=RecentBloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=ReplicaFilteringProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=ReplicaFilteringProtectionRowsCachedPerQuery
+: attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=RowCacheHit:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=RowCacheHitOutOfRange:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=RowCacheMiss:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=SSTablesPerReadHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=ShortReadProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=SnapshotsSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=SpeculativeRetries:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=TombstoneScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=TotalDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=ViewLockAcquireTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batches,name=ViewReadTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=AllMemtablesHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=AllMemtablesLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=AllMemtablesOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=BloomFilterDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=BloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=BloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=BloomFilterOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=ColUpdateTimeDeltaHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=CompressionMetadataOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=CompressionRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=EstimatedColumnCountHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=EstimatedRowCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=EstimatedRowSizeHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=IndexSummaryOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=KeyCacheHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=LiveDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=LiveSSTableCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=LiveScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=MaxRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=MeanRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=MemtableColumnsCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=MemtableLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=MemtableOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=MemtableOnHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=MemtableSwitchCount:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=MinRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=PendingCompactions:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=PendingFlushes:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=ReadRepairRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=RecentBloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=RecentBloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=ReplicaFilteringProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=ReplicaFilteringProtectionRowsCachedPerQuery
+: attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=RowCacheHit:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=RowCacheHitOutOfRange:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=RowCacheMiss:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=SSTablesPerReadHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=ShortReadProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=SnapshotsSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=SpeculativeRetries:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=TombstoneScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=TotalDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=ViewLockAcquireTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=batchlog,name=ViewReadTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=AllMemtablesHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=AllMemtablesLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=AllMemtablesOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BloomFilterDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=BloomFilterOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=ColUpdateTimeDeltaHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=CompressionMetadataOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=CompressionRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=EstimatedColumnCountHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=EstimatedRowCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=EstimatedRowSizeHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=IndexSummaryOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=KeyCacheHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=LiveDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=LiveSSTableCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=LiveScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=MaxRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=MeanRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=MemtableColumnsCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=MemtableLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=MemtableOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=MemtableOnHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=MemtableSwitchCount:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=MinRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=PendingCompactions:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=PendingFlushes:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=ReadRepairRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=RecentBloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=RecentBloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=ReplicaFilteringProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=ReplicaFilteringProtectionRowsCachedPerQuery
+: attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=RowCacheHit:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=RowCacheHitOutOfRange:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=RowCacheMiss:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=SSTablesPerReadHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=ShortReadProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=SnapshotsSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=SpeculativeRetries:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=TombstoneScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=TotalDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=ViewLockAcquireTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=built_views,name=ViewReadTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=AllMemtablesHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=AllMemtablesLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=AllMemtablesOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BloomFilterDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=BloomFilterOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=ColUpdateTimeDeltaHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=CompressionMetadataOffHeapMemoryUsed
+: attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=CompressionRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=EstimatedColumnCountHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=EstimatedRowCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=EstimatedRowSizeHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=IndexSummaryOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=KeyCacheHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=LiveDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=LiveSSTableCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=LiveScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=MaxRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=MeanRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=MemtableColumnsCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=MemtableLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=MemtableOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=MemtableOnHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=MemtableSwitchCount:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=MinRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=PendingCompactions:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=PendingFlushes:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=ReadRepairRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=RecentBloomFilterFalsePositives
+: attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=RecentBloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=ReplicaFilteringProtectionRequests
+: attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=ReplicaFilteringProtectionRowsCachedPerQuery
+: attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=RowCacheHit:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=RowCacheHitOutOfRange:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=RowCacheMiss:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=SSTablesPerReadHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=ShortReadProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=SnapshotsSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=SpeculativeRetries:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=TombstoneScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=TotalDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=ViewLockAcquireTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=compaction_history,name=ViewReadTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=AllMemtablesHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=AllMemtablesLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=AllMemtablesOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=BloomFilterDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=BloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=BloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=BloomFilterOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=ColUpdateTimeDeltaHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=CompressionMetadataOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=CompressionRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=EstimatedColumnCountHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=EstimatedRowCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=EstimatedRowSizeHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=IndexSummaryOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=KeyCacheHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=LiveDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=LiveSSTableCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=LiveScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=MaxRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=MeanRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=MemtableColumnsCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=MemtableLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=MemtableOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=MemtableOnHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=MemtableSwitchCount:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=MinRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=PendingCompactions:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=PendingFlushes:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=ReadRepairRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=RecentBloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=RecentBloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=ReplicaFilteringProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=ReplicaFilteringProtectionRowsCachedPerQuery
+: attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=RowCacheHit:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=RowCacheHitOutOfRange:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=RowCacheMiss:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=SSTablesPerReadHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=ShortReadProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=SnapshotsSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=SpeculativeRetries:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=TombstoneScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=TotalDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=ViewLockAcquireTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=hints,name=ViewReadTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=AllMemtablesHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=AllMemtablesLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=AllMemtablesOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BloomFilterDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=BloomFilterOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=ColUpdateTimeDeltaHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=CompressionMetadataOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=CompressionRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=EstimatedColumnCountHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=EstimatedRowCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=EstimatedRowSizeHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=IndexSummaryOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=KeyCacheHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=LiveDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=LiveSSTableCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=LiveScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=MaxRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=MeanRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=MemtableColumnsCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=MemtableLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=MemtableOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=MemtableOnHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=MemtableSwitchCount:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=MinRowSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=PendingCompactions:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=PendingFlushes:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=ReadRepairRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=RecentBloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=RecentBloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=ReplicaFilteringProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+? org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=ReplicaFilteringProtectionRowsCachedPerQuery
+: attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=RowCacheHit:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=RowCacheHitOutOfRange:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=RowCacheMiss:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=SSTablesPerReadHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=ShortReadProtectionRequests:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=SnapshotsSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=SpeculativeRetries:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=TombstoneScannedHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=TotalDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Count, type: long}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=ViewLockAcquireTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=local,name=ViewReadTime:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: DurationUnit, type: java.lang.String}
+  - {access: read-only, name: FifteenMinuteRate, type: double}
+  - {access: read-only, name: FiveMinuteRate, type: double}
+  - {access: read-only, name: Max, type: double}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: MeanRate, type: double}
+  - {access: read-only, name: Min, type: double}
+  - {access: read-only, name: OneMinuteRate, type: double}
+  - {access: read-only, name: RateUnit, type: java.lang.String}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=AllMemtablesHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=AllMemtablesLiveDataSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=AllMemtablesOffHeapSize:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BloomFilterDiskSpaceUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BloomFilterFalsePositives:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BloomFilterFalseRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=BloomFilterOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=ColUpdateTimeDeltaHistogram:
+  attributes:
+  - {access: read-only, name: 50thPercentile, type: double}
+  - {access: read-only, name: 75thPercentile, type: double}
+  - {access: read-only, name: 95thPercentile, type: double}
+  - {access: read-only, name: 98thPercentile, type: double}
+  - {access: read-only, name: 999thPercentile, type: double}
+  - {access: read-only, name: 99thPercentile, type: double}
+  - {access: read-only, name: Count, type: long}
+  - {access: read-only, name: Max, type: long}
+  - {access: read-only, name: Mean, type: double}
+  - {access: read-only, name: Min, type: long}
+  - {access: read-only, name: StdDev, type: double}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+  - name: values
+    parameters: []
+    returnType: long[]
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=CompressionMetadataOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=CompressionRatio:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=EstimatedColumnCountHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=EstimatedRowCount:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=EstimatedRowSizeHistogram:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=IndexSummaryOffHeapMemoryUsed:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
+org.apache.cassandra.metrics:type=ColumnFamily,keyspace=system,scope=paxos,name=KeyCacheHitRate:
+  attributes:
+  - {access: read-only, name: Value, type: java.lang.Object}
+  operations:
+  - name: objectName
+    parameters: []
+    returnType: javax.management.ObjectName
... 205942 lines suppressed ...


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org