From a596b2d813f297cf1816a8ec03533c11444cd4e0 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Mon, 26 Feb 2024 20:06:23 +0100 Subject: [PATCH] CASSANDRA-14572 Expose all table metrics in virtual tables --- .build/build-compile-annotations.xml | 54 ++ .build/checkstyle_suppressions.xml | 1 + .gitignore | 1 + build.xml | 18 +- .../managing/operating/virtualtables.adoc | 60 +- ide/idea-iml-file.xml | 3 + ide/nbproject/project.xml | 6 + pylib/cqlshlib/cql3handling.py | 5 +- pylib/cqlshlib/test/test_cqlsh_completion.py | 8 +- .../db/virtual/BatchMetricsTable.java | 13 +- .../cassandra/db/virtual/CQLMetricsTable.java | 9 +- .../CollectionVirtualTableAdapter.java | 575 ++++++++++++++++++ .../db/virtual/SystemViewsKeyspace.java | 12 +- .../db/virtual/ThreadPoolsTable.java | 94 --- .../cassandra/db/virtual/VirtualKeyspace.java | 17 +- .../db/virtual/model/CounterMetricRow.java | 59 ++ .../db/virtual/model/GaugeMetricRow.java | 59 ++ .../db/virtual/model/HistogramMetricRow.java | 102 ++++ .../db/virtual/model/MeterMetricRow.java | 83 +++ .../db/virtual/model/MetricGroupRow.java | 50 ++ .../cassandra/db/virtual/model/MetricRow.java | 81 +++ .../db/virtual/model/ThreadPoolRow.java | 96 +++ .../db/virtual/model/TimerMetricRow.java | 83 +++ .../index/sai/metrics/AbstractMetrics.java | 4 +- .../io/sstable/format/big/RowIndexEntry.java | 3 +- .../cassandra/metrics/BatchMetrics.java | 4 +- .../cassandra/metrics/BufferPoolMetrics.java | 3 +- .../metrics/CASClientWriteRequestMetrics.java | 4 +- .../metrics/CIDRAuthorizerMetrics.java | 3 +- .../apache/cassandra/metrics/CQLMetrics.java | 3 +- .../cassandra/metrics/CacheMetrics.java | 3 +- .../metrics/CassandraMetricsRegistry.java | 541 +++++++++++++--- .../cassandra/metrics/ChunkCacheMetrics.java | 3 +- .../metrics/ClientMessageSizeMetrics.java | 2 +- .../cassandra/metrics/ClientMetrics.java | 28 +- .../metrics/ClientRequestMetrics.java | 3 +- .../metrics/ClientRequestSizeMetrics.java | 2 +- .../cassandra/metrics/CommitLogMetrics.java | 3 +- .../cassandra/metrics/CompactionMetrics.java | 3 +- .../cassandra/metrics/DenylistMetrics.java | 3 +- .../metrics/DroppedMessageMetrics.java | 2 +- .../metrics/HintedHandoffMetrics.java | 3 +- .../metrics/HintsServiceMetrics.java | 3 +- .../metrics/InternodeInboundMetrics.java | 3 +- .../metrics/InternodeOutboundMetrics.java | 2 +- .../cassandra/metrics/KeyspaceMetrics.java | 5 +- .../cassandra/metrics/LatencyMetrics.java | 17 +- .../cassandra/metrics/MessagingMetrics.java | 3 +- .../cassandra/metrics/PaxosMetrics.java | 3 +- .../cassandra/metrics/ReadRepairMetrics.java | 3 +- .../cassandra/metrics/StorageMetrics.java | 3 +- .../cassandra/metrics/StreamingMetrics.java | 2 +- .../apache/cassandra/metrics/TCMMetrics.java | 3 +- .../cassandra/metrics/TableMetrics.java | 50 +- .../cassandra/metrics/ThreadPoolMetrics.java | 7 +- .../metrics/TrieMemtableMetricsView.java | 11 +- .../metrics/UnweightedCacheMetrics.java | 3 +- .../cassandra/schema/SchemaConstants.java | 1 + .../cassandra/service/CassandraDaemon.java | 9 +- .../cassandra/utils/EstimatedHistogram.java | 6 + .../apache/cassandra/utils/FBUtilities.java | 3 + .../cassandra/utils/memory/MemtablePool.java | 4 +- .../cassandra/db/virtual/proc/Column.java | 54 ++ .../cassandra/db/virtual/proc/RowWalker.java | 43 ++ .../proc/SystemViewAnnotationProcessor.java | 320 ++++++++++ .../javax.annotation.processing.Processor | 18 + .../org/apache/cassandra/cql3/CQLTester.java | 230 +++++-- .../CollectionVirtualTableAdapterTest.java | 225 +++++++ .../db/virtual/model/CollectionEntry.java | 118 ++++ .../virtual/model/CollectionEntryTestRow.java | 95 +++ .../virtual/model/PartitionEntryTestRow.java | 55 ++ .../cassandra/metrics/BatchMetricsTest.java | 10 + .../cassandra/metrics/CQLMetricsTest.java | 33 +- .../metrics/CassandraMetricsRegistryTest.java | 125 +++- .../metrics/ClientRequestMetricsTest.java | 44 +- .../ClientRequestRowAndColumnMetricsTest.java | 4 + .../metrics/JmxVirtualTableMetricsTest.java | 324 ++++++++++ .../metrics/KeyspaceMetricsTest.java | 31 + .../cassandra/metrics/LatencyMetricsTest.java | 10 +- .../cassandra/metrics/TableMetricsTest.java | 21 + .../metrics/TrieMemtableMetricsTest.java | 52 +- .../cassandra/utils/FBUtilitiesTest.java | 3 +- 82 files changed, 3646 insertions(+), 419 deletions(-) create mode 100644 .build/build-compile-annotations.xml create mode 100644 src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java delete mode 100644 src/java/org/apache/cassandra/db/virtual/ThreadPoolsTable.java create mode 100644 src/java/org/apache/cassandra/db/virtual/model/CounterMetricRow.java create mode 100644 src/java/org/apache/cassandra/db/virtual/model/GaugeMetricRow.java create mode 100644 src/java/org/apache/cassandra/db/virtual/model/HistogramMetricRow.java create mode 100644 src/java/org/apache/cassandra/db/virtual/model/MeterMetricRow.java create mode 100644 src/java/org/apache/cassandra/db/virtual/model/MetricGroupRow.java create mode 100644 src/java/org/apache/cassandra/db/virtual/model/MetricRow.java create mode 100644 src/java/org/apache/cassandra/db/virtual/model/ThreadPoolRow.java create mode 100644 src/java/org/apache/cassandra/db/virtual/model/TimerMetricRow.java create mode 100644 test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/Column.java create mode 100644 test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/RowWalker.java create mode 100644 test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/SystemViewAnnotationProcessor.java create mode 100644 test/annotation-processor/src/main/resources/services/javax.annotation.processing.Processor create mode 100644 test/unit/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapterTest.java create mode 100644 test/unit/org/apache/cassandra/db/virtual/model/CollectionEntry.java create mode 100644 test/unit/org/apache/cassandra/db/virtual/model/CollectionEntryTestRow.java create mode 100644 test/unit/org/apache/cassandra/db/virtual/model/PartitionEntryTestRow.java create mode 100644 test/unit/org/apache/cassandra/metrics/JmxVirtualTableMetricsTest.java diff --git a/.build/build-compile-annotations.xml b/.build/build-compile-annotations.xml new file mode 100644 index 000000000000..d9021bdfe123 --- /dev/null +++ b/.build/build-compile-annotations.xml @@ -0,0 +1,54 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/.build/checkstyle_suppressions.xml b/.build/checkstyle_suppressions.xml index ed4d1443f7fc..cd351a85afb6 100644 --- a/.build/checkstyle_suppressions.xml +++ b/.build/checkstyle_suppressions.xml @@ -22,4 +22,5 @@ + diff --git a/.gitignore b/.gitignore index 16cc10a5ae47..63919c26e038 100644 --- a/.gitignore +++ b/.gitignore @@ -4,6 +4,7 @@ build build/ src/gen-java/ +test/gen-java/ src/resources/org/apache/cassandra/config/ logs/ data/ diff --git a/build.xml b/build.xml index 11b7254a145a..cc3d45b8acea 100644 --- a/build.xml +++ b/build.xml @@ -77,6 +77,7 @@ + @@ -391,6 +392,7 @@ + @@ -415,6 +417,7 @@ + @@ -529,13 +532,18 @@ + + + + + - @@ -1031,6 +1039,11 @@ + + + + + @@ -1040,6 +1053,7 @@ + @@ -1979,6 +1993,7 @@ + @@ -2081,4 +2096,5 @@ + diff --git a/doc/modules/cassandra/pages/managing/operating/virtualtables.adoc b/doc/modules/cassandra/pages/managing/operating/virtualtables.adoc index f4a5608507b2..a34475f6f140 100644 --- a/doc/modules/cassandra/pages/managing/operating/virtualtables.adoc +++ b/doc/modules/cassandra/pages/managing/operating/virtualtables.adoc @@ -24,10 +24,11 @@ Since a virtual table is not replicated the partitioner sorts in order of partit == Virtual Keyspaces -Apache Cassandra 4.0 has added two new keyspaces for virtual tables: +Apache Cassandra has the following virtual keyspaces: * `system_virtual_schema` -* `system_views`. +* `system_views` +* `system_metrics` The `system_virtual_schema` keyspace has three tables: `keyspaces`, `columns` and `tables` for the virtual keyspace, table, and column definitions, respectively. @@ -36,6 +37,10 @@ It is used by Cassandra internally and a user should not access it directly. The `system_views` keyspace contains the actual virtual tables. +The `system_metrics` keyspace contains the virtual tables of all the metrics available in Apache Cassandra +which can be used to monitor the health of the cluster. These metrics are split into several tables each +containing a different set of metrics corresponding to the type of metric or the component of the system. + == Virtual Table Limitations Before discussing virtual keyspaces and tables, note that virtual keyspaces and tables have some limitations. @@ -55,6 +60,57 @@ In fact, mutations for virtual and regular tables cannot occur in the same batch == Virtual Tables +=== Virtual Tables _system_metrics_ Keyspace + +The `system_metrics` virtual keyspace contains the virtual tables of all the metrics available in Apache Cassandra. +Currently, the `system_metrics` keyspace contains the following virtual tables (subject to change), which can be found +in the `system_metrics.all_groups` virtual table: + +[source, console] +---- +cqlsh> select * from system_metrics.all_groups ; + + group_name | virtual_table +-------------------+--------------------------- + Batch | batch_group + BufferPool | buffer_pool_group + CIDRAuthorizer | cidr_authorizer_group + CQL | cql_group + Cache | cache_group + ChunkCache | chunk_cache_group + Client | client_group + ClientMessageSize | client_message_size_group + ClientRequest | client_request_group + ClientRequestSize | client_request_size_group + ColumnFamily | column_family_group + CommitLog | commit_log_group + Compaction | compaction_group + Connection | connection_group + DroppedMessage | dropped_message_group + HintsService | hints_service_group + Index | index_group + IndexColumnFamily | index_column_family_group + IndexTable | index_table_group + InternodeInbound | internode_inbound_group + Keyspace | keyspace_group + MemtablePool | memtable_pool_group + Messaging | messaging_group + Paxos | paxos_group + ReadRepair | read_repair_group + Repair | repair_group + Storage | storage_group + StorageProxy | storage_proxy_group + Streaming | streaming_group + TCM | tcm_group + Table | table_group + ThreadPools | thread_pools_group + TrieMemtable | trie_memtable_group + +(33 rows) +---- + +=== Virtual Tables _system_views_ Keyspace + Each of the virtual tables in the `system_views` virtual keyspace contain different information. The following table describes the virtual tables: diff --git a/ide/idea-iml-file.xml b/ide/idea-iml-file.xml index 13e66fa61308..6604f385ad2b 100644 --- a/ide/idea-iml-file.xml +++ b/ide/idea-iml-file.xml @@ -25,6 +25,9 @@ + + + diff --git a/ide/nbproject/project.xml b/ide/nbproject/project.xml index 038b2b1cf906..785ce5116552 100644 --- a/ide/nbproject/project.xml +++ b/ide/nbproject/project.xml @@ -47,6 +47,12 @@ ${project.dir}/test/unit UTF-8 + + + java + ${project.dir}/test/gen-java + UTF-8 + java diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py index 252ebabe30bc..9d28486237e7 100644 --- a/pylib/cqlshlib/cql3handling.py +++ b/pylib/cqlshlib/cql3handling.py @@ -34,8 +34,9 @@ def __str__(self): SYSTEM_KEYSPACES = ('system', 'system_schema', 'system_traces', 'system_auth', 'system_distributed', 'system_views', - 'system_virtual_schema', 'system_cluster_metadata') -NONALTERBALE_KEYSPACES = ('system', 'system_schema', 'system_views', 'system_virtual_schema', 'system_cluster_metadata') + 'system_metrics', 'system_virtual_schema', 'system_cluster_metadata') +NONALTERBALE_KEYSPACES = ('system', 'system_schema', 'system_views', 'system_metrics', 'system_virtual_schema', + 'system_cluster_metadata') class Cql3ParsingRuleSet(CqlParsingRuleSet): diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py index c2a99dc7b53a..371af9384246 100644 --- a/pylib/cqlshlib/test/test_cqlsh_completion.py +++ b/pylib/cqlshlib/test/test_cqlsh_completion.py @@ -588,7 +588,7 @@ def test_complete_in_create_type(self): self.trycompletions('CREATE TYPE foo ', choices=['(', '.']) def test_complete_in_drop_type(self): - self.trycompletions('DROP TYPE ', choices=['IF', 'system_views.', + self.trycompletions('DROP TYPE ', choices=['IF', 'system_views.', 'system_metrics.', 'tags', 'system_traces.', 'system_distributed.', 'system_cluster_metadata.', 'phone_number', 'quote_udt', 'band_info_type', 'address', 'system.', 'system_schema.', 'system_auth.', 'system_virtual_schema.', self.cqlsh.keyspace + '.' @@ -895,7 +895,7 @@ def test_complete_in_truncate(self): def test_complete_in_use(self): self.trycompletions('US', immediate='E ') - self.trycompletions('USE ', choices=[self.cqlsh.keyspace, 'system', 'system_auth', + self.trycompletions('USE ', choices=[self.cqlsh.keyspace, 'system', 'system_auth', 'system_metrics', 'system_distributed', 'system_schema', 'system_traces', 'system_views', 'system_virtual_schema', 'system_cluster_metadata' ]) @@ -991,7 +991,7 @@ def test_complete_in_alter_table(self): 'dynamic_columns', 'twenty_rows_composite_table', 'utf8_with_special_chars', - 'system_traces.', 'songs', 'system_views.', + 'system_traces.', 'songs', 'system_views.', 'system_metrics.', 'system_virtual_schema.', 'system_schema.', 'system_distributed.', 'system_cluster_metadata.', @@ -1020,7 +1020,7 @@ def test_complete_in_alter_table(self): def test_complete_in_alter_type(self): self.trycompletions('ALTER TYPE I', immediate='F EXISTS ') self.trycompletions('ALTER TYPE ', choices=['IF', 'system_views.', - 'tags', 'system_traces.', 'system_distributed.', + 'tags', 'system_traces.', 'system_distributed.', 'system_metrics.', 'phone_number', 'quote_udt', 'band_info_type', 'address', 'system.', 'system_schema.', 'system_auth.', 'system_virtual_schema.', 'system_cluster_metadata.', self.cqlsh.keyspace + '.' ]) diff --git a/src/java/org/apache/cassandra/db/virtual/BatchMetricsTable.java b/src/java/org/apache/cassandra/db/virtual/BatchMetricsTable.java index 948f2a1e9c08..62e58fafa09a 100644 --- a/src/java/org/apache/cassandra/db/virtual/BatchMetricsTable.java +++ b/src/java/org/apache/cassandra/db/virtual/BatchMetricsTable.java @@ -26,10 +26,16 @@ import org.apache.cassandra.dht.LocalPartitioner; import org.apache.cassandra.metrics.BatchMetrics; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.ClientWarn; +/** + * @deprecated The "batch_metrics" virtual table is no longer used, and will be removed in a future release. Please, + * use either "system_metrics.batch_group", or "system_metrics.type_histogram" virtual tables instead. + */ +@Deprecated(since = "5.0") public class BatchMetricsTable extends AbstractVirtualTable { - + protected static final String TABLE_NAME = "batch_metrics"; private static final String PARTITIONS_PER_LOGGED_BATCH = "partitions_per_logged_batch"; private static final String PARTITIONS_PER_UNLOGGED_BATCH = "partitions_per_unlogged_batch"; private static final String PARTITIONS_PER_COUNTER_BATCH = "partitions_per_counter_batch"; @@ -40,7 +46,7 @@ public class BatchMetricsTable extends AbstractVirtualTable BatchMetricsTable(String keyspace) { - super(TableMetadata.builder(keyspace, "batch_metrics") + super(TableMetadata.builder(keyspace, TABLE_NAME) .comment("Metrics specific to batch statements") .kind(TableMetadata.Kind.VIRTUAL) .partitioner(new LocalPartitioner(UTF8Type.instance)) @@ -61,6 +67,9 @@ public DataSet data() addRow(result, PARTITIONS_PER_UNLOGGED_BATCH, metrics.partitionsPerUnloggedBatch.getSnapshot()); addRow(result, PARTITIONS_PER_COUNTER_BATCH, metrics.partitionsPerCounterBatch.getSnapshot()); + ClientWarn.instance.warn(String.format("The \"%s\" virtual table is deprecated. " + + "Please, use either \"%s.batch_group\", or \"%s.type_histogram\" virtual tables instead.", + TABLE_NAME, metadata.keyspace, metadata.keyspace)); return result; } diff --git a/src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java b/src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java index acd8947e2f8a..39e41fcc5d6b 100644 --- a/src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java +++ b/src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java @@ -26,8 +26,13 @@ import org.apache.cassandra.metrics.CQLMetrics; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.cql3.QueryProcessor; +import org.apache.cassandra.service.ClientWarn; - +/** + * @deprecated The "cql_metrics" virtual table is no longer used, and will be removed in a future release. Please, + * use "system_metrics.clq_group" virtual table instead. + */ +@Deprecated(since = "5.0") final class CQLMetricsTable extends AbstractVirtualTable { public static final String TABLE_NAME = "cql_metrics"; @@ -70,6 +75,8 @@ public DataSet data() addRow(result, PREPARED_STATEMENTS_RATIO, cqlMetrics.preparedStatementsRatio.getValue()); addRow(result, REGULAR_STATEMENTS_EXECUTED, cqlMetrics.regularStatementsExecuted.getCount()); + ClientWarn.instance.warn("The \"" + TABLE_NAME + "\" virtual table is deprecated. " + + "Please, use \"system_metrics.clq_group\" virtual table instead."); return result; } diff --git a/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java b/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java new file mode 100644 index 000000000000..b975f26a4001 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java @@ -0,0 +1,575 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual; + +import com.google.common.collect.AbstractIterator; +import com.google.common.collect.ImmutableMap; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.EmptyIterators; +import org.apache.cassandra.db.filter.ClusteringIndexFilter; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.ByteType; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.DoubleType; +import org.apache.cassandra.db.marshal.FloatType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.ShortType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.db.partitions.AbstractUnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.partitions.SingletonUnfilteredPartitionIterator; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.AbstractUnfilteredRowIterator; +import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.rows.BufferCell; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Rows; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.virtual.proc.Column; +import org.apache.cassandra.db.virtual.proc.RowWalker; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.Pair; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.EnumMap; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.regex.Pattern; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import static org.apache.cassandra.db.rows.Cell.NO_DELETION_TIME; +import static org.apache.cassandra.utils.FBUtilities.camelToSnake; + +/** + * This is a virtual table that iteratively builds rows using a data set provided by internal collection. + * Some metric views might be too large to fit in memory, for example, virtual tables that contain metrics + * for all the keyspaces registered in the cluster. Such a technique is also facilitates keeping the low + * memory footprint of the virtual tables in general. + *

+ * It doesn't require the input data set to be sorted, but it does require that the partition keys are + * provided in the order of the partitioner of the table metadata. + */ +public class CollectionVirtualTableAdapter implements VirtualTable +{ + private static final Pattern ONLY_ALPHABET_PATTERN = Pattern.compile("[^a-zA-Z1-9]"); + private static final List> knownAbbreviations = Arrays.asList(Pair.create("CAS", "Cas"), + Pair.create("CIDR", "Cidr")); + private static final Map, ? extends AbstractType> converters = ImmutableMap., AbstractType>builder() + .put(String.class, UTF8Type.instance) + .put(Integer.class, Int32Type.instance) + .put(Integer.TYPE, Int32Type.instance) + .put(Long.class, LongType.instance) + .put(Long.TYPE, LongType.instance) + .put(Float.class, FloatType.instance) + .put(Float.TYPE, FloatType.instance) + .put(Double.class, DoubleType.instance) + .put(Double.TYPE, DoubleType.instance) + .put(Boolean.class, BooleanType.instance) + .put(Boolean.TYPE, BooleanType.instance) + .put(Byte.class, ByteType.instance) + .put(Byte.TYPE, ByteType.instance) + .put(Short.class, ShortType.instance) + .put(Short.TYPE, ShortType.instance) + .put(UUID.class, UUIDType.instance) + .build(); + /** The map is used to avoid getting column metadata for each regular column for each row. */ + private final ConcurrentHashMap columnMetas = new ConcurrentHashMap<>(); + private final RowWalker walker; + private final Iterable data; + private final Function decorateKeyToRowExtractor; + private final TableMetadata metadata; + + private CollectionVirtualTableAdapter(String keySpaceName, + String tableName, + String description, + RowWalker walker, + Iterable data) + { + this(keySpaceName, tableName, description, walker, data, null); + } + + private CollectionVirtualTableAdapter(String keySpaceName, + String tableName, + String description, + RowWalker walker, + Iterable data, + Function keyToRowExtractor) + { + this.walker = walker; + this.data = data; + this.metadata = buildMetadata(keySpaceName, tableName, description, walker); + this.decorateKeyToRowExtractor = keyToRowExtractor; + } + + public static CollectionVirtualTableAdapter create( + String keySpaceName, + String rawTableName, + String description, + RowWalker walker, + Iterable container, + Function rowFunc) + { + return new CollectionVirtualTableAdapter<>(keySpaceName, + virtualTableNameStyle(rawTableName), + description, + walker, + () -> StreamSupport.stream(container.spliterator(), false) + .map(rowFunc).iterator()); + } + + public static CollectionVirtualTableAdapter createSinglePartitionedKeyFiltered( + String keySpaceName, + String rawTableName, + String description, + RowWalker walker, + Map map, + Predicate mapKeyFilter, + BiFunction rowConverter) + { + return createSinglePartitioned(keySpaceName, rawTableName, description, walker, map, mapKeyFilter, + Objects::nonNull, rowConverter); + } + + public static CollectionVirtualTableAdapter createSinglePartitionedValueFiltered( + String keySpaceName, + String rawTableName, + String description, + RowWalker walker, + Map map, + Predicate mapValueFilter, + BiFunction rowConverter) + { + return createSinglePartitioned(keySpaceName, rawTableName, description, walker, map, key -> true, + mapValueFilter, rowConverter); + } + + private static CollectionVirtualTableAdapter createSinglePartitioned( + String keySpaceName, + String rawTableName, + String description, + RowWalker walker, + Map map, + Predicate mapKeyFilter, + Predicate mapValueFilter, + BiFunction rowConverter) + { + assert walker.count(Column.Type.PARTITION_KEY) == 1 : "Partition key must be a single column"; + assert walker.count(Column.Type.CLUSTERING) == 0 : "Clustering columns are not supported"; + + AtomicReference> partitionKeyClass = new AtomicReference<>(); + walker.visitMeta(new RowWalker.MetadataVisitor() + { + @Override + public void accept(Column.Type type, String columnName, Class clazz) + { + if (type == Column.Type.PARTITION_KEY) + partitionKeyClass.set(clazz); + } + }); + + return new CollectionVirtualTableAdapter<>(keySpaceName, + virtualTableNameStyle(rawTableName), + description, + walker, + () -> map.entrySet() + .stream() + .filter(e -> mapKeyFilter.test(e.getKey())) + .filter(e -> mapValueFilter.test(e.getValue())) + .map(e -> rowConverter.apply(e.getKey(), e.getValue())) + .iterator(), + decoratedKey -> + { + K partitionKey = compose(converters.get(partitionKeyClass.get()), decoratedKey.getKey()); + boolean keyRequired = mapKeyFilter.test(partitionKey); + if (!keyRequired) + return null; + + C value = map.get(partitionKey); + return mapValueFilter.test(value) ? rowConverter.apply(partitionKey, value) : null; + }); + } + + public static String virtualTableNameStyle(String camel) + { + // Process sub names in the full metrics group name separately and then join them. + // For example: "ClientRequest.Write-EACH_QUORUM" will be converted to "client_request_write_each_quorum". + String[] subNames = ONLY_ALPHABET_PATTERN.matcher(camel).replaceAll(".").split("\\."); + return Arrays.stream(subNames) + .map(CollectionVirtualTableAdapter::camelToSnakeWithAbbreviations) + .reduce((a, b) -> a + '_' + b) + .orElseThrow(() -> new IllegalArgumentException("Invalid table name: " + camel)); + } + + private static String camelToSnakeWithAbbreviations(String camel) + { + Pattern pattern = Pattern.compile("^[A-Z1-9_]+$"); + // Contains only uppercase letters, numbers and underscores, so it's already snake case. + if (pattern.matcher(camel).matches()) + return camel.toLowerCase(); + + // Some special cases must be handled manually. + String modifiedCamel = camel; + for (Pair replacement : knownAbbreviations) + modifiedCamel = modifiedCamel.replace(replacement.left, replacement.right); + + return camelToSnake(modifiedCamel); + } + + private TableMetadata buildMetadata(String keyspaceName, String tableName, String description, RowWalker walker) + { + TableMetadata.Builder builder = TableMetadata.builder(keyspaceName, tableName) + .comment(description) + .kind(TableMetadata.Kind.VIRTUAL); + + List> partitionKeyTypes = new ArrayList<>(walker.count(Column.Type.PARTITION_KEY)); + walker.visitMeta( + new RowWalker.MetadataVisitor() + { + @Override + public void accept(Column.Type type, String columnName, Class clazz) + { + switch (type) + { + case PARTITION_KEY: + partitionKeyTypes.add(converters.get(clazz)); + builder.addPartitionKeyColumn(columnName, converters.get(clazz)); + break; + case CLUSTERING: + builder.addClusteringColumn(columnName, converters.get(clazz)); + break; + case REGULAR: + builder.addRegularColumn(columnName, converters.get(clazz)); + break; + default: + throw new IllegalStateException("Unknown column type: " + type); + } + } + }); + + if (partitionKeyTypes.size() == 1) + builder.partitioner(new LocalPartitioner(partitionKeyTypes.get(0))); + else if (partitionKeyTypes.size() > 1) + builder.partitioner(new LocalPartitioner(CompositeType.getInstance(partitionKeyTypes))); + + return builder.build(); + } + + /** {@inheritDoc} */ + @Override + public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringFilter, ColumnFilter columnFilter) + { + if (!data.iterator().hasNext()) + return EmptyIterators.unfilteredPartition(metadata); + + NavigableMap, Row> rows = new TreeMap<>(metadata.comparator); + Stream stream; + if (decorateKeyToRowExtractor == null) + { + // The benchmark shows that if we continuously read the data from the virtual table e.g. by metric name, + // then the parallel stream is slightly faster to get the first result, but for continuous reads it gives us + // a higher GC pressure. The sequential stream is slightly slower to get the first result, but it has the + // same throughput as the parallel stream, and it gives us less GC pressure. + // See the details in the benchmark: https://gist.github.com/Mmuzaf/80c73b7f9441ff21f6d22efe5746541a + stream = StreamSupport.stream(data.spliterator(), false) + .map(row -> makeRow(row, columnFilter)) + .filter(cr -> partitionKey.equals(cr.key.get())) + .filter(cr -> clusteringFilter.selects(cr.clustering)); + } + else + { + R row = decorateKeyToRowExtractor.apply(partitionKey); + if (row == null) + return EmptyIterators.unfilteredPartition(metadata); + stream = Stream.of(makeRow(row, columnFilter)); + } + + // If there are no clustering columns, we've found a unique partition that matches the partition key, + // so we can stop the stream without looping through all the rows. + if (walker.count(Column.Type.CLUSTERING) == 0) + stream.findFirst().ifPresent(cr -> rows.put(cr.clustering, cr.rowSup.get())); + else + stream.forEach(cr -> rows.put(cr.clustering, cr.rowSup.get())); + + return new SingletonUnfilteredPartitionIterator(new DataRowUnfilteredIterator(partitionKey, + clusteringFilter, + columnFilter, + rows)); + } + + /** {@inheritDoc} */ + @Override + public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter) + { + return createPartitionIterator(metadata, new AbstractIterator<>() + { + private final Iterator partitions = buildDataRangeIterator(dataRange, columnFilter); + + @Override + protected UnfilteredRowIterator computeNext() + { + return partitions.hasNext() ? partitions.next() : endOfData(); + } + + private Iterator buildDataRangeIterator(DataRange dataRange, ColumnFilter columnFilter) + { + NavigableMap, Row>> partitionMap = new ConcurrentSkipListMap<>(DecoratedKey.comparator); + StreamSupport.stream(data.spliterator(), true) + .map(row -> makeRow(row, columnFilter)) + .filter(cr -> dataRange.keyRange().contains(cr.key.get())) + .forEach(cr -> partitionMap.computeIfAbsent(cr.key.get(), key -> new TreeMap<>(metadata.comparator)) + .put(cr.clustering, cr.rowSup.get())); + return partitionMap.entrySet() + .stream() + .map(entry -> new DataRowUnfilteredIterator(entry.getKey(), + dataRange.clusteringIndexFilter(entry.getKey()), + columnFilter, + entry.getValue())) + .iterator(); + } + }); + } + + private class DataRowUnfilteredIterator extends AbstractUnfilteredRowIterator + { + private final Iterator rows; + + public DataRowUnfilteredIterator(DecoratedKey partitionKey, + ClusteringIndexFilter indexFilter, + ColumnFilter columnFilter, + NavigableMap, Row> data) + { + super(CollectionVirtualTableAdapter.this.metadata, + partitionKey, + DeletionTime.LIVE, + columnFilter.queriedColumns(), + Rows.EMPTY_STATIC_ROW, + indexFilter.isReversed(), + EncodingStats.NO_STATS); + this.rows = indexFilter.isReversed() ? data.descendingMap().values().iterator() : data.values().iterator(); + } + + @Override + protected Unfiltered computeNext() + { + return rows.hasNext() ? rows.next() : endOfData(); + } + } + + private CollectionRow makeRow(R row, ColumnFilter columnFilter) + { + assert metadata.partitionKeyColumns().size() == walker.count(Column.Type.PARTITION_KEY) : + "Invalid number of partition key columns"; + assert metadata.clusteringColumns().size() == walker.count(Column.Type.CLUSTERING) : + "Invalid number of clustering columns"; + + Map fiterable = new EnumMap<>(Column.Type.class); + fiterable.put(Column.Type.PARTITION_KEY, new Object[metadata.partitionKeyColumns().size()]); + if (walker.count(Column.Type.CLUSTERING) > 0) + fiterable.put(Column.Type.CLUSTERING, new Object[metadata.clusteringColumns().size()]); + + Map> cells = new HashMap<>(); + + walker.visitRow(row, new RowWalker.RowMetadataVisitor() + { + private int pIdx, cIdx = 0; + + @Override + public void accept(Column.Type type, String columnName, Class clazz, Supplier value) + { + switch (type) + { + case PARTITION_KEY: + fiterable.get(type)[pIdx++] = value.get(); + break; + case CLUSTERING: + fiterable.get(type)[cIdx++] = value.get(); + break; + case REGULAR: + { + if (columnFilter.equals(ColumnFilter.NONE)) + break; + + // Push down the column filter to the walker, so we don't have to process the value if it's not queried + ColumnMetadata cm = columnMetas.computeIfAbsent(columnName, name -> metadata.getColumn(ByteBufferUtil.bytes(name))); + if (columnFilter.queriedColumns().contains(cm)) + cells.put(cm, value); + + break; + } + default: + throw new IllegalStateException("Unknown column type: " + type); + } + } + }); + + return new CollectionRow(() -> makeRowKey(metadata, fiterable.get(Column.Type.PARTITION_KEY)), + makeRowClustering(metadata, fiterable.get(Column.Type.CLUSTERING)), + clustering -> + { + Row.Builder rowBuilder = BTreeRow.unsortedBuilder(); + rowBuilder.newRow(clustering); + cells.forEach((column, value) -> { + Object valueObj = value.get(); + if (valueObj == null) + return; + rowBuilder.addCell(BufferCell.live(column, NO_DELETION_TIME, decompose(column.type, valueObj))); + }); + return rowBuilder.build(); + }); + } + + private static class CollectionRow + { + private final Supplier key; + private final Clustering clustering; + private final Supplier rowSup; + + public CollectionRow(Supplier key, Clustering clustering, Function, Row> rowSup) + { + this.key = new ValueHolder<>(key); + this.clustering = clustering; + this.rowSup = new ValueHolder<>(() -> rowSup.apply(clustering)); + } + + private static class ValueHolder implements Supplier + { + private final Supplier delegate; + private volatile T value; + + public ValueHolder(Supplier delegate) + { + this.delegate = delegate; + } + + @Override + public T get() + { + return value == null ? value = delegate.get() : value; + } + } + } + + private static Clustering makeRowClustering(TableMetadata metadata, Object... clusteringValues) + { + if (clusteringValues == null || clusteringValues.length == 0) + return Clustering.EMPTY; + + ByteBuffer[] clusteringByteBuffers = new ByteBuffer[clusteringValues.length]; + for (int i = 0; i < clusteringValues.length; i++) + clusteringByteBuffers[i] = decompose(metadata.clusteringColumns().get(i).type, clusteringValues[i]); + return Clustering.make(clusteringByteBuffers); + } + + /** + * @param table the table metadata + * @param partitionKeyValues the partition key values + * @return the decorated key + */ + private static DecoratedKey makeRowKey(TableMetadata table, Object...partitionKeyValues) + { + ByteBuffer key; + if (partitionKeyValues.length > 1) + key = ((CompositeType)table.partitionKeyType).decompose(partitionKeyValues); + else + key = decompose(table.partitionKeyType, partitionKeyValues[0]); + return table.partitioner.decorateKey(key); + } + + private static UnfilteredPartitionIterator createPartitionIterator( + TableMetadata metadata, + Iterator partitions) + { + return new AbstractUnfilteredPartitionIterator() + { + public UnfilteredRowIterator next() + { + return partitions.next(); + } + + public boolean hasNext() + { + return partitions.hasNext(); + } + + public TableMetadata metadata() + { + return metadata; + } + }; + } + + @SuppressWarnings("unchecked") + private static ByteBuffer decompose(AbstractType type, T value) + { + return ((AbstractType) type).decompose(value); + } + + @SuppressWarnings("unchecked") + private static T compose(AbstractType type, ByteBuffer value) + { + return (T) type.compose(value); + } + + @Override + public TableMetadata metadata() + { + return metadata; + } + + @Override + public void apply(PartitionUpdate update) + { + throw new InvalidRequestException("Modification is not supported by table " + metadata); + } + + @Override + public void truncate() + { + throw new InvalidRequestException("Truncate is not supported by table " + metadata); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java index 7b7e28e30aaf..f248a17fe38f 100644 --- a/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/SystemViewsKeyspace.java @@ -18,9 +18,11 @@ package org.apache.cassandra.db.virtual; import com.google.common.collect.ImmutableList; - +import org.apache.cassandra.db.virtual.model.ThreadPoolRow; +import org.apache.cassandra.db.virtual.model.ThreadPoolRowWalker; import org.apache.cassandra.index.sai.virtual.StorageAttachedIndexTables; +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_VIEWS; public final class SystemViewsKeyspace extends VirtualKeyspace @@ -35,7 +37,13 @@ private SystemViewsKeyspace() .add(new SettingsTable(VIRTUAL_VIEWS)) .add(new SystemPropertiesTable(VIRTUAL_VIEWS)) .add(new SSTableTasksTable(VIRTUAL_VIEWS)) - .add(new ThreadPoolsTable(VIRTUAL_VIEWS)) + // Fully backward/forward compatible with the legace ThreadPoolsTable under the same "system_views.thread_pools" name. + .add(CollectionVirtualTableAdapter.create(VIRTUAL_VIEWS, + "thread_pools", + "Thread pool metrics for all thread pools", + new ThreadPoolRowWalker(), + Metrics.allThreadPoolMetrics(), + ThreadPoolRow::new)) .add(new InternodeOutboundTable(VIRTUAL_VIEWS)) .add(new InternodeInboundTable(VIRTUAL_VIEWS)) .add(new PendingHintsTable(VIRTUAL_VIEWS)) diff --git a/src/java/org/apache/cassandra/db/virtual/ThreadPoolsTable.java b/src/java/org/apache/cassandra/db/virtual/ThreadPoolsTable.java deleted file mode 100644 index 4bd4624d589e..000000000000 --- a/src/java/org/apache/cassandra/db/virtual/ThreadPoolsTable.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.db.virtual; - -import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.marshal.Int32Type; -import org.apache.cassandra.db.marshal.LongType; -import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.dht.LocalPartitioner; -import org.apache.cassandra.metrics.ThreadPoolMetrics; -import org.apache.cassandra.schema.TableMetadata; - -import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; - -final class ThreadPoolsTable extends AbstractVirtualTable -{ - private static final String NAME = "name"; - private static final String ACTIVE_TASKS = "active_tasks"; - private static final String ACTIVE_TASKS_LIMIT = "active_tasks_limit"; - private static final String PENDING_TASKS = "pending_tasks"; - private static final String COMPLETED_TASKS = "completed_tasks"; - private static final String BLOCKED_TASKS = "blocked_tasks"; - private static final String BLOCKED_TASKS_ALL_TIME = "blocked_tasks_all_time"; - private static final String CORE_POOL_SIZE = "core_pool_size"; - private static final String MAX_POOL_SIZE = "max_pool_size"; - private static final String MAX_TASKS_QUEUED = "max_tasks_queued"; - - ThreadPoolsTable(String keyspace) - { - super(TableMetadata.builder(keyspace, "thread_pools") - .kind(TableMetadata.Kind.VIRTUAL) - .partitioner(new LocalPartitioner(UTF8Type.instance)) - .addPartitionKeyColumn(NAME, UTF8Type.instance) - .addRegularColumn(ACTIVE_TASKS, Int32Type.instance) - .addRegularColumn(ACTIVE_TASKS_LIMIT, Int32Type.instance) - .addRegularColumn(PENDING_TASKS, Int32Type.instance) - .addRegularColumn(COMPLETED_TASKS, LongType.instance) - .addRegularColumn(BLOCKED_TASKS, LongType.instance) - .addRegularColumn(BLOCKED_TASKS_ALL_TIME, LongType.instance) - .addRegularColumn(CORE_POOL_SIZE, Int32Type.instance) - .addRegularColumn(MAX_POOL_SIZE, Int32Type.instance) - .addRegularColumn(MAX_TASKS_QUEUED, Int32Type.instance) - .build()); - } - - @Override - public DataSet data(DecoratedKey partitionKey) - { - String poolName = UTF8Type.instance.compose(partitionKey.getKey()); - - SimpleDataSet result = new SimpleDataSet(metadata()); - Metrics.getThreadPoolMetrics(poolName) - .ifPresent(metrics -> addRow(result, metrics)); - return result; - } - - @Override - public DataSet data() - { - SimpleDataSet result = new SimpleDataSet(metadata()); - Metrics.allThreadPoolMetrics() - .forEach(metrics -> addRow(result, metrics)); - return result; - } - - private void addRow(SimpleDataSet dataSet, ThreadPoolMetrics metrics) - { - dataSet.row(metrics.poolName) - .column(ACTIVE_TASKS, metrics.activeTasks.getValue()) - .column(ACTIVE_TASKS_LIMIT, metrics.maxPoolSize.getValue()) - .column(PENDING_TASKS, metrics.pendingTasks.getValue()) - .column(COMPLETED_TASKS, metrics.completedTasks.getValue()) - .column(BLOCKED_TASKS, metrics.currentBlocked.getCount()) - .column(BLOCKED_TASKS_ALL_TIME, metrics.totalBlocked.getCount()) - .column(CORE_POOL_SIZE, metrics.corePoolSize.getValue()) - .column(MAX_POOL_SIZE, metrics.maxPoolSize.getValue()) - .column(MAX_TASKS_QUEUED, metrics.maxTasksQueued.getValue()); - } -} diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualKeyspace.java b/src/java/org/apache/cassandra/db/virtual/VirtualKeyspace.java index 675021547c6d..1377c8ba84d7 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualKeyspace.java @@ -17,15 +17,17 @@ */ package org.apache.cassandra.db.virtual; -import java.util.Collection; - import com.google.common.collect.ImmutableCollection; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; - import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.Tables; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + public class VirtualKeyspace { private final String name; @@ -38,6 +40,15 @@ public VirtualKeyspace(String name, Collection tables) this.name = name; this.tables = ImmutableList.copyOf(tables); + List duplicates = tables.stream() + .map(VirtualTable::name) + .distinct() + .filter(entry -> Collections.frequency(tables, entry) > 1) + .collect(Collectors.toList()); + + if (!duplicates.isEmpty()) + throw new IllegalArgumentException(String.format("Duplicate table names in virtual keyspace %s: %s", name, duplicates)); + metadata = KeyspaceMetadata.virtual(name, Tables.of(Iterables.transform(tables, VirtualTable::metadata))); } diff --git a/src/java/org/apache/cassandra/db/virtual/model/CounterMetricRow.java b/src/java/org/apache/cassandra/db/virtual/model/CounterMetricRow.java new file mode 100644 index 000000000000..05ee93f60dff --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/model/CounterMetricRow.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.model; + +import com.codahale.metrics.Counter; +import com.codahale.metrics.Metric; +import org.apache.cassandra.db.virtual.proc.Column; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + + +/** + * Counter metric representation for a {@link org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter}. + */ +public class CounterMetricRow +{ + private final String key; + private final Counter value; + + public CounterMetricRow(String key, Metric value) + { + this.key = key; + this.value = (Counter) value; + } + + @Column(type = Column.Type.PARTITION_KEY) + public String name() + { + return key; + } + + @Column + public String scope() + { + return Metrics.getMetricScope(key); + } + + @Column + public long value() + { + return value.getCount(); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/model/GaugeMetricRow.java b/src/java/org/apache/cassandra/db/virtual/model/GaugeMetricRow.java new file mode 100644 index 000000000000..09c1a5745f3e --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/model/GaugeMetricRow.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.model; + +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Metric; +import org.apache.cassandra.db.virtual.proc.Column; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + + +/** + * Gauge metric representation for a {@link org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter}. + */ +public class GaugeMetricRow +{ + private final String key; + private final Gauge value; + + public GaugeMetricRow(String key, Metric value) + { + this.key = key; + this.value = (Gauge) value; + } + + @Column(type = Column.Type.PARTITION_KEY) + public String name() + { + return key; + } + + @Column + public String scope() + { + return Metrics.getMetricScope(key); + } + + @Column + public String value() + { + return value.getValue().toString(); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/model/HistogramMetricRow.java b/src/java/org/apache/cassandra/db/virtual/model/HistogramMetricRow.java new file mode 100644 index 000000000000..c26fa311a7c8 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/model/HistogramMetricRow.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.model; + +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Metric; +import com.codahale.metrics.Snapshot; +import org.apache.cassandra.db.virtual.proc.Column; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + + +/** + * Historgam metric representation for a {@link org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter}. + */ +public class HistogramMetricRow +{ + private final String key; + private final Snapshot value; + + public HistogramMetricRow(String key, Metric value) + { + this.key = key; + this.value = ((Histogram) value).getSnapshot(); + } + + @Column + public String scope() + { + return Metrics.getMetricScope(key); + } + + @Column(type = Column.Type.PARTITION_KEY) + public String name() + { + return key; + } + + @Column + public double p75th() + { + return value.get75thPercentile(); + } + + @Column + public double p95th() + { + return value.get95thPercentile(); + } + + @Column + public double p98th() + { + return value.get98thPercentile(); + } + + @Column + public double p99th() + { + return value.get99thPercentile(); + } + + @Column + public double p999th() + { + return value.get999thPercentile(); + } + + @Column + public long max() + { + return value.getMax(); + } + + @Column + public double mean() + { + return value.getMean(); + } + + @Column + public long min() + { + return value.getMin(); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/model/MeterMetricRow.java b/src/java/org/apache/cassandra/db/virtual/model/MeterMetricRow.java new file mode 100644 index 000000000000..ff107509b8ab --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/model/MeterMetricRow.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.model; + +import com.codahale.metrics.Meter; +import com.codahale.metrics.Metric; +import org.apache.cassandra.db.virtual.proc.Column; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + + +/** + * Meter metric representation for a {@link org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter}. + */ +public class MeterMetricRow +{ + private final String key; + private final Meter value; + + public MeterMetricRow(String key, Metric value) + { + this.key = key; + this.value = (Meter) value; + } + + @Column(type = Column.Type.PARTITION_KEY) + public String name() + { + return key; + } + + @Column + public String scope() + { + return Metrics.getMetricScope(key); + } + + @Column + public long count() + { + return value.getCount(); + } + + @Column + public double fifteenMinuteRate() + { + return value.getFifteenMinuteRate(); + } + + @Column + public double fiveMinuteRate() + { + return value.getFiveMinuteRate(); + } + + @Column + public double meanRate() + { + return value.getMeanRate(); + } + + @Column + public double oneMinuteRate() + { + return value.getOneMinuteRate(); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/model/MetricGroupRow.java b/src/java/org/apache/cassandra/db/virtual/model/MetricGroupRow.java new file mode 100644 index 000000000000..50a79951f048 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/model/MetricGroupRow.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.model; + +import org.apache.cassandra.db.virtual.proc.Column; + +import static org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter.virtualTableNameStyle; +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.METRICS_GROUP_POSTFIX; + + +/** + * Metric group row representation for a {@link org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter}. + */ +public class MetricGroupRow +{ + private final String group; + + public MetricGroupRow(String group) + { + this.group = group; + } + + @Column(type = Column.Type.PARTITION_KEY) + public String groupName() + { + return group; + } + + @Column + public String virtualTable() + { + return METRICS_GROUP_POSTFIX.apply(virtualTableNameStyle(group)); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/model/MetricRow.java b/src/java/org/apache/cassandra/db/virtual/model/MetricRow.java new file mode 100644 index 000000000000..e5619dd490e2 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/model/MetricRow.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.model; + +import com.codahale.metrics.Counter; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Meter; +import com.codahale.metrics.Metric; +import com.codahale.metrics.Timer; +import org.apache.cassandra.db.virtual.proc.Column; +import org.apache.cassandra.metrics.CassandraMetricsRegistry; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + +/** + * Metric row representation for a {@link org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter}. + */ +public class MetricRow +{ + private final String key; + private final Metric metricEntry; + + public MetricRow(String key, Metric metricEntry) + { + this.key = key; + this.metricEntry = metricEntry; + } + + @Column(type = Column.Type.PARTITION_KEY) + public String name() + { + return key; + } + + @Column + public String scope() + { + return Metrics.getMetricScope(key); + } + + @Column + public String type() + { + Class clazz = metricEntry.getClass(); + if (Counter.class.isAssignableFrom(clazz)) + return "counter"; + else if (Gauge.class.isAssignableFrom(clazz)) + return "gauge"; + else if (Histogram.class.isAssignableFrom(clazz)) + return "histogram"; + else if (Meter.class.isAssignableFrom(clazz)) + return "meter"; + else if (Timer.class.isAssignableFrom(clazz)) + return "timer"; + else + throw new IllegalStateException("Unknown metric type: " + metricEntry.getClass()); + } + + @Column + public String value() + { + return CassandraMetricsRegistry.getValueAsString(metricEntry); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/model/ThreadPoolRow.java b/src/java/org/apache/cassandra/db/virtual/model/ThreadPoolRow.java new file mode 100644 index 000000000000..8b84506408ee --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/model/ThreadPoolRow.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.model; + +import org.apache.cassandra.db.virtual.proc.Column; +import org.apache.cassandra.metrics.ThreadPoolMetrics; + + +/** + * Thread pool metrics representation for a {@link org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter}. + */ +public class ThreadPoolRow +{ + private final ThreadPoolMetrics entry; + + public ThreadPoolRow(ThreadPoolMetrics entry) + { + this.entry = entry; + } + + @Column(type = Column.Type.PARTITION_KEY) + public String name() + { + return entry.poolName; + } + + @Column + public Integer activeTasks() + { + return entry.activeTasks.getValue(); + } + + @Column + public Integer activeTasksLimit() + { + return entry.maxPoolSize.getValue(); + } + + @Column + public Integer pendingTasks() + { + return entry.pendingTasks.getValue(); + } + + @Column + public Long completedTasks() + { + return entry.completedTasks.getValue(); + } + + @Column + public long blockedTasks() + { + return entry.currentBlocked.getCount(); + } + + @Column + public long blockedTasksAllTime() + { + return entry.totalBlocked.getCount(); + } + + @Column + public Integer corePoolSize() + { + return entry.corePoolSize.getValue(); + } + + @Column + public Integer maxPoolSize() + { + return entry.maxPoolSize.getValue(); + } + + @Column + public Integer maxTasksQueued() + { + return entry.maxTasksQueued.getValue(); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/model/TimerMetricRow.java b/src/java/org/apache/cassandra/db/virtual/model/TimerMetricRow.java new file mode 100644 index 000000000000..398658580249 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/model/TimerMetricRow.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.model; + +import com.codahale.metrics.Metric; +import com.codahale.metrics.Timer; +import org.apache.cassandra.db.virtual.proc.Column; + +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; + + +/** + * Timer metric representation for a {@link org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter}. + */ +public class TimerMetricRow +{ + private final String key; + private final Timer value; + + public TimerMetricRow(String key, Metric value) + { + this.key = key; + this.value = (Timer) value; + } + + @Column + public String scope() + { + return Metrics.getMetricScope(key); + } + + @Column(type = Column.Type.PARTITION_KEY) + public String name() + { + return key; + } + + @Column + public long count() + { + return value.getCount(); + } + + @Column + public double fifteenMinuteRate() + { + return value.getFifteenMinuteRate(); + } + + @Column + public double fiveMinuteRate() + { + return value.getFiveMinuteRate(); + } + + @Column + public double meanRate() + { + return value.getMeanRate(); + } + + @Column + public double oneMinuteRate() + { + return value.getOneMinuteRate(); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/metrics/AbstractMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/AbstractMetrics.java index 90956e4b32dc..b305720ebb66 100644 --- a/src/java/org/apache/cassandra/index/sai/metrics/AbstractMetrics.java +++ b/src/java/org/apache/cassandra/index/sai/metrics/AbstractMetrics.java @@ -34,7 +34,7 @@ public abstract class AbstractMetrics protected final String table; private final String index; private final String scope; - protected final List tracked = new ArrayList<>(); + protected final List tracked = new ArrayList<>(); AbstractMetrics(IndexIdentifier indexIdentifier, String scope) { @@ -77,7 +77,7 @@ protected CassandraMetricsRegistry.MetricName createMetricName(String name, Stri CassandraMetricsRegistry.MetricName metricName = new CassandraMetricsRegistry.MetricName(DefaultNameFactory.GROUP_NAME, TYPE, name, metricScope, createMBeanName(name, scope)); - tracked.add(metricName); + tracked.add(metricName.getMetricName()); return metricName; } diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/RowIndexEntry.java b/src/java/org/apache/cassandra/io/sstable/format/big/RowIndexEntry.java index 9d2ab6e93a9a..7828599e14ff 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/RowIndexEntry.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/RowIndexEntry.java @@ -137,6 +137,7 @@ */ public class RowIndexEntry extends AbstractRowIndexEntry { + public static final String TYPE_NAME = "Index"; private static final BigFormat FORMAT = BigFormat.getInstance(); private static final long EMPTY_SIZE = ObjectSizes.measure(new RowIndexEntry(0)); @@ -151,7 +152,7 @@ public class RowIndexEntry extends AbstractRowIndexEntry static final Histogram indexInfoReadsHistogram; static { - MetricNameFactory factory = new DefaultNameFactory("Index", "RowIndexEntry"); + MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME, "RowIndexEntry"); indexEntrySizeHistogram = Metrics.histogram(factory.createMetricName("IndexedEntrySize"), false); indexInfoCountHistogram = Metrics.histogram(factory.createMetricName("IndexInfoCount"), false); indexInfoGetsHistogram = Metrics.histogram(factory.createMetricName("IndexInfoGets"), false); diff --git a/src/java/org/apache/cassandra/metrics/BatchMetrics.java b/src/java/org/apache/cassandra/metrics/BatchMetrics.java index 9bea16211694..28bf4cfbb2b8 100644 --- a/src/java/org/apache/cassandra/metrics/BatchMetrics.java +++ b/src/java/org/apache/cassandra/metrics/BatchMetrics.java @@ -23,8 +23,8 @@ public class BatchMetrics { - private static final MetricNameFactory factory = new DefaultNameFactory("Batch"); - + public static final String TYPE_NAME = "Batch"; + private static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); public final Histogram partitionsPerLoggedBatch; public final Histogram partitionsPerUnloggedBatch; public final Histogram partitionsPerCounterBatch; diff --git a/src/java/org/apache/cassandra/metrics/BufferPoolMetrics.java b/src/java/org/apache/cassandra/metrics/BufferPoolMetrics.java index 71373b35e886..afcf62a09bb5 100644 --- a/src/java/org/apache/cassandra/metrics/BufferPoolMetrics.java +++ b/src/java/org/apache/cassandra/metrics/BufferPoolMetrics.java @@ -25,6 +25,7 @@ public class BufferPoolMetrics { + public static final String TYPE_NAME = "BufferPool"; /** Total number of hits */ public final Meter hits; @@ -49,7 +50,7 @@ public class BufferPoolMetrics public BufferPoolMetrics(String scope, BufferPool bufferPool) { - MetricNameFactory factory = new DefaultNameFactory("BufferPool", scope); + MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME, scope); hits = Metrics.meter(factory.createMetricName("Hits")); diff --git a/src/java/org/apache/cassandra/metrics/CASClientWriteRequestMetrics.java b/src/java/org/apache/cassandra/metrics/CASClientWriteRequestMetrics.java index 87c0d5354132..f42f49592a2c 100644 --- a/src/java/org/apache/cassandra/metrics/CASClientWriteRequestMetrics.java +++ b/src/java/org/apache/cassandra/metrics/CASClientWriteRequestMetrics.java @@ -48,8 +48,8 @@ public CASClientWriteRequestMetrics(String scope) public void release() { super.release(); - Metrics.remove(factory.createMetricName("ConditionNotMet"), - DefaultNameFactory.createMetricName("ClientRequest", "ConditionNotMet", "CASRead")); + // Aliases are already known to the parent metrics, so we don't need to remove them here. + Metrics.remove(factory.createMetricName("ConditionNotMet")); Metrics.remove(factory.createMetricName("MutationSizeHistogram")); } } diff --git a/src/java/org/apache/cassandra/metrics/CIDRAuthorizerMetrics.java b/src/java/org/apache/cassandra/metrics/CIDRAuthorizerMetrics.java index 0de51dfb09b8..39a6e357c316 100644 --- a/src/java/org/apache/cassandra/metrics/CIDRAuthorizerMetrics.java +++ b/src/java/org/apache/cassandra/metrics/CIDRAuthorizerMetrics.java @@ -30,7 +30,8 @@ */ public class CIDRAuthorizerMetrics { - private static final MetricNameFactory factory = new DefaultNameFactory("CIDRAuthorizer"); + public static final String TYPE_NAME = "CIDRAuthorizer"; + private static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); // Number of user accesses rejected by CIDR authorization public static final String CIDR_ACCESSES_REJECTED_COUNT_PREFIX = "CIDRAccessesRejectedCount - "; diff --git a/src/java/org/apache/cassandra/metrics/CQLMetrics.java b/src/java/org/apache/cassandra/metrics/CQLMetrics.java index ce9133394284..edf13dfb4e3f 100644 --- a/src/java/org/apache/cassandra/metrics/CQLMetrics.java +++ b/src/java/org/apache/cassandra/metrics/CQLMetrics.java @@ -26,7 +26,8 @@ public class CQLMetrics { - private static final MetricNameFactory factory = new DefaultNameFactory("CQL"); + public static final String TYPE_NAME = "CQL"; + private static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); public final Counter regularStatementsExecuted; public final Counter preparedStatementsExecuted; diff --git a/src/java/org/apache/cassandra/metrics/CacheMetrics.java b/src/java/org/apache/cassandra/metrics/CacheMetrics.java index a6017a39f879..574b0f065c20 100644 --- a/src/java/org/apache/cassandra/metrics/CacheMetrics.java +++ b/src/java/org/apache/cassandra/metrics/CacheMetrics.java @@ -27,6 +27,7 @@ */ public class CacheMetrics extends AbstractCacheMetrics { + public static final String TYPE_NAME = "Cache"; /** Cache capacity in bytes */ public final Gauge capacity; @@ -44,7 +45,7 @@ public class CacheMetrics extends AbstractCacheMetrics */ public CacheMetrics(String type, CacheSize cache) { - super(new DefaultNameFactory("Cache", type), type); + super(new DefaultNameFactory(TYPE_NAME, type), type); capacity = Metrics.register(factory.createMetricName("Capacity"), cache::capacity); size = Metrics.register(factory.createMetricName("Size"), cache::weightedSize); entries = Metrics.register(factory.createMetricName("Entries"), cache::size); diff --git a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java index cd98cebce9df..e2566a1badcf 100644 --- a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java +++ b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java @@ -17,86 +17,309 @@ */ package org.apache.cassandra.metrics; +import com.codahale.metrics.Counter; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Meter; +import com.codahale.metrics.Metered; +import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.MetricRegistryListener; +import com.codahale.metrics.MetricSet; +import com.codahale.metrics.Timer; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter; +import org.apache.cassandra.db.virtual.VirtualTable; +import org.apache.cassandra.db.virtual.model.CounterMetricRow; +import org.apache.cassandra.db.virtual.model.CounterMetricRowWalker; +import org.apache.cassandra.db.virtual.model.GaugeMetricRow; +import org.apache.cassandra.db.virtual.model.GaugeMetricRowWalker; +import org.apache.cassandra.db.virtual.model.HistogramMetricRow; +import org.apache.cassandra.db.virtual.model.HistogramMetricRowWalker; +import org.apache.cassandra.db.virtual.model.MeterMetricRow; +import org.apache.cassandra.db.virtual.model.MeterMetricRowWalker; +import org.apache.cassandra.db.virtual.model.MetricGroupRow; +import org.apache.cassandra.db.virtual.model.MetricGroupRowWalker; +import org.apache.cassandra.db.virtual.model.MetricRow; +import org.apache.cassandra.db.virtual.model.MetricRowWalker; +import org.apache.cassandra.db.virtual.model.TimerMetricRow; +import org.apache.cassandra.db.virtual.model.TimerMetricRowWalker; +import org.apache.cassandra.index.sai.metrics.AbstractMetrics; +import org.apache.cassandra.io.sstable.format.big.RowIndexEntry; +import org.apache.cassandra.utils.MBeanWrapper; +import org.apache.cassandra.utils.memory.MemtablePool; +import org.apache.commons.lang3.ArrayUtils; + +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; import java.lang.reflect.Method; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.Deque; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; -import javax.management.MalformedObjectNameException; -import javax.management.ObjectName; +import java.util.function.UnaryOperator; +import java.util.stream.Collectors; +import java.util.stream.Stream; -import com.google.common.annotations.VisibleForTesting; - -import com.codahale.metrics.*; -import org.apache.cassandra.utils.MBeanWrapper; +import static java.util.Optional.ofNullable; +import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_METRICS; /** - * Makes integrating 3.0 metrics API with 2.0. - *

- * The 3.0 API comes with poor JMX integration - *

+ * Dropwizard metrics registry extension for Cassandra, as of for now uses the latest version of Dropwizard metrics + * library {@code 4.2.x} that has a pretty good integration with JMX. The registry is used by Cassandra to + * store all metrics and expose them to JMX and {@link org.apache.cassandra.db.virtual.VirtualTable}. + * In addition to that, the registry provides a way to store aliases for metrics and group metrics by + * the Cassandra-specific metric groups, which are used to expose metrics in that way. + * + * @see org.apache.cassandra.db.virtual.VirtualTable + * @see org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter */ public class CassandraMetricsRegistry extends MetricRegistry { - public static final CassandraMetricsRegistry Metrics = new CassandraMetricsRegistry(); - private final Map threadPoolMetrics = new ConcurrentHashMap<>(); + public static final UnaryOperator METRICS_GROUP_POSTFIX = name -> name + "_group"; + /** A map of metric name constructed by {@link com.codahale.metrics.MetricRegistry#name(String, String...)} and + * its full name in the way how it is represented in JMX. The map is used by {@link CassandraJmxMetricsExporter} + * to export metrics to JMX. */ + private static final ConcurrentMap> ALIASES = new ConcurrentHashMap<>(); + /** A set of all known metric groups, used to validate metric groups that are statically defined in Cassandra. */ + static final Set metricGroups; + + /** + * Root metrics registry that is used by Cassandra to store all metrics. + * All modifications to the registry are delegated to the corresponding listeners as well. + * Metrics from the root registry are exported to JMX by {@link CassandraJmxMetricsExporter} and to virtual tables + * via {@link #createMetricsKeyspaceTables()}. + */ + public static final CassandraMetricsRegistry Metrics = init(); + private final MetricRegistryListener jmxExporter = new CassandraJmxMetricsExporter(ALIASES); + /** We have to make sure that this metrics listener is called the last, so that it can clean up aliases. */ + private final MetricRegistryListener housekeepingListener = new BaseMetricRegistryListener() + { + @Override + protected void onMetricRemove(String name) + { + ALIASES.remove(name); + } + }; - private final MBeanWrapper mBeanServer = MBeanWrapper.instance; + private final Map threadPoolMetrics = new ConcurrentHashMap<>(); public final static TimeUnit DEFAULT_TIMER_UNIT = TimeUnit.MICROSECONDS; + static + { + // We have to initialize metric group names like this for now, because we can't register them dynamically + // as it is done for the jmx metrics. So we have to be sure, that all these metric groups are initialized + // at the time #start() method is called. The virtual kespaces are immutable, drivers also rely on the + // fact that virtual keyspaces are immutable, so they won't receive any updates if we change them. + metricGroups = ImmutableSet.builder() + .add(AbstractMetrics.TYPE) + .add(BatchMetrics.TYPE_NAME) + .add(BufferPoolMetrics.TYPE_NAME) + .add(CIDRAuthorizerMetrics.TYPE_NAME) + .add(CQLMetrics.TYPE_NAME) + .add(CacheMetrics.TYPE_NAME) + .add(ChunkCacheMetrics.TYPE_NAME) + .add(ClientMessageSizeMetrics.TYPE) + .add(ClientMetrics.TYPE_NAME) + .add(ClientRequestMetrics.TYPE_NAME) + .add(ClientRequestSizeMetrics.TYPE) + .add(CommitLogMetrics.TYPE_NAME) + .add(CompactionMetrics.TYPE_NAME) + .add(DenylistMetrics.TYPE_NAME) + .add(DroppedMessageMetrics.TYPE) + .add(HintedHandoffMetrics.TYPE_NAME) + .add(HintsServiceMetrics.TYPE_NAME) + .add(InternodeInboundMetrics.TYPE_NAME) + .add(InternodeOutboundMetrics.TYPE_NAME) + .add(KeyspaceMetrics.TYPE_NAME) + .add(MemtablePool.TYPE_NAME) + .add(MessagingMetrics.TYPE_NAME) + .add(PaxosMetrics.TYPE_NAME) + .add(ReadRepairMetrics.TYPE_NAME) + .add(RepairMetrics.TYPE_NAME) + .add(RowIndexEntry.TYPE_NAME) + .add(StorageMetrics.TYPE_NAME) + .add(StreamingMetrics.TYPE_NAME) + .add(TCMMetrics.TYPE_NAME) + .add(TableMetrics.ALIAS_TYPE_NAME) + .add(TableMetrics.TYPE_NAME) + .add(TableMetrics.INDEX_TYPE_NAME) + .add(TableMetrics.INDEX_ALIAS_TYPE_NAME) + .add(ThreadPoolMetrics.TYPE_NAME) + .add(TrieMemtableMetricsView.TYPE_NAME) + .add(UnweightedCacheMetrics.TYPE_NAME) + .build(); + } + private CassandraMetricsRegistry() { - super(); } - public Counter counter(MetricName name) + private static CassandraMetricsRegistry init() { - Counter counter = counter(name.getMetricName()); - registerMBean(counter, name.getMBeanName()); + CassandraMetricsRegistry registry = new CassandraMetricsRegistry(); + // Adding listeners to the root registry, so that they can be notified about all metrics changes. + registry.addListener(registry.jmxExporter); + registry.addListener(registry.housekeepingListener); + return registry; + } - return counter; + @SuppressWarnings("rawtypes") + public static String getValueAsString(Metric metric) + { + if (metric instanceof Counter) + return Long.toString(((Counter) metric).getCount()); + else if (metric instanceof Gauge) + return ((Gauge) metric).getValue().toString(); + else if (metric instanceof Histogram) + return Double.toString(((Histogram) metric).getSnapshot().getMedian()); + else if (metric instanceof Meter) + return Long.toString(((Meter) metric).getCount()); + else if (metric instanceof Timer) + return Long.toString(((Timer) metric).getCount()); + else + throw new IllegalStateException("Unknown metric type: " + metric.getClass().getName()); } - public Counter counter(MetricName name, MetricName alias) + public static List createMetricsKeyspaceTables() { - Counter counter = counter(name); - registerAlias(name, alias); - return counter; + ImmutableList.Builder builder = ImmutableList.builder(); + metricGroups.forEach(groupName -> { + // This is a very efficient way to filter metrics by group name, so make sure that metrics group name + // and metric type following the same order as it constructed in MetricName class. + final String groupPrefix = DefaultNameFactory.GROUP_NAME + '.' + groupName + '.'; + builder.add(CollectionVirtualTableAdapter.createSinglePartitionedKeyFiltered(VIRTUAL_METRICS, + METRICS_GROUP_POSTFIX.apply(groupName), + "All metrics for \"" + groupName + "\" metric group", + new MetricRowWalker(), + Metrics.getMetrics(), + key -> key.startsWith(groupPrefix), + MetricRow::new)); + }); + // Register virtual table of all known metric groups. + builder.add(CollectionVirtualTableAdapter.create(VIRTUAL_METRICS, + "all_groups", + "All metric group names", + new MetricGroupRowWalker(), + metricGroups, + MetricGroupRow::new)) + // Register virtual tables of all metrics types similar to the JMX MBean structure, + // e.g.: HistogramJmxMBean, MeterJmxMBean, etc. + .add(CollectionVirtualTableAdapter.createSinglePartitionedValueFiltered(VIRTUAL_METRICS, + "type_counter", + "All metrics with type \"Counter\"", + new CounterMetricRowWalker(), + Metrics.getMetrics(), + value -> value instanceof Counter, + CounterMetricRow::new)) + .add(CollectionVirtualTableAdapter.createSinglePartitionedValueFiltered(VIRTUAL_METRICS, + "type_gauge", + "All metrics with type \"Gauge\"", + new GaugeMetricRowWalker(), + Metrics.getMetrics(), + value -> value instanceof Gauge, + GaugeMetricRow::new)) + .add(CollectionVirtualTableAdapter.createSinglePartitionedValueFiltered(VIRTUAL_METRICS, + "type_histogram", + "All metrics with type \"Histogram\"", + new HistogramMetricRowWalker(), + Metrics.getMetrics(), + value -> value instanceof Histogram, + HistogramMetricRow::new)) + .add(CollectionVirtualTableAdapter.createSinglePartitionedValueFiltered(VIRTUAL_METRICS, + "type_meter", + "All metrics with type \"Meter\"", + new MeterMetricRowWalker(), + Metrics.getMetrics(), + value -> value instanceof Meter, + MeterMetricRow::new)) + .add(CollectionVirtualTableAdapter.createSinglePartitionedValueFiltered(VIRTUAL_METRICS, + "type_timer", + "All metrics with type \"Timer\"", + new TimerMetricRowWalker(), + Metrics.getMetrics(), + value -> value instanceof Timer, + TimerMetricRow::new)); + return builder.build(); } - public Meter meter(MetricName name) + private static void addUnknownMetric(MetricName newMetricName) { - Meter meter = meter(name.getMetricName()); - registerMBean(meter, name.getMBeanName()); + String type = newMetricName.getType(); + if (type.indexOf('.') >= 0) + throw new IllegalStateException("Metric type must not contain '.' character as it results in the efficiency of the metric collection traversal: " + type); + if (!metricGroups.contains(newMetricName.getType())) + throw new IllegalStateException("Unknown metric group: " + newMetricName.getType()); + if (!metricGroups.contains(newMetricName.getSystemViewName())) + throw new IllegalStateException("Metric view name must match statically registered groups: " + newMetricName.getSystemViewName()); + + // We have to be sure that aliases are registered the same order as they are added to the registry. + ALIASES.computeIfAbsent(newMetricName.getMetricName(), k -> new LinkedList<>()) + .add(newMetricName); + } - return meter; + private static void setAliases(MetricName... names) + { + Arrays.asList(names).forEach(CassandraMetricsRegistry::addUnknownMetric); + ALIASES.get(names[0].getMetricName()).addAll(Arrays.asList(names)); + } + + public String getMetricScope(String metricName) + { + Deque deque = ALIASES.get(metricName); + return deque == null ? "unknown" : deque.stream().findFirst().map(MetricName::getScope).orElse("unknown"); + } + + public Counter counter(MetricName... name) + { + setAliases(name); + Counter counter = super.counter(name[0].getMetricName()); + Stream.of(name).skip(1).forEach(n -> register(n, counter)); + return counter; } - public Meter meter(MetricName name, MetricName alias) + public Meter meter(MetricName... name) { - Meter meter = meter(name); - registerAlias(name, alias); + setAliases(name); + Meter meter = super.meter(name[0].getMetricName()); + Stream.of(name).skip(1).forEach(n -> register(n, meter)); return meter; } public Histogram histogram(MetricName name, boolean considerZeroes) { - Histogram histogram = register(name, new ClearableHistogram(new DecayingEstimatedHistogramReservoir(considerZeroes))); - registerMBean(histogram, name.getMBeanName()); - - return histogram; + return register(name, new ClearableHistogram(new DecayingEstimatedHistogramReservoir(considerZeroes))); } public Histogram histogram(MetricName name, MetricName alias, boolean considerZeroes) { + setAliases(name, alias); Histogram histogram = histogram(name, considerZeroes); - registerAlias(name, alias); + register(alias, histogram); return histogram; } + @SuppressWarnings({"rawtypes", "unchecked"}) + public T gauge(MetricName name, MetricName alias, MetricSupplier gauge) + { + setAliases(name, alias); + Gauge gaugeLoc = super.gauge(name.getMetricName(), gauge); + register(alias, gaugeLoc); + return (T) gaugeLoc; + } + public Timer timer(MetricName name) { return timer(name, DEFAULT_TIMER_UNIT); @@ -107,17 +330,16 @@ public SnapshottingTimer timer(MetricName name, MetricName alias) return timer(name, alias, DEFAULT_TIMER_UNIT); } - public SnapshottingTimer timer(MetricName name, TimeUnit durationUnit) + private SnapshottingTimer timer(MetricName name, TimeUnit durationUnit) { - SnapshottingTimer timer = register(name, new SnapshottingTimer(CassandraMetricsRegistry.createReservoir(durationUnit))); - registerMBean(timer, name.getMBeanName()); - return timer; + return register(name, new SnapshottingTimer(CassandraMetricsRegistry.createReservoir(durationUnit))); } public SnapshottingTimer timer(MetricName name, MetricName alias, TimeUnit durationUnit) { + setAliases(name, alias); SnapshottingTimer timer = timer(name, durationUnit); - registerAlias(name, alias); + register(alias, timer); return timer; } @@ -144,11 +366,13 @@ public static SnapshottingReservoir createReservoir(TimeUnit durationUnit) public T register(MetricName name, T metric) { + if (metric instanceof MetricSet) + throw new IllegalArgumentException("MetricSet registration using MetricName is not supported"); + try { - register(name.getMetricName(), metric); - registerMBean(metric, name.getMBeanName()); - return metric; + addUnknownMetric(name); + return super.register(name.getMetricName(), metric); } catch (IllegalArgumentException e) { @@ -164,7 +388,7 @@ public Collection allThreadPoolMetrics() public Optional getThreadPoolMetrics(String poolName) { - return Optional.ofNullable(threadPoolMetrics.get(poolName)); + return ofNullable(threadPoolMetrics.get(poolName)); } ThreadPoolMetrics register(ThreadPoolMetrics metrics) @@ -178,45 +402,35 @@ void remove(ThreadPoolMetrics metrics) threadPoolMetrics.remove(metrics.poolName, metrics); } - public T register(MetricName name, MetricName aliasName, T metric) - { - T ret = register(name, metric); - registerAlias(name, aliasName); - return ret; - } - public T register(MetricName name, T metric, MetricName... aliases) { - T ret = register(name, metric); - for (MetricName aliasName : aliases) - { - registerAlias(name, aliasName); - } - return ret; + setAliases(ArrayUtils.addAll(new MetricName[]{name}, aliases)); + T metricLoc = register(name, metric); + Stream.of(aliases).forEach(n -> register(n, metricLoc)); + return metricLoc; } - public boolean remove(MetricName name) + public void remove(MetricName name) { - boolean removed = remove(name.getMetricName()); - - mBeanServer.unregisterMBean(name.getMBeanName(), MBeanWrapper.OnException.IGNORE); - return removed; + // Aliases are removed in onMetricRemoved by metrics listener. + remove(name.getMetricName()); } - public boolean remove(MetricName name, MetricName... aliases) + public boolean remove(String name) { - if (remove(name)) - { - for (MetricName alias : aliases) - { - removeAlias(alias); - } - return true; - } - return false; + LinkedList delete = ofNullable(ALIASES.get(name)) + .map(s -> s.stream().map(MetricName::getMetricName) + .collect(Collectors.toCollection(LinkedList::new))) + .orElse(new LinkedList<>(Collections.singletonList(name))); + // Aliases are removed in onMetricRemoved by metrics listener. + Iterator iter = delete.descendingIterator(); + boolean removed = true; + while (iter.hasNext()) + removed &= super.remove(iter.next()); + return removed; } - public void registerMBean(Metric metric, ObjectName name) + private void registerMBean(Metric metric, ObjectName name, MBeanWrapper mBeanServer) { AbstractBean mbean; @@ -233,22 +447,13 @@ else if (metric instanceof Metered) else throw new IllegalArgumentException("Unknown metric type: " + metric.getClass()); - if (!mBeanServer.isRegistered(name)) + if (mBeanServer != null && !mBeanServer.isRegistered(name)) mBeanServer.registerMBean(mbean, name, MBeanWrapper.OnException.LOG); } - private void registerAlias(MetricName existingName, MetricName aliasName) - { - Metric existing = Metrics.getMetrics().get(existingName.getMetricName()); - assert existing != null : existingName + " not registered"; - - registerMBean(existing, aliasName.getMBeanName()); - } - - private void removeAlias(MetricName name) + private void unregisterMBean(ObjectName name, MBeanWrapper mBeanWrapper) { - if (mBeanServer.isRegistered(name.getMBeanName())) - MBeanWrapper.instance.unregisterMBean(name.getMBeanName(), MBeanWrapper.OnException.IGNORE); + mBeanWrapper.unregisterMBean(name, MBeanWrapper.OnException.IGNORE); } /** @@ -284,7 +489,10 @@ public ObjectName objectName() } } - + /** + * Exports a gauge as a JMX MBean, check corresponding {@link org.apache.cassandra.db.virtual.model.GaugeMetricRow} + * for the same functionality for virtual tables. + */ public interface JmxGaugeMBean extends MetricMBean { Object getValue(); @@ -307,6 +515,10 @@ public Object getValue() } } + /** + * Exports a histogram as a JMX MBean, check corresponding {@link org.apache.cassandra.db.virtual.model.HistogramMetricRow} + * for the same functionality for virtual tables. + */ public interface JmxHistogramMBean extends MetricMBean { long getCount(); @@ -437,6 +649,10 @@ public synchronized long[] getRecentValues() } } + /** + * Exports a counter as a JMX MBean, check corresponding {@link org.apache.cassandra.db.virtual.model.CounterMetricRow} + * for the same functionality for virtual tables. + */ public interface JmxCounterMBean extends MetricMBean { long getCount(); @@ -459,6 +675,10 @@ public long getCount() } } + /** + * Exports a meter as a JMX MBean, check corresponding {@link org.apache.cassandra.db.virtual.model.MeterMetricRow} + * for the same functionality for virtual tables. + */ public interface JmxMeterMBean extends MetricMBean { long getCount(); @@ -474,6 +694,10 @@ public interface JmxMeterMBean extends MetricMBean String getRateUnit(); } + /** + * Exports a timer as a JMX MBean, check corresponding {@link org.apache.cassandra.db.virtual.model.TimerMetricRow} + * for the same functionality for virtual tables. + */ private static class JmxMeter extends AbstractBean implements JmxMeterMBean { private final Metered metric; @@ -531,6 +755,10 @@ private String calculateRateUnit(TimeUnit unit) } } + /** + * Exports a timer as a JMX MBean, check corresponding {@link org.apache.cassandra.db.virtual.model.TimerMetricRow} + * for the same functionality for virtual tables. + */ public interface JmxTimerMBean extends JmxMeterMBean { double getMin(); @@ -693,11 +921,13 @@ static long[] delta(long[] now, long[] last) */ public static class MetricName implements Comparable { + public static final MetricName EMPTY = new MetricName(MetricName.class, "EMPTY"); private final String group; private final String type; private final String name; private final String scope; private final String mBeanName; + private final String systemViewName; /** * Creates a new {@link MetricName} without a scope. @@ -750,17 +980,22 @@ public MetricName(String group, String type, String name, String scope) this(group, type, name, scope, createMBeanName(group, type, name, scope)); } + public MetricName(String group, String type, String name, String scope, String mBeanName) + { + this(group, type, name, scope, mBeanName, type); + } + /** * Creates a new {@link MetricName} without a scope. * - * @param group the group to which the {@link Metric} belongs - * @param type the type to which the {@link Metric} belongs - * @param name the name of the {@link Metric} - * @param scope the scope of the {@link Metric} - * @param mBeanName the 'ObjectName', represented as a string, to use when registering the - * MBean. + * @param group the group to which the {@link Metric} belongs + * @param type the type to which the {@link Metric} belongs + * @param name the name of the {@link Metric} + * @param scope the scope of the {@link Metric} + * @param mBeanName the 'ObjectName', represented as a string, to use when registering the MBean. + * @param systemViewName the name of the virtual table to which the {@link Metric} belongs. */ - public MetricName(String group, String type, String name, String scope, String mBeanName) + public MetricName(String group, String type, String name, String scope, String mBeanName, String systemViewName) { if (group == null || type == null) { @@ -775,6 +1010,7 @@ public MetricName(String group, String type, String name, String scope, String m this.name = name; this.scope = scope; this.mBeanName = mBeanName; + this.systemViewName = systemViewName; } /** @@ -863,6 +1099,11 @@ public ObjectName getMBeanName() } } + public String getSystemViewName() + { + return systemViewName; + } + @Override public boolean equals(Object o) { @@ -963,6 +1204,122 @@ public static String chooseName(String name, Method method) return name; } } + + private static class CassandraJmxMetricsExporter extends BaseMetricRegistryListener + { + private final MBeanWrapper mBeanWrapper = MBeanWrapper.instance; + private final Map> aliases; + + public CassandraJmxMetricsExporter(Map> aliases) + { + this.aliases = aliases; + } + + protected void onMetricAdded(String name, Metric metric) + { + Deque deque = aliases.get(name); + if (deque == null) + return; + + assert deque.getFirst().getMetricName().equals(name); + Metrics.registerMBean(metric, deque.getFirst().getMBeanName(), mBeanWrapper); + } + + protected void onMetricRemove(String name) + { + Deque deque = aliases.get(name); + if (deque == null) + return; + + assert deque.getFirst().getMetricName().equals(name); + Metrics.unregisterMBean(deque.getFirst().getMBeanName(), mBeanWrapper); + } + } + + private static abstract class BaseMetricRegistryListener implements MetricRegistryListener + { + protected void onMetricAdded(String name, Metric metric) {} + + protected void onMetricRemove(String name) {} + + @Override + public void onGaugeAdded(String metricName, Gauge gauge) + { + onMetricAdded(metricName, gauge); + } + + @Override + public void onGaugeRemoved(String metricName) + { + onMetricRemove(metricName); + } + + @Override + public void onCounterAdded(String metricName, Counter counter) + { + onMetricAdded(metricName, counter); + } + + @Override + public void onCounterRemoved(String metricName) + { + onMetricRemove(metricName); + } + + @Override + public void onHistogramAdded(String metricName, Histogram histogram) + { + onMetricAdded(metricName, histogram); + } + + @Override + public void onHistogramRemoved(String metricName) + { + onMetricRemove(metricName); + } + + @Override + public void onMeterAdded(String metricName, Meter meter) + { + onMetricAdded(metricName, meter); + } + + @Override + public void onMeterRemoved(String metricName) + { + onMetricRemove(metricName); + } + + @Override + public void onTimerAdded(String metricName, Timer timer) + { + onMetricAdded(metricName, timer); + } + + @Override + public void onTimerRemoved(String metricName) + { + onMetricRemove(metricName); + } + } + + private static class MetricNameFactoryWrapper implements MetricNameFactory + { + private final MetricNameFactory delegate; + private final UnaryOperator handler; + + public MetricNameFactoryWrapper(MetricNameFactory delegate, UnaryOperator handler) + { + this.delegate = delegate; + this.handler = handler; + } + + @Override + public MetricName createMetricName(String metricName) + { + return handler.apply(delegate.createMetricName(metricName)); + } + } } diff --git a/src/java/org/apache/cassandra/metrics/ChunkCacheMetrics.java b/src/java/org/apache/cassandra/metrics/ChunkCacheMetrics.java index 8195aafbf33c..a208998c9a21 100644 --- a/src/java/org/apache/cassandra/metrics/ChunkCacheMetrics.java +++ b/src/java/org/apache/cassandra/metrics/ChunkCacheMetrics.java @@ -33,6 +33,7 @@ */ public class ChunkCacheMetrics extends CacheMetrics implements StatsCounter { + public static final String TYPE_NAME = "ChunkCache"; /** Latency of misses */ public final Timer missLatency; @@ -43,7 +44,7 @@ public class ChunkCacheMetrics extends CacheMetrics implements StatsCounter */ public ChunkCacheMetrics(ChunkCache cache) { - super("ChunkCache", cache); + super(TYPE_NAME, cache); missLatency = Metrics.timer(factory.createMetricName("MissLatency")); } diff --git a/src/java/org/apache/cassandra/metrics/ClientMessageSizeMetrics.java b/src/java/org/apache/cassandra/metrics/ClientMessageSizeMetrics.java index acdbf2851b76..416e6b44b48c 100644 --- a/src/java/org/apache/cassandra/metrics/ClientMessageSizeMetrics.java +++ b/src/java/org/apache/cassandra/metrics/ClientMessageSizeMetrics.java @@ -28,7 +28,7 @@ */ public class ClientMessageSizeMetrics { - private static final String TYPE = "ClientMessageSize"; + public static final String TYPE = "ClientMessageSize"; public static final Counter bytesReceived = Metrics.counter(DefaultNameFactory.createMetricName(TYPE, "BytesReceived", null)); public static final Counter bytesSent = Metrics.counter(DefaultNameFactory.createMetricName(TYPE, "BytesSent", null)); public static final Histogram bytesReceivedPerRequest = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesReceivedPerRequest", null), true); diff --git a/src/java/org/apache/cassandra/metrics/ClientMetrics.java b/src/java/org/apache/cassandra/metrics/ClientMetrics.java index a8801bad5f20..570b8703d1dd 100644 --- a/src/java/org/apache/cassandra/metrics/ClientMetrics.java +++ b/src/java/org/apache/cassandra/metrics/ClientMetrics.java @@ -18,21 +18,11 @@ */ package org.apache.cassandra.metrics; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Predicate; - -import com.google.common.annotations.VisibleForTesting; - import com.codahale.metrics.Gauge; import com.codahale.metrics.Histogram; import com.codahale.metrics.Meter; import com.codahale.metrics.Reservoir; +import com.google.common.annotations.VisibleForTesting; import org.apache.cassandra.auth.AuthenticatedUser; import org.apache.cassandra.auth.IAuthenticator; import org.apache.cassandra.auth.IAuthenticator.AuthenticationMode; @@ -43,13 +33,23 @@ import org.apache.cassandra.transport.Server; import org.apache.cassandra.transport.ServerConnection; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Predicate; + import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; public final class ClientMetrics { + public static final String TYPE_NAME = "Client"; public static final ClientMetrics instance = new ClientMetrics(); - private static final MetricNameFactory factory = new DefaultNameFactory("Client"); + private static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); private volatile boolean initialized = false; private Server server = null; @@ -282,9 +282,7 @@ private Gauge registerGauge(MetricNameFactory metricNameFactory, String n private Gauge registerGauge(String name, String deprecated, Gauge gauge) { - Gauge registeredGauge = registerGauge(name, gauge); - Metrics.registerMBean(registeredGauge, factory.createMetricName(deprecated).getMBeanName()); - return registeredGauge; + return Metrics.gauge(factory.createMetricName(name), factory.createMetricName(deprecated), () -> gauge); } private Meter registerMeter(String name) diff --git a/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java b/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java index 408087051e2a..61fcc34bf129 100644 --- a/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java +++ b/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java @@ -31,6 +31,7 @@ public class ClientRequestMetrics extends LatencyMetrics { + public static final String TYPE_NAME = "ClientRequest"; public final Meter timeouts; public final Meter unavailables; public final Meter failures; @@ -42,7 +43,7 @@ public class ClientRequestMetrics extends LatencyMetrics public ClientRequestMetrics(String scope) { - super("ClientRequest", scope); + super(TYPE_NAME, scope); timeouts = Metrics.meter(factory.createMetricName("Timeouts")); unavailables = Metrics.meter(factory.createMetricName("Unavailables")); diff --git a/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java b/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java index 4def87ea16a1..f7719762a448 100644 --- a/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java +++ b/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java @@ -34,7 +34,7 @@ public class ClientRequestSizeMetrics { - private static final String TYPE = "ClientRequestSize"; + public static final String TYPE = "ClientRequestSize"; public static final Counter totalColumnsRead = Metrics.counter(DefaultNameFactory.createMetricName(TYPE, "ColumnsRead", null)); public static final Counter totalRowsRead = Metrics.counter(DefaultNameFactory.createMetricName(TYPE, "RowsRead", null)); diff --git a/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java b/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java index cb53575b2b22..1c7390522a3b 100644 --- a/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java +++ b/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java @@ -30,7 +30,8 @@ */ public class CommitLogMetrics { - public static final MetricNameFactory factory = new DefaultNameFactory("CommitLog"); + public static final String TYPE_NAME = "CommitLog"; + public static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); /** Number of completed tasks */ public Gauge completedTasks; diff --git a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java index 0fe1ec7418ce..8bd48e520ec7 100644 --- a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java +++ b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java @@ -39,7 +39,8 @@ */ public class CompactionMetrics { - public static final MetricNameFactory factory = new DefaultNameFactory("Compaction"); + public static final String TYPE_NAME = "Compaction"; + public static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); /** Estimated number of compactions remaining to perform */ public final Gauge pendingTasks; diff --git a/src/java/org/apache/cassandra/metrics/DenylistMetrics.java b/src/java/org/apache/cassandra/metrics/DenylistMetrics.java index 0372787af980..e3c34ab35756 100644 --- a/src/java/org/apache/cassandra/metrics/DenylistMetrics.java +++ b/src/java/org/apache/cassandra/metrics/DenylistMetrics.java @@ -24,6 +24,7 @@ public class DenylistMetrics { + public static final String TYPE_NAME = "StorageProxy"; private final Meter writesRejected; private final Meter readsRejected; private final Meter rangeReadsRejected; @@ -31,7 +32,7 @@ public class DenylistMetrics public DenylistMetrics() { - final MetricNameFactory factory = new DefaultNameFactory("StorageProxy", "PartitionDenylist"); + final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME, "PartitionDenylist"); writesRejected = Metrics.meter(factory.createMetricName("WriteRejected")); readsRejected = Metrics.meter(factory.createMetricName("ReadRejected")); rangeReadsRejected = Metrics.meter(factory.createMetricName("RangeReadRejected")); diff --git a/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java b/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java index 91a680e50d99..e17c24de7a67 100644 --- a/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java +++ b/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java @@ -35,7 +35,7 @@ */ public class DroppedMessageMetrics { - private static final String TYPE = "DroppedMessage"; + public static final String TYPE = "DroppedMessage"; // backward compatibility for request metrics which names have changed in 4.0 as part of CASSANDRA-15066 private static final ImmutableMap REQUEST_VERB_ALIAS; diff --git a/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java b/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java index bc87218bbdb5..187f5afe436b 100644 --- a/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java +++ b/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java @@ -38,9 +38,10 @@ */ public class HintedHandoffMetrics { + public static final String TYPE_NAME = "HintsService"; private static final Logger logger = LoggerFactory.getLogger(HintedHandoffMetrics.class); - private static final MetricNameFactory factory = new DefaultNameFactory("HintsService"); + private static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); /** Total number of hints which are not stored, This is not a cache. */ private final LoadingCache notStored = Caffeine.newBuilder() diff --git a/src/java/org/apache/cassandra/metrics/HintsServiceMetrics.java b/src/java/org/apache/cassandra/metrics/HintsServiceMetrics.java index bcff38957715..2a8ce92776d9 100644 --- a/src/java/org/apache/cassandra/metrics/HintsServiceMetrics.java +++ b/src/java/org/apache/cassandra/metrics/HintsServiceMetrics.java @@ -34,9 +34,10 @@ */ public final class HintsServiceMetrics { + public static final String TYPE_NAME = "HintsService"; private static final Logger logger = LoggerFactory.getLogger(HintsServiceMetrics.class); - private static final MetricNameFactory factory = new DefaultNameFactory("HintsService"); + private static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); public static final Meter hintsSucceeded = Metrics.meter(factory.createMetricName("HintsSucceeded")); public static final Meter hintsFailed = Metrics.meter(factory.createMetricName("HintsFailed")); diff --git a/src/java/org/apache/cassandra/metrics/InternodeInboundMetrics.java b/src/java/org/apache/cassandra/metrics/InternodeInboundMetrics.java index cc8dae9f4cfa..16870321bee8 100644 --- a/src/java/org/apache/cassandra/metrics/InternodeInboundMetrics.java +++ b/src/java/org/apache/cassandra/metrics/InternodeInboundMetrics.java @@ -27,6 +27,7 @@ */ public class InternodeInboundMetrics { + public static final String TYPE_NAME = "InternodeInbound"; private final MetricName corruptFramesRecovered; private final MetricName corruptFramesUnrecovered; private final MetricName errorBytes; @@ -50,7 +51,7 @@ public class InternodeInboundMetrics public InternodeInboundMetrics(InetAddressAndPort peer, InboundMessageHandlers handlers) { // ipv6 addresses will contain colons, which are invalid in a JMX ObjectName - MetricNameFactory factory = new DefaultNameFactory("InboundConnection", peer.getHostAddressAndPortForJMX()); + MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME, peer.getHostAddressAndPortForJMX()); register(corruptFramesRecovered = factory.createMetricName("CorruptFramesRecovered"), handlers::corruptFramesRecovered); register(corruptFramesUnrecovered = factory.createMetricName("CorruptFramesUnrecovered"), handlers::corruptFramesUnrecovered); diff --git a/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java b/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java index 2b0348eb5556..f2a1a8bb6f03 100644 --- a/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java +++ b/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java @@ -119,7 +119,7 @@ public InternodeOutboundMetrics(InetAddressAndPort ip, final OutboundConnections // ipv6 addresses will contain colons, which are invalid in a JMX ObjectName address = ip.getHostAddressAndPortForJMX(); - factory = new DefaultNameFactory("Connection", address); + factory = new DefaultNameFactory(TYPE_NAME, address); largeMessagePendingTasks = Metrics.register(factory.createMetricName("LargeMessagePendingTasks"), messagingPool.large::pendingCount); largeMessagePendingBytes = Metrics.register(factory.createMetricName("LargeMessagePendingBytes"), messagingPool.large::pendingBytes); diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java index ae15bbf95a04..83a1651fa746 100644 --- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java +++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java @@ -43,6 +43,7 @@ */ public class KeyspaceMetrics { + public static final String TYPE_NAME = "keyspace"; /** Total amount of live data stored in the memtable, excluding any data structure overhead */ public final Gauge memtableLiveDataSize; /** Total amount of data stored in the memtable that resides on-heap, including column related overhead and partitions overwritten. */ @@ -424,11 +425,11 @@ public MetricName createMetricName(String metricName) StringBuilder mbeanName = new StringBuilder(); mbeanName.append(groupName).append(":"); - mbeanName.append("type=Keyspace"); + mbeanName.append("type=").append("Keyspace"); mbeanName.append(",keyspace=").append(keyspaceName); mbeanName.append(",name=").append(metricName); - return new MetricName(groupName, "keyspace", metricName, keyspaceName, mbeanName.toString()); + return new MetricName(groupName, TYPE_NAME, metricName, keyspaceName, mbeanName.toString()); } } } diff --git a/src/java/org/apache/cassandra/metrics/LatencyMetrics.java b/src/java/org/apache/cassandra/metrics/LatencyMetrics.java index af8ad71be124..e4abd40bbf01 100644 --- a/src/java/org/apache/cassandra/metrics/LatencyMetrics.java +++ b/src/java/org/apache/cassandra/metrics/LatencyMetrics.java @@ -99,8 +99,8 @@ public LatencyMetrics(MetricNameFactory factory, MetricNameFactory aliasFactory, } else { - latency = Metrics.register(factory.createMetricName(namePrefix + "Latency"), aliasFactory.createMetricName(namePrefix + "Latency"), timer); - totalLatency = Metrics.register(factory.createMetricName(namePrefix + "TotalLatency"), aliasFactory.createMetricName(namePrefix + "TotalLatency"), counter); + latency = Metrics.register(factory.createMetricName(namePrefix + "Latency"), timer, aliasFactory.createMetricName(namePrefix + "Latency")); + totalLatency = Metrics.register(factory.createMetricName(namePrefix + "TotalLatency"), counter, aliasFactory.createMetricName(namePrefix + "TotalLatency")); } } @@ -164,16 +164,9 @@ public void release() { parent.removeChildren(this); } - if (aliasFactory == null) - { - Metrics.remove(factory.createMetricName(namePrefix + "Latency")); - Metrics.remove(factory.createMetricName(namePrefix + "TotalLatency")); - } - else - { - Metrics.remove(factory.createMetricName(namePrefix + "Latency"), aliasFactory.createMetricName(namePrefix + "Latency")); - Metrics.remove(factory.createMetricName(namePrefix + "TotalLatency"), aliasFactory.createMetricName(namePrefix + "TotalLatency")); - } + // Aliases are already known to the parent metrics, so we don't need to remove them here. + Metrics.remove(factory.createMetricName(namePrefix + "Latency")); + Metrics.remove(factory.createMetricName(namePrefix + "TotalLatency")); } public class LatencyMetricsTimer extends Timer diff --git a/src/java/org/apache/cassandra/metrics/MessagingMetrics.java b/src/java/org/apache/cassandra/metrics/MessagingMetrics.java index bef6d087373f..6151f57ee4d0 100644 --- a/src/java/org/apache/cassandra/metrics/MessagingMetrics.java +++ b/src/java/org/apache/cassandra/metrics/MessagingMetrics.java @@ -49,7 +49,8 @@ */ public class MessagingMetrics implements InboundMessageHandlers.GlobalMetricCallbacks { - private static final MetricNameFactory factory = new DefaultNameFactory("Messaging"); + public static final String TYPE_NAME = "Messaging"; + private static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); private static final Logger logger = LoggerFactory.getLogger(MessagingMetrics.class); private static final int LOG_DROPPED_INTERVAL_IN_MS = 5000; diff --git a/src/java/org/apache/cassandra/metrics/PaxosMetrics.java b/src/java/org/apache/cassandra/metrics/PaxosMetrics.java index 1e1ba89ba263..45088a1d1789 100644 --- a/src/java/org/apache/cassandra/metrics/PaxosMetrics.java +++ b/src/java/org/apache/cassandra/metrics/PaxosMetrics.java @@ -25,7 +25,8 @@ public class PaxosMetrics { - private static final MetricNameFactory factory = new DefaultNameFactory("Paxos"); + public static final String TYPE_NAME = "Paxos"; + private static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); public static final Counter linearizabilityViolations = Metrics.counter(factory.createMetricName("LinearizabilityViolations")); public static final Meter repairPaxosTopologyRetries = Metrics.meter(factory.createMetricName("RepairPaxosTopologyRetries")); diff --git a/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java b/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java index 32d62efd204e..1adb5dcd7f30 100644 --- a/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java +++ b/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java @@ -26,7 +26,8 @@ */ public class ReadRepairMetrics { - private static final MetricNameFactory factory = new DefaultNameFactory("ReadRepair"); + public static final String TYPE_NAME = "ReadRepair"; + private static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); public static final Meter repairedBlocking = Metrics.meter(factory.createMetricName("RepairedBlocking")); public static final Meter reconcileRead = Metrics.meter(factory.createMetricName("ReconcileRead")); diff --git a/src/java/org/apache/cassandra/metrics/StorageMetrics.java b/src/java/org/apache/cassandra/metrics/StorageMetrics.java index ebb031cc4d98..8281ce83a26f 100644 --- a/src/java/org/apache/cassandra/metrics/StorageMetrics.java +++ b/src/java/org/apache/cassandra/metrics/StorageMetrics.java @@ -31,7 +31,8 @@ */ public class StorageMetrics { - private static final MetricNameFactory factory = new DefaultNameFactory("Storage"); + public static final String TYPE_NAME = "Storage"; + private static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); public static final Counter load = Metrics.counter(factory.createMetricName("Load")); public static final Counter uncompressedLoad = Metrics.counter(factory.createMetricName("UncompressedLoad")); diff --git a/src/java/org/apache/cassandra/metrics/StreamingMetrics.java b/src/java/org/apache/cassandra/metrics/StreamingMetrics.java index 524690863711..eb5d84006340 100644 --- a/src/java/org/apache/cassandra/metrics/StreamingMetrics.java +++ b/src/java/org/apache/cassandra/metrics/StreamingMetrics.java @@ -78,7 +78,7 @@ public static StreamingMetrics get(InetAddressAndPort ip) public StreamingMetrics(final InetAddressAndPort peer) { - MetricNameFactory factory = new DefaultNameFactory("Streaming", peer.toString().replace(':', '.')); + MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME, peer.toString().replace(':', '.')); incomingBytes = Metrics.counter(factory.createMetricName("IncomingBytes")); outgoingBytes= Metrics.counter(factory.createMetricName("OutgoingBytes")); incomingProcessTime = Metrics.timer(factory.createMetricName("IncomingProcessTime")); diff --git a/src/java/org/apache/cassandra/metrics/TCMMetrics.java b/src/java/org/apache/cassandra/metrics/TCMMetrics.java index 61a3b9d1e90f..134a1a34e2ac 100644 --- a/src/java/org/apache/cassandra/metrics/TCMMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TCMMetrics.java @@ -33,7 +33,8 @@ public class TCMMetrics { - private static final MetricNameFactory factory = new DefaultNameFactory("TCM"); + public static final String TYPE_NAME = "TCM"; + private static final MetricNameFactory factory = new DefaultNameFactory(TYPE_NAME); public static final TCMMetrics instance = new TCMMetrics(); diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java index 410562cdaa20..7ab1cbbfbf6d 100644 --- a/src/java/org/apache/cassandra/metrics/TableMetrics.java +++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java @@ -71,13 +71,17 @@ */ public class TableMetrics { + public static final String TYPE_NAME = "Table"; + public static final String INDEX_TYPE_NAME = "IndexTable"; + public static final String ALIAS_TYPE_NAME = "ColumnFamily"; + public static final String INDEX_ALIAS_TYPE_NAME = "IndexColumnFamily"; /** * stores metrics that will be rolled into a single global metric */ private static final ConcurrentMap> ALL_TABLE_METRICS = Maps.newConcurrentMap(); public static final long[] EMPTY = new long[0]; - private static final MetricNameFactory GLOBAL_FACTORY = new AllTableMetricNameFactory("Table"); - private static final MetricNameFactory GLOBAL_ALIAS_FACTORY = new AllTableMetricNameFactory("ColumnFamily"); + private static final MetricNameFactory GLOBAL_FACTORY = new AllTableMetricNameFactory(TYPE_NAME); + private static final MetricNameFactory GLOBAL_ALIAS_FACTORY = new AllTableMetricNameFactory(ALIAS_TYPE_NAME); public final static LatencyMetrics GLOBAL_READ_LATENCY = new LatencyMetrics(GLOBAL_FACTORY, GLOBAL_ALIAS_FACTORY, "Read"); public final static LatencyMetrics GLOBAL_WRITE_LATENCY = new LatencyMetrics(GLOBAL_FACTORY, GLOBAL_ALIAS_FACTORY, "Write"); @@ -403,8 +407,8 @@ public static long[] addHistogram(long[] sums, long[] buckets) */ public TableMetrics(final ColumnFamilyStore cfs, ReleasableMetric memtableMetrics) { - factory = new TableMetricNameFactory(cfs, "Table"); - aliasFactory = new TableMetricNameFactory(cfs, "ColumnFamily"); + factory = new TableMetricNameFactory(cfs, cfs.isIndex() ? INDEX_TYPE_NAME : TYPE_NAME); + aliasFactory = new TableMetricNameFactory(cfs, cfs.isIndex() ? INDEX_ALIAS_TYPE_NAME : ALIAS_TYPE_NAME); if (memtableMetrics != null) { @@ -932,10 +936,10 @@ protected Gauge createTableGauge(String name, Gauge gauge, Gauge protected Gauge createTableGauge(String name, String alias, Gauge gauge, Gauge globalGauge) { - Gauge cfGauge = Metrics.register(factory.createMetricName(name), aliasFactory.createMetricName(alias), gauge); + Gauge cfGauge = Metrics.register(factory.createMetricName(name), gauge, aliasFactory.createMetricName(alias)); if (register(name, alias, cfGauge) && globalGauge != null) { - Metrics.register(GLOBAL_FACTORY.createMetricName(name), GLOBAL_ALIAS_FACTORY.createMetricName(alias), globalGauge); + Metrics.register(GLOBAL_FACTORY.createMetricName(name), globalGauge, GLOBAL_ALIAS_FACTORY.createMetricName(alias)); } return cfGauge; } @@ -984,19 +988,14 @@ protected Counter createTableCounter(final String name, final String alias) if (register(name, alias, cfCounter)) { Metrics.register(GLOBAL_FACTORY.createMetricName(name), - GLOBAL_ALIAS_FACTORY.createMetricName(alias), - new Gauge() - { - public Long getValue() - { - long total = 0; - for (Metric cfGauge : ALL_TABLE_METRICS.get(name)) + (Gauge) () -> { - total += ((Counter) cfGauge).getCount(); - } - return total; - } - }); + long total = 0; + for (Metric cfGauge : ALL_TABLE_METRICS.get(name)) + total += ((Counter) cfGauge).getCount(); + return total; + }, + GLOBAL_ALIAS_FACTORY.createMetricName(alias)); } return cfCounter; } @@ -1151,16 +1150,8 @@ private void releaseMetric(String tableMetricName, String cfMetricName, String t { // Metric will be null if we are releasing a view metric. Views have null for ViewLockAcquireTime and ViewLockReadTime ALL_TABLE_METRICS.get(tableMetricName).remove(metric); - CassandraMetricsRegistry.MetricName cfAlias = aliasFactory.createMetricName(cfMetricName); - - if (tableMetricAlias != null) - { - Metrics.remove(name, cfAlias, factory.createMetricName(tableMetricAlias), aliasFactory.createMetricName(tableMetricAlias)); - } - else - { - Metrics.remove(name, cfAlias); - } + // Aliases are already known to the parent metrics, so we don't need to remove them here. + Metrics.remove(name); } } @@ -1258,21 +1249,18 @@ static class TableMetricNameFactory implements MetricNameFactory { private final String keyspaceName; private final String tableName; - private final boolean isIndex; private final String type; TableMetricNameFactory(ColumnFamilyStore cfs, String type) { this.keyspaceName = cfs.getKeyspaceName(); this.tableName = cfs.name; - this.isIndex = cfs.isIndex(); this.type = type; } public CassandraMetricsRegistry.MetricName createMetricName(String metricName) { String groupName = TableMetrics.class.getPackage().getName(); - String type = isIndex ? "Index" + this.type : this.type; StringBuilder mbeanName = new StringBuilder(); mbeanName.append(groupName).append(":"); diff --git a/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java b/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java index 95ab601fe5ee..c8a4dedc63a5 100644 --- a/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java +++ b/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java @@ -33,6 +33,7 @@ */ public class ThreadPoolMetrics { + public static final String TYPE_NAME = "ThreadPools"; public static final String ACTIVE_TASKS = "ActiveTasks"; public static final String PENDING_TASKS = "PendingTasks"; public static final String COMPLETED_TASKS = "CompletedTasks"; @@ -123,10 +124,10 @@ public void release() private static MetricName makeMetricName(String path, String poolName, String metricName) { return new MetricName("org.apache.cassandra.metrics", - "ThreadPools", + TYPE_NAME, metricName, path + '.' + poolName, - format("org.apache.cassandra.metrics:type=ThreadPools,path=%s,scope=%s,name=%s", - path, poolName, metricName)); + format("org.apache.cassandra.metrics:type=%s,path=%s,scope=%s,name=%s", + TYPE_NAME, path, poolName, metricName)); } } diff --git a/src/java/org/apache/cassandra/metrics/TrieMemtableMetricsView.java b/src/java/org/apache/cassandra/metrics/TrieMemtableMetricsView.java index 934350399945..a308fbbab866 100644 --- a/src/java/org/apache/cassandra/metrics/TrieMemtableMetricsView.java +++ b/src/java/org/apache/cassandra/metrics/TrieMemtableMetricsView.java @@ -21,9 +21,11 @@ import com.codahale.metrics.Counter; import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics; +import static org.apache.cassandra.metrics.DefaultNameFactory.GROUP_NAME; public class TrieMemtableMetricsView { + public static final String TYPE_NAME = "TrieMemtable"; private static final String UNCONTENDED_PUTS = "Uncontended memtable puts"; private static final String CONTENDED_PUTS = "Contended memtable puts"; private static final String CONTENTION_TIME = "Contention time"; @@ -74,17 +76,14 @@ static class TrieMemtableMetricNameFactory implements MetricNameFactory public CassandraMetricsRegistry.MetricName createMetricName(String metricName) { - String groupName = TableMetrics.class.getPackage().getName(); - String type = "TrieMemtable"; - StringBuilder mbeanName = new StringBuilder(); - mbeanName.append(groupName).append(":"); - mbeanName.append("type=").append(type); + mbeanName.append(GROUP_NAME).append(":"); + mbeanName.append("type=").append(TYPE_NAME); mbeanName.append(",keyspace=").append(keyspace); mbeanName.append(",scope=").append(table); mbeanName.append(",name=").append(metricName); - return new CassandraMetricsRegistry.MetricName(groupName, type, metricName, keyspace + "." + table, mbeanName.toString()); + return new CassandraMetricsRegistry.MetricName(GROUP_NAME, TYPE_NAME, metricName, keyspace + '.' + table, mbeanName.toString()); } } } diff --git a/src/java/org/apache/cassandra/metrics/UnweightedCacheMetrics.java b/src/java/org/apache/cassandra/metrics/UnweightedCacheMetrics.java index ff40f3a03853..b651a5199547 100644 --- a/src/java/org/apache/cassandra/metrics/UnweightedCacheMetrics.java +++ b/src/java/org/apache/cassandra/metrics/UnweightedCacheMetrics.java @@ -27,6 +27,7 @@ */ public class UnweightedCacheMetrics extends AbstractCacheMetrics { + public static final String TYPE_NAME = "UnweightedCache"; /** * Cache capacity (maximum number of entries) */ @@ -45,7 +46,7 @@ public class UnweightedCacheMetrics extends AbstractCacheMetrics */ public UnweightedCacheMetrics(String type, UnweightedCacheSize cache) { - super(new DefaultNameFactory("UnweightedCache", type), type); + super(new DefaultNameFactory(TYPE_NAME, type), type); maxEntries = Metrics.register(factory.createMetricName("MaxEntries"), cache::maxEntries); entries = Metrics.register(factory.createMetricName("Entries"), cache::entries); diff --git a/src/java/org/apache/cassandra/schema/SchemaConstants.java b/src/java/org/apache/cassandra/schema/SchemaConstants.java index a6561e2522e4..bc733d367e1f 100644 --- a/src/java/org/apache/cassandra/schema/SchemaConstants.java +++ b/src/java/org/apache/cassandra/schema/SchemaConstants.java @@ -50,6 +50,7 @@ public final class SchemaConstants public static final String VIRTUAL_SCHEMA = "system_virtual_schema"; public static final String VIRTUAL_VIEWS = "system_views"; + public static final String VIRTUAL_METRICS = "system_metrics"; public static final String DUMMY_KEYSPACE_OR_TABLE_NAME = "--dummy--"; diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index 8518a9ba9953..23ed5e110298 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -32,12 +32,13 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; -import javax.management.ObjectName; import javax.management.StandardMBean; import javax.management.remote.JMXConnectorServer; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; +import org.apache.cassandra.db.virtual.VirtualKeyspace; +import org.apache.cassandra.db.virtual.VirtualSchemaKeyspace; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,7 +59,6 @@ import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.db.virtual.SystemViewsKeyspace; import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry; -import org.apache.cassandra.db.virtual.VirtualSchemaKeyspace; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.exceptions.StartupException; import org.apache.cassandra.io.util.File; @@ -100,6 +100,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VM_NAME; import static org.apache.cassandra.config.CassandraRelevantProperties.SIZE_RECORDER_INTERVAL; import static org.apache.cassandra.config.CassandraRelevantProperties.START_NATIVE_TRANSPORT; +import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_METRICS; /** * The CassandraDaemon is an abstraction for a Cassandra daemon @@ -132,8 +133,7 @@ public void onMeterAdded(String metricName, Meter meter) int separator = metricName.lastIndexOf('.'); String appenderName = metricName.substring(0, separator); String metric = metricName.substring(separator + 1); // remove "." - ObjectName name = DefaultNameFactory.createMetricName(appenderName, metric, null).getMBeanName(); - CassandraMetricsRegistry.Metrics.registerMBean(meter, name); + CassandraMetricsRegistry.Metrics.register(DefaultNameFactory.createMetricName(appenderName, metric, null), meter); } }); logger = LoggerFactory.getLogger(CassandraDaemon.class); @@ -546,6 +546,7 @@ public void setupVirtualKeyspaces() { VirtualKeyspaceRegistry.instance.register(VirtualSchemaKeyspace.instance); VirtualKeyspaceRegistry.instance.register(SystemViewsKeyspace.instance); + VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(VIRTUAL_METRICS, CassandraMetricsRegistry.createMetricsKeyspaceTables())); // flush log messages to system_views.system_logs virtual table as there were messages already logged // before that virtual table was instantiated diff --git a/src/java/org/apache/cassandra/utils/EstimatedHistogram.java b/src/java/org/apache/cassandra/utils/EstimatedHistogram.java index 198f92286f74..7cd8dc4a2076 100644 --- a/src/java/org/apache/cassandra/utils/EstimatedHistogram.java +++ b/src/java/org/apache/cassandra/utils/EstimatedHistogram.java @@ -442,4 +442,10 @@ public long serializedSize(EstimatedHistogram eh) return size; } } + + @Override + public String toString() + { + return String.valueOf(mean()); + } } diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index d3728f95d852..414811aff3e3 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -1402,6 +1402,9 @@ public static void preventIllegalAccessWarnings() public static String camelToSnake(String camel) { + if (camel.chars().allMatch(Character::isUpperCase)) + return camel.toLowerCase(); + StringBuilder sb = new StringBuilder(); for (char c : camel.toCharArray()) { diff --git a/src/java/org/apache/cassandra/utils/memory/MemtablePool.java b/src/java/org/apache/cassandra/utils/memory/MemtablePool.java index 26c47912a2eb..f77ea2fab110 100644 --- a/src/java/org/apache/cassandra/utils/memory/MemtablePool.java +++ b/src/java/org/apache/cassandra/utils/memory/MemtablePool.java @@ -29,6 +29,7 @@ import com.codahale.metrics.Timer; import org.apache.cassandra.metrics.CassandraMetricsRegistry; import org.apache.cassandra.metrics.DefaultNameFactory; +import org.apache.cassandra.metrics.MetricNameFactory; import org.apache.cassandra.utils.concurrent.WaitQueue; import org.apache.cassandra.utils.ExecutorUtils; @@ -41,6 +42,7 @@ */ public abstract class MemtablePool { + public static final String TYPE_NAME = "MemtablePool"; final MemtableCleanerThread cleaner; // the total memory used by this pool @@ -59,7 +61,7 @@ public abstract class MemtablePool this.onHeap = getSubPool(maxOnHeapMemory, cleanThreshold); this.offHeap = getSubPool(maxOffHeapMemory, cleanThreshold); this.cleaner = getCleaner(cleaner); - DefaultNameFactory nameFactory = new DefaultNameFactory("MemtablePool"); + MetricNameFactory nameFactory = new DefaultNameFactory(TYPE_NAME); blockedOnAllocating = CassandraMetricsRegistry.Metrics.timer(nameFactory.createMetricName("BlockedOnAllocation")); numPendingTasks = CassandraMetricsRegistry.Metrics.register(nameFactory.createMetricName("PendingFlushTasks"), () -> (long) this.cleaner.numPendingTasks()); diff --git a/test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/Column.java b/test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/Column.java new file mode 100644 index 000000000000..5358627d5578 --- /dev/null +++ b/test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/Column.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.proc; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Annotation to mark a walk order for a {@link RowWalker}. Annotation is used on a method that returns a value and + * represents a column value for a row. The columns order is defined by the column type and the column name + * in lexicographical order (case-insensitive) and the same way as in CQL (partition key columns first, + * then clustering columns, then regular columns). + *

+ * The column {@link org.apache.cassandra.db.virtual.proc.Column.Type} type is the same as the existing CQL + * {@code ColumnMetadata.Kind} and is used to define the column types for the virtual tables. All the classes + * in the {@link org.apache.cassandra.db.virtual.proc} package are used by annotation processor to generate + * {@link RowWalker} implementations, so they should have zero dependencies on the rest of the codebase. + */ +@Retention(RetentionPolicy.SOURCE) +@Target(ElementType.METHOD) +public @interface Column +{ + /** The type of the column */ + Type type() default Type.REGULAR; + + /** Be sure the order of the enum values is preserved as they are used in ordinal comparisons for the virtual tables. */ + enum Type + { + /** The column is a partition key column. */ + PARTITION_KEY, + /** The column is a clustering column. */ + CLUSTERING, + /** The column is a regular column. */ + REGULAR + } +} diff --git a/test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/RowWalker.java b/test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/RowWalker.java new file mode 100644 index 000000000000..15d8b3d34475 --- /dev/null +++ b/test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/RowWalker.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.proc; + +import java.util.function.Supplier; + +/** + * Utility class for quick iteration over row attributes and row values. + * Walk order is defined by {@link org.apache.cassandra.db.virtual.proc.Column} annotations and is the same as + * in CQL (partition key columns first, then clustering columns, then regular columns). + */ +public interface RowWalker +{ + int count(Column.Type type); + void visitMeta(MetadataVisitor visitor); + void visitRow(R row, RowMetadataVisitor visitor); + + interface MetadataVisitor + { + void accept(Column.Type type, String name, Class clazz); + } + + interface RowMetadataVisitor + { + void accept(Column.Type type, String name, Class clazz, Supplier value); + } +} diff --git a/test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/SystemViewAnnotationProcessor.java b/test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/SystemViewAnnotationProcessor.java new file mode 100644 index 000000000000..a8795597e069 --- /dev/null +++ b/test/annotation-processor/src/main/java/org/apache/cassandra/db/virtual/proc/SystemViewAnnotationProcessor.java @@ -0,0 +1,320 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.proc; + +import javax.annotation.processing.AbstractProcessor; +import javax.annotation.processing.RoundEnvironment; +import javax.lang.model.SourceVersion; +import javax.lang.model.element.Element; +import javax.lang.model.element.TypeElement; +import javax.lang.model.type.ExecutableType; +import javax.lang.model.type.TypeKind; +import javax.tools.Diagnostic; +import javax.tools.JavaFileObject; +import java.io.IOException; +import java.io.PrintWriter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +/** Generates {@link RowWalker} implementations for {@link Column} annotated class methods. */ +public class SystemViewAnnotationProcessor extends AbstractProcessor +{ + private static final Set SYS_METHODS = new HashSet<>(Arrays.asList("equals", "hashCode", "toString", + "getClass")); + private static final String TAB = " "; + private static final String WALKER_SUFFIX = "Walker"; + private static final Pattern DOLLAR_PATTERN = Pattern.compile("\\$"); + private static final Map> namePrimitiveMap = new HashMap<>(); + private static final Map, Class> primitiveWrapperMap = new HashMap<>(); + private static final Pattern CAMEL_SNAKE_PATTERN = Pattern.compile("\\B([A-Z])"); + + static + { + namePrimitiveMap.put("boolean", Boolean.TYPE); + namePrimitiveMap.put("byte", Byte.TYPE); + namePrimitiveMap.put("char", Character.TYPE); + namePrimitiveMap.put("short", Short.TYPE); + namePrimitiveMap.put("int", Integer.TYPE); + namePrimitiveMap.put("long", Long.TYPE); + namePrimitiveMap.put("double", Double.TYPE); + namePrimitiveMap.put("float", Float.TYPE); + namePrimitiveMap.put("void", Void.TYPE); + primitiveWrapperMap.put(Boolean.TYPE, Boolean.class); + primitiveWrapperMap.put(Byte.TYPE, Byte.class); + primitiveWrapperMap.put(Character.TYPE, Character.class); + primitiveWrapperMap.put(Short.TYPE, Short.class); + primitiveWrapperMap.put(Integer.TYPE, Integer.class); + primitiveWrapperMap.put(Long.TYPE, Long.class); + primitiveWrapperMap.put(Double.TYPE, Double.class); + primitiveWrapperMap.put(Float.TYPE, Float.class); + primitiveWrapperMap.put(Void.TYPE, Void.TYPE); + } + + @Override + public boolean process(Set annotations, RoundEnvironment roundEnv) + { + for (TypeElement annotation : annotations) + { + Set annotatedElements = roundEnv.getElementsAnnotatedWith(annotation); + + List getters = annotatedElements.stream() + .filter(element -> !SYS_METHODS.contains(element.getSimpleName().toString())) + .filter(element -> element.getModifiers().contains(javax.lang.model.element.Modifier.PUBLIC)) + .filter(element -> ((ExecutableType) element.asType()).getReturnType().getKind() != TypeKind.VOID) + .filter(element -> ((ExecutableType) element.asType()).getParameterTypes().isEmpty()) + .collect(Collectors.toList()); + + if (getters.isEmpty()) + continue; + + if (getters.size() != annotatedElements.size()) + { + processingEnv.getMessager().printMessage(Diagnostic.Kind.ERROR, + "@Column must be applied to a method without an argument and can't have void type"); + return false; + } + + Map> gettersByClass = getters.stream() + .collect(Collectors.groupingBy(element -> ((TypeElement) element.getEnclosingElement()).getQualifiedName().toString())); + + + for (Map.Entry> classEntry : gettersByClass.entrySet()) + { + String className = classEntry.getKey(); + Collection code = generate(className, classEntry.getValue()); + try + { + JavaFileObject builderFile = processingEnv.getFiler().createSourceFile(className + WALKER_SUFFIX); + try (PrintWriter writer = new PrintWriter(builderFile.openWriter())) + { + for (String line : code) + { + writer.write(line); + writer.write('\n'); + } + } + } catch (IOException e) + { + processingEnv.getMessager().printMessage(Diagnostic.Kind.ERROR, e.getMessage()); + return false; + } + } + } + + return true; + } + + /** + * Generates {@link RowWalker} implementation. + */ + private Collection generate(String className, List columns) + { + final List code = new ArrayList<>(); + final Set imports = new TreeSet<>(Comparator.comparing(s -> s.replace(";", ""))); + String packageName = className.substring(0, className.lastIndexOf('.')); + String simpleClassName = className.substring(className.lastIndexOf('.') + 1); + + addImport(imports, Column.class.getName()); + addImport(imports, RowWalker.class.getName()); + addImport(imports, className); + + code.add("package " + packageName + ';'); + code.add(""); + code.add(""); + code.add("/**"); + code.add(" * Generated by {@code " + SystemViewAnnotationProcessor.class.getName() + "}."); + code.add(" * {@link " + simpleClassName + "} row metadata and data walker."); + code.add(" * "); + code.add(" * @see " + simpleClassName); + code.add(" */"); + code.add("public class " + simpleClassName + WALKER_SUFFIX + " implements RowWalker<" + simpleClassName + '>'); + code.add("{"); + code.add(TAB + "/** {@inheritDoc} */"); + code.add(TAB + "@Override public void visitMeta(RowWalker.MetadataVisitor visitor)"); + code.add(TAB + '{'); + + forEachColumn(columns, (method, annotation) -> { + String name = method.getSimpleName().toString(); + String returnType = ((ExecutableType) method.asType()).getReturnType().toString(); + + if (!isPrimitive(returnType) && !returnType.startsWith("java.lang")) + addImport(imports, returnType); + + String line = TAB + TAB + + "visitor.accept(" + innerClassName(Column.Type.class.getName()) + '.' + annotation.type() + ", \"" + camelToSnake(name) + "\", " + + getPrimitiveWrapperClass(returnType) + + (isPrimitive(returnType) ? ".TYPE);" : ".class);"); + + code.add(line); + }); + + code.add(TAB + '}'); + code.add(""); + code.add(TAB + "/** {@inheritDoc} */"); + code.add(TAB + "@Override public void visitRow(" + simpleClassName + " row, RowWalker.RowMetadataVisitor visitor)"); + code.add(TAB + '{'); + + forEachColumn(columns, (method, annotation) -> { + String name = method.getSimpleName().toString(); + String returnType = ((ExecutableType) method.asType()).getReturnType().toString(); + String line = TAB + TAB + + "visitor.accept(" + innerClassName(Column.Type.class.getName()) + '.' + annotation.type() + ", \"" + camelToSnake(name) + "\", " + + getPrimitiveWrapperClass(returnType) + + (isPrimitive(returnType) ? ".TYPE, row::" : ".class, row::") + + name + ");"; + code.add(line); + }); + + code.add(TAB + '}'); + code.add(""); + + Map countsMap = Arrays.stream(Column.Type.values()) + .collect(Collectors.toMap(e -> e, e -> new AtomicInteger())); + forEachColumn(columns, (method, annotation) -> countsMap.get(annotation.type()).incrementAndGet()); + + code.add(TAB + "/** {@inheritDoc} */"); + code.add(TAB + "@Override"); + code.add(TAB + "public int count(" + innerClassName(Column.Type.class.getName()) + " type)"); + code.add(TAB + '{'); + code.add(TAB + TAB + "switch (type)"); + code.add(TAB + TAB + '{'); + countsMap.forEach((key, value) -> code.add(TAB + TAB + TAB + "case " + key + ": return " + value + ';')); + code.add(TAB + TAB + TAB +"default: throw new IllegalStateException(\"Unknown column type: \" + type);"); + code.add(TAB + TAB + '}'); + code.add(TAB + '}'); + code.add("}"); + + code.addAll(2, imports); + + addLicenseHeader(code); + + return code; + } + + private static String innerClassName(String className) + { + String classNameDotted = DOLLAR_PATTERN.matcher(className).replaceAll("."); + String basicClassName = classNameDotted.substring(0, classNameDotted.lastIndexOf('.')); + return classNameDotted.substring(basicClassName.lastIndexOf('.') + 1); + } + + private void addImport(Set imports, String className) + { + imports.add("import " + DOLLAR_PATTERN.matcher(className).replaceAll(".") + ';'); + } + + private void addLicenseHeader(List code) + { + List license = new ArrayList<>(); + + license.add("/*"); + license.add(" * Licensed to the Apache Software Foundation (ASF) under one or more"); + license.add(" * contributor license agreements. See the NOTICE file distributed with"); + license.add(" * this work for additional information regarding copyright ownership."); + license.add(" * The ASF licenses this file to You under the Apache License, Version 2.0"); + license.add(" * (the \"License\"); you may not use this file except in compliance with"); + license.add(" * the License. You may obtain a copy of the License at"); + license.add(" *"); + license.add(" * https://www.apache.org/licenses/LICENSE-2.0"); + license.add(" *"); + license.add(" * Unless required by applicable law or agreed to in writing, software"); + license.add(" * distributed under the License is distributed on an \"AS IS\" BASIS,"); + license.add(" * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied."); + license.add(" * See the License for the specific language governing permissions and"); + license.add(" * limitations under the License."); + license.add(" */"); + license.add(""); + code.addAll(0, license); + } + + /** + * Iterates each over the {@code columns} and consumes {@code method} for it. + */ + private static void forEachColumn(List columns, BiConsumer consumer) + { + // sort columns by type (partition key, clustering, regular) and then by column name + columns.stream() + .sorted(new ColumnAnnotationComparator()) + .forEach(method -> consumer.accept(method, method.getAnnotation(Column.class))); + } + + private static class ColumnAnnotationComparator implements Comparator { + @Override + public int compare(Element m1, Element m2) { + Column.Type type1 = getColumnType(m1); + Column.Type type2 = getColumnType(m2); + + if (type1 == type2) + return methodName(m1).compareTo(methodName(m2)); + + return type1.compareTo(type2); + } + + private static String methodName(Element method) + { + return method.getSimpleName().toString().toLowerCase(Locale.US); + } + + private static Column.Type getColumnType(Element method) { + return method.getAnnotation(Column.class).type(); + } + } + + public static boolean isPrimitive(String className) + { + return namePrimitiveMap.containsKey(className); + } + + public static String getPrimitiveWrapperClass(String className) + { + return isPrimitive(className) ? + primitiveWrapperMap.get(namePrimitiveMap.get(className)).getSimpleName() : className; + } + + private static String camelToSnake(String camelCase) + { + return CAMEL_SNAKE_PATTERN.matcher(camelCase).replaceAll("_$1").toLowerCase(); + } + + @Override + public SourceVersion getSupportedSourceVersion() + { + return SourceVersion.latest(); + } + + @Override + public Set getSupportedAnnotationTypes() + { + return Collections.singleton("org.apache.cassandra.db.virtual.proc.Column"); + } +} diff --git a/test/annotation-processor/src/main/resources/services/javax.annotation.processing.Processor b/test/annotation-processor/src/main/resources/services/javax.annotation.processing.Processor new file mode 100644 index 000000000000..2ff9ae15d326 --- /dev/null +++ b/test/annotation-processor/src/main/resources/services/javax.annotation.processing.Processor @@ -0,0 +1,18 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# +org.apache.cassandra.db.virtual.proc.SystemViewAnnotationProcessor diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 18beee3d2a5d..8b728ae5d1df 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -17,65 +17,6 @@ */ package org.apache.cassandra.cql3; -import java.io.IOException; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.MalformedURLException; -import java.net.ServerSocket; -import java.nio.ByteBuffer; -import java.rmi.server.RMISocketFactory; -import java.util.AbstractList; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Consumer; -import java.util.function.Supplier; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Collectors; -import javax.annotation.Nullable; -import javax.management.MBeanServerConnection; -import javax.management.remote.JMXConnector; -import javax.management.remote.JMXConnectorFactory; -import javax.management.remote.JMXConnectorServer; -import javax.management.remote.JMXServiceURL; -import javax.management.remote.rmi.RMIConnectorServer; -import javax.net.ssl.SSLException; - -import com.google.common.base.Objects; -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; -import org.apache.commons.lang3.ArrayUtils; -import org.apache.commons.lang3.StringUtils; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.TestName; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import com.codahale.metrics.Gauge; import com.datastax.driver.core.CloseFuture; import com.datastax.driver.core.Cluster; @@ -88,10 +29,15 @@ import com.datastax.driver.core.SimpleStatement; import com.datastax.driver.core.SocketOptions; import com.datastax.driver.core.Statement; +import com.datastax.driver.core.TypeCodec; import com.datastax.driver.core.UDTValue; import com.datastax.driver.core.UserType; import com.datastax.driver.core.exceptions.UnauthorizedException; import com.datastax.shaded.netty.channel.EventLoopGroup; +import com.google.common.base.Objects; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.ServerTestUtils; import org.apache.cassandra.Util; @@ -134,6 +80,7 @@ import org.apache.cassandra.db.marshal.UTF8Type; import org.apache.cassandra.db.marshal.UUIDType; import org.apache.cassandra.db.marshal.VectorType; +import org.apache.cassandra.db.virtual.VirtualKeyspace; import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry; import org.apache.cassandra.db.virtual.VirtualSchemaKeyspace; import org.apache.cassandra.dht.Murmur3Partitioner; @@ -160,9 +107,9 @@ import org.apache.cassandra.service.ClientState; import org.apache.cassandra.service.QueryState; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.transport.Event; import org.apache.cassandra.transport.Message; -import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.transport.ProtocolVersion; import org.apache.cassandra.transport.Server; import org.apache.cassandra.transport.SimpleClient; @@ -173,8 +120,65 @@ import org.apache.cassandra.utils.JMXServerUtils; import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.TimeUUID; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.commons.lang3.StringUtils; import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.management.MBeanServerConnection; +import javax.management.remote.JMXConnector; +import javax.management.remote.JMXConnectorFactory; +import javax.management.remote.JMXConnectorServer; +import javax.management.remote.JMXServiceURL; +import javax.management.remote.rmi.RMIConnectorServer; +import javax.net.ssl.SSLException; +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.MalformedURLException; +import java.net.ServerSocket; +import java.nio.ByteBuffer; +import java.rmi.server.RMISocketFactory; +import java.util.AbstractList; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; +import java.util.function.Supplier; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_DRIVER_CONNECTION_TIMEOUT_MS; @@ -187,8 +191,9 @@ import static org.apache.cassandra.cql3.SchemaElement.SchemaElementType.MATERIALIZED_VIEW; import static org.apache.cassandra.cql3.SchemaElement.SchemaElementType.TABLE; import static org.apache.cassandra.cql3.SchemaElement.SchemaElementType.TYPE; -import static org.junit.Assert.assertNotNull; +import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_METRICS; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -393,6 +398,11 @@ public static void createMBeanServerConnection() throws Exception jmxConnection = jmxc.getMBeanServerConnection(); } + public static MBeanServerConnection getJmxConnection() + { + return jmxConnection; + } + public static JMXServiceURL getJMXServiceURL() throws MalformedURLException { assert jmxServer != null : "jmxServer not started"; @@ -476,6 +486,12 @@ public void afterTest() throws Throwable user = null; } + protected static void addMetricsKeyspace() + { + VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(VIRTUAL_METRICS, + CassandraMetricsRegistry.createMetricsKeyspaceTables())); + } + protected void resetSchema() throws Throwable { for (TableMetadata table : SchemaKeyspace.metadata().tables) @@ -1577,7 +1593,7 @@ private Session getSession(ProtocolVersion protocolVersion) return sessions.computeIfAbsent(new ClusterSettings(user, protocolVersion, useEncryption, useClientCert), settings -> cluster.connect()); } - private Cluster getCluster(ProtocolVersion protocolVersion) + protected Cluster getCluster(ProtocolVersion protocolVersion) { return clusters.computeIfAbsent(new ClusterSettings(user, protocolVersion, useEncryption, useClientCert), settings -> initClientCluster(user, protocolVersion, useEncryption, useClientCert)); } @@ -1763,6 +1779,96 @@ protected void assertRowsNet(ProtocolVersion protocolVersion, ResultSet result, rows.length>i ? "less" : "more", rows.length, i, protocolVersion), i == rows.length); } + public void assertRowsContains(ResultSet result, Object[]... rows) + { + assertRowsContains(getCluster(getDefaultVersion()), result, rows); + } + + public void assertRowsContains(ResultSet result, List rows) + { + assertRowsContains(getCluster(getDefaultVersion()), result, rows); + } + + public static void assertRowsContains(Cluster cluster, ResultSet result, Object[]... rows) + { + assertRowsContains(cluster, result, rows == null ? Collections.emptyList() : Arrays.asList(rows)); + } + + public static void assertRowsContains(Cluster cluster, ResultSet result, List rows) + { + if (result == null && rows.isEmpty()) + return; + assertNotNull(String.format("No rows returned by query but %d expected", rows.size()), result); + assertTrue(result.iterator().hasNext()); + + // It is necessary that all rows match the column definitions + for (Object[] row : rows) + { + if (row == null || row.length == 0) + Assert.fail("Rows must not be null or empty"); + + if (result.getColumnDefinitions().size() == row.length) + continue; + + Assert.fail(String.format("Rows do not match column definitions. Expected %d columns but got %d", + result.getColumnDefinitions().size(), row.length)); + } + + ColumnDefinitions defs = result.getColumnDefinitions(); + int size = defs.size(); + List resultSetValues = StreamSupport.stream(result.spliterator(), false) + .map(row -> IntStream.range(0, size) + .mapToObj(row::getBytesUnsafe) + .toArray(ByteBuffer[]::new)) + .collect(Collectors.toList()); + + AtomicInteger columnCounter = new AtomicInteger(); + com.datastax.driver.core.ProtocolVersion version = com.datastax.driver.core.ProtocolVersion.fromInt(getDefaultVersion().asInt()); + List expectedRowsValues = rows.stream() + .map(row -> Stream.of(row) + .map(cell -> { + int index = columnCounter.getAndIncrement() % size; + return cell instanceof ByteBuffer ? (ByteBuffer) cell : + cluster.getConfiguration() + .getCodecRegistry() + .codecFor(defs.getType(index)) + .serialize(cell, version); + }) + .toArray(ByteBuffer[]::new)) + .collect(Collectors.toList()); + + int found = 0; + for (ByteBuffer[] expected : expectedRowsValues) + for (ByteBuffer[] actual : resultSetValues) + if (Arrays.equals(expected, actual)) + found++; + + if (found == expectedRowsValues.size()) + return; + + Assert.fail(String.format("Result set does not contain expected rows. Result set %s doesn't contain %s", + resultSetValues.stream().map(e -> deserializeCells(e, cluster.getConfiguration().getCodecRegistry(), + result.getColumnDefinitions(), version)).collect(Collectors.joining(", ")), + expectedRowsValues.stream().map(e -> deserializeCells(e, cluster.getConfiguration().getCodecRegistry(), + result.getColumnDefinitions(), version)).collect(Collectors.joining(", ")))); + } + + private static String deserializeCells(ByteBuffer[] row, + com.datastax.driver.core.CodecRegistry codecRegistry, + ColumnDefinitions columnDefinitions, + com.datastax.driver.core.ProtocolVersion version) + { + AtomicInteger index = new AtomicInteger(); + int size = columnDefinitions.size(); + return Stream.of(row) + .map(b -> { + int idx = index.getAndIncrement() % size; + TypeCodec codec = codecRegistry.codecFor(columnDefinitions.getType(idx)); + return codec.format(codec.deserialize(b, version)); + }) + .collect(Collectors.joining(", ", "[", "]")); + } + private static String safeToString(Supplier fn) { try diff --git a/test/unit/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapterTest.java b/test/unit/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapterTest.java new file mode 100644 index 000000000000..98e5560afec9 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapterTest.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual; + +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Row; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.virtual.model.CollectionEntry; +import org.apache.cassandra.db.virtual.model.CollectionEntryTestRow; +import org.apache.cassandra.db.virtual.model.CollectionEntryTestRowWalker; +import org.apache.cassandra.db.virtual.model.PartitionEntryTestRow; +import org.apache.cassandra.db.virtual.model.PartitionEntryTestRowWalker; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.Assert.assertEquals; + +public class CollectionVirtualTableAdapterTest extends CQLTester +{ + private static final String KS_NAME = "vts"; + private static final String VT_NAME = "collection_virtual_table"; + private static final String VT_NAME_1 = "map_key_filter_virtual_table"; + private static final String VT_NAME_2 = "map_value_filter_virtual_table"; + private final List tables = new ArrayList<>(); + private final List internalTestCollection = new ArrayList<>(); + private final Map internalTestMap = new HashMap<>(); + + private static void addSinglePartitionData(Collection list) + { + list.add(new CollectionEntry("1984", "key", 3, "value", + 1, 1, 1, (short) 1, (byte) 1, true)); + list.add(new CollectionEntry("1984", "key", 2, "value", + 1, 1, 1, (short) 1, (byte) 1, true)); + list.add(new CollectionEntry("1984", "key", 1, "value", + 1, 1, 1, (short) 1, (byte) 1, true)); + } + + private static void addMultiPartitionData(Collection list) + { + addSinglePartitionData(list); + list.add(new CollectionEntry("1985", "key", 3, "value", + 1, 1, 1, (short) 1, (byte) 1, true)); + list.add(new CollectionEntry("1985", "key", 2, "value", + 1, 1, 1, (short) 1, (byte) 1, true)); + list.add(new CollectionEntry("1985", "key", 1, "value", + 1, 1, 1, (short) 1, (byte) 1, true)); + } + + @Before + public void config() throws Exception + { + tables.add(CollectionVirtualTableAdapter.create( + KS_NAME, + VT_NAME, + "The collection virtual table", + new CollectionEntryTestRowWalker(), + internalTestCollection, + CollectionEntryTestRow::new)); + tables.add(CollectionVirtualTableAdapter.createSinglePartitionedKeyFiltered( + KS_NAME, + VT_NAME_1, + "The partition key filtered virtual table", + new PartitionEntryTestRowWalker(), + internalTestMap, + internalTestMap::containsKey, + PartitionEntryTestRow::new)); + tables.add(CollectionVirtualTableAdapter.createSinglePartitionedValueFiltered( + KS_NAME, + VT_NAME_2, + "The partition value filtered virtual table", + new PartitionEntryTestRowWalker(), + internalTestMap, + value -> value instanceof CollectionEntryExt, + PartitionEntryTestRow::new)); + VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, tables)); + } + + @After + public void postCleanup() + { + internalTestCollection.clear(); + internalTestMap.clear(); + } + + @Test + public void testSelectAll() + { + addSinglePartitionData(internalTestCollection); + List sortedClustering = new ArrayList<>(internalTestCollection); + sortedClustering.sort(Comparator.comparingLong(CollectionEntry::getOrderedKey)); + ResultSet result = executeNet(String.format("SELECT * FROM %s.%s", KS_NAME, VT_NAME)); + + int index = 0; + for (Row row : result) + { + assertEquals(sortedClustering.get(index).getPrimaryKey(), row.getString("primary_key")); + assertEquals(sortedClustering.get(index).getSecondaryKey(), row.getString("secondary_key")); + assertEquals(sortedClustering.get(index).getOrderedKey(), row.getLong("ordered_key")); + assertEquals(sortedClustering.get(index).getIntValue(), row.getInt("int_value")); + assertEquals(sortedClustering.get(index).getLongValue(), row.getLong("long_value")); + assertEquals(sortedClustering.get(index).getValue(), row.getString("value")); + assertEquals(sortedClustering.get(index).getDoubleValue(), row.getDouble("double_value"), 0.0); + assertEquals(sortedClustering.get(index).getShortValue(), row.getShort("short_value")); + assertEquals(sortedClustering.get(index).getByteValue(), row.getByte("byte_value")); + assertEquals(sortedClustering.get(index).getBooleanValue(), row.getBool("boolean_value")); + index++; + } + assertEquals(sortedClustering.size(), index); + } + + @Test + public void testSelectPartition() + { + addMultiPartitionData(internalTestCollection); + ResultSet result = executeNet(String.format("SELECT * FROM %s.%s WHERE primary_key = ? AND secondary_key = ?", KS_NAME, VT_NAME), + "1984", "key"); + + AtomicInteger size = new AtomicInteger(3); + result.forEach(row -> { + assertEquals("1984", row.getString("primary_key")); + assertEquals("key", row.getString("secondary_key")); + size.decrementAndGet(); + }); + assertEquals(0, size.get()); + } + + @Test + public void testSelectPartitionMap() + { + internalTestMap.put("1984", new CollectionEntry("primary", "key", 3, "value", + 1, 1, 1, (short) 1, (byte) 1, true)); + ResultSet result = executeNet(String.format("SELECT * FROM %s.%s WHERE key = ?", KS_NAME, VT_NAME_1), + "1984"); + + AtomicInteger size = new AtomicInteger(1); + result.forEach(row -> { + assertEquals("1984", row.getString("key")); + assertEquals("primary", row.getString("primary_key")); + assertEquals("key", row.getString("secondary_key")); + size.decrementAndGet(); + }); + assertEquals(0, size.get()); + } + + @Test + public void testSelectPartitionUnknownKey() + { + internalTestMap.put("1984", new CollectionEntry("primary", "key", 3, "value", + 1, 1, 1, (short) 1, (byte) 1, true)); + ResultSet first = executeNet(String.format("SELECT * FROM %s.%s WHERE key = ?", KS_NAME, VT_NAME_1), + "unknown"); + assertEquals(0, first.all().size()); + + addSinglePartitionData(internalTestCollection); + ResultSet second = executeNet(String.format("SELECT * FROM %s.%s WHERE primary_key = ? AND secondary_key = ?", KS_NAME, VT_NAME), + "unknown", "key"); + assertEquals(0, second.all().size()); + } + + @Test + public void testSelectPartitionValueFitered() + { + internalTestMap.put("1984", new CollectionEntry("primary", "key", 3, "value", + 1, 1, 1, (short) 1, (byte) 1, true)); + internalTestMap.put("1985", new CollectionEntryExt("primary", "key", 3, "value", + 1, 1, 1, (short) 1, (byte) 1, true, "extra")); + + ResultSet first = executeNet(String.format("SELECT * FROM %s.%s WHERE key = ?", KS_NAME, VT_NAME_2), + "1984"); + assertEquals(0, first.all().size()); + + ResultSet second = executeNet(String.format("SELECT * FROM %s.%s WHERE key = ?", KS_NAME, VT_NAME_2), + "1985"); + assertEquals(1, second.all().size()); + } + + @Test + public void testSelectEmptyPartition() + { + addSinglePartitionData(internalTestCollection); + assertRowsNet(executeNet(String.format("SELECT * FROM %s.%s WHERE primary_key = 'EMPTY'", KS_NAME, VT_NAME))); + } + + @Test + public void testSelectEmptyCollection() + { + internalTestCollection.clear(); + assertRowsNet(executeNet(String.format("SELECT * FROM %s.%s", KS_NAME, VT_NAME))); + } + + private static class CollectionEntryExt extends CollectionEntry + { + public CollectionEntryExt(String primaryKey, String secondaryKey, long orderedKey, String value, int intValue, + long longValue, double doubleValue, short shortValue, byte byteValue, boolean booleanValue, + String extraColumn) + { + super(primaryKey, secondaryKey, orderedKey, value, intValue, longValue, doubleValue, shortValue, byteValue, booleanValue); + } + } +} diff --git a/test/unit/org/apache/cassandra/db/virtual/model/CollectionEntry.java b/test/unit/org/apache/cassandra/db/virtual/model/CollectionEntry.java new file mode 100644 index 000000000000..ccba6cbad053 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/virtual/model/CollectionEntry.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.model; + +/** + * Test collection entry represents an internal collection entry of a Cassandra collection. + */ +public class CollectionEntry +{ + private final String primaryKey; + private final String secondaryKey; + private final long orderedKey; + private final String value; + private final int intValue; + private final long longValue; + private final double doubleValue; + private final short shortValue; + private final byte byteValue; + private final boolean booleanValue; + + public CollectionEntry(String primaryKey, String secondaryKey, long orderedKey, String value, int intValue, + long longValue, double doubleValue, short shortValue, byte byteValue, boolean booleanValue) + { + this.primaryKey = primaryKey; + this.secondaryKey = secondaryKey; + this.orderedKey = orderedKey; + this.value = value; + this.intValue = intValue; + this.longValue = longValue; + this.doubleValue = doubleValue; + this.shortValue = shortValue; + this.byteValue = byteValue; + this.booleanValue = booleanValue; + } + + public String getPrimaryKey() + { + return primaryKey; + } + + public String getSecondaryKey() + { + return secondaryKey; + } + + public long getOrderedKey() + { + return orderedKey; + } + + public String getValue() + { + return value; + } + + public int getIntValue() + { + return intValue; + } + + public long getLongValue() + { + return longValue; + } + + public double getDoubleValue() + { + return doubleValue; + } + + public short getShortValue() + { + return shortValue; + } + + public byte getByteValue() + { + return byteValue; + } + + public boolean getBooleanValue() + { + return booleanValue; + } + + @Override + public String toString() + { + return "CollectionEntry{" + + "primaryKey='" + primaryKey + '\'' + + ", secondaryKey='" + secondaryKey + '\'' + + ", orderedKey=" + orderedKey + + ", value='" + value + '\'' + + ", intValue=" + intValue + + ", longValue=" + longValue + + ", doubleValue=" + doubleValue + + ", shortValue=" + shortValue + + ", byteValue=" + byteValue + + ", booleanValue=" + booleanValue + + '}'; + } +} diff --git a/test/unit/org/apache/cassandra/db/virtual/model/CollectionEntryTestRow.java b/test/unit/org/apache/cassandra/db/virtual/model/CollectionEntryTestRow.java new file mode 100644 index 000000000000..dce9ccc4a224 --- /dev/null +++ b/test/unit/org/apache/cassandra/db/virtual/model/CollectionEntryTestRow.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.model; + +import org.apache.cassandra.db.virtual.proc.Column; + + +/** + * Test meter metric test representation for a {@link org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter}. + */ +public class CollectionEntryTestRow +{ + private final CollectionEntry collectionEntry; + + public CollectionEntryTestRow(CollectionEntry collectionEntry) + { + this.collectionEntry = collectionEntry; + } + + @Column(type = Column.Type.PARTITION_KEY) + public String primaryKey() + { + return collectionEntry.getPrimaryKey(); + } + + @Column(type = Column.Type.PARTITION_KEY) + public String secondaryKey() + { + return collectionEntry.getSecondaryKey(); + } + + @Column(type = Column.Type.CLUSTERING) + public long orderedKey() + { + return collectionEntry.getOrderedKey(); + } + + @Column + public long longValue() + { + return collectionEntry.getLongValue(); + } + + @Column + public int intValue() + { + return collectionEntry.getIntValue(); + } + + @Column + public short shortValue() + { + return collectionEntry.getShortValue(); + } + + @Column + public byte byteValue() + { + return collectionEntry.getByteValue(); + } + + @Column + public double doubleValue() + { + return collectionEntry.getDoubleValue(); + } + + @Column + public String value() + { + return collectionEntry.getValue(); + } + + @Column + public boolean booleanValue() + { + return collectionEntry.getBooleanValue(); + } +} diff --git a/test/unit/org/apache/cassandra/db/virtual/model/PartitionEntryTestRow.java b/test/unit/org/apache/cassandra/db/virtual/model/PartitionEntryTestRow.java new file mode 100644 index 000000000000..8dd14471496b --- /dev/null +++ b/test/unit/org/apache/cassandra/db/virtual/model/PartitionEntryTestRow.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.db.virtual.model; + +import org.apache.cassandra.db.virtual.proc.Column; + + +/** + * Test meter metric test representation for a {@link org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter}. + */ +public class PartitionEntryTestRow +{ + private final String key; + private final CollectionEntry collectionEntry; + + public PartitionEntryTestRow(String key, CollectionEntry collectionEntry) + { + this.key = key; + this.collectionEntry = collectionEntry; + } + + @Column(type = Column.Type.PARTITION_KEY) + public String key() + { + return key; + } + + @Column + public String primaryKey() + { + return collectionEntry.getPrimaryKey(); + } + + @Column + public String secondaryKey() + { + return collectionEntry.getSecondaryKey(); + } +} diff --git a/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java b/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java index b90f19a7fc51..f6823ce0b693 100644 --- a/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java +++ b/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java @@ -34,6 +34,8 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.service.EmbeddedCassandraService; +import static org.apache.cassandra.cql3.CQLTester.assertRowsContains; +import static org.apache.cassandra.cql3.CQLTester.row; import static org.apache.cassandra.cql3.statements.BatchStatement.metrics; import static org.apache.cassandra.metrics.DecayingEstimatedHistogramReservoir.EstimatedHistogramReservoirSnapshot; import static org.apache.cassandra.metrics.DecayingEstimatedHistogramReservoir.Range; @@ -164,6 +166,14 @@ private void assertMetrics(BatchStatement.Type batchTypeTested, int[] rounds, in assertEquals(expectedPartitionsPerLoggedBatchCount, metrics.partitionsPerLoggedBatch.getCount()); assertEquals(expectedPartitionsPerCounterBatchCount, metrics.partitionsPerCounterBatch.getCount()); + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.batch_group"), + row("org.apache.cassandra.metrics.Batch.PartitionsPerUnloggedBatch", + "unknown", "histogram", String.valueOf(metrics.partitionsPerUnloggedBatch.getSnapshot().getMedian())), + row("org.apache.cassandra.metrics.Batch.PartitionsPerLoggedBatch", + "unknown", "histogram", String.valueOf(metrics.partitionsPerLoggedBatch.getSnapshot().getMedian())), + row("org.apache.cassandra.metrics.Batch.PartitionsPerCounterBatch", + "unknown", "histogram", String.valueOf(metrics.partitionsPerCounterBatch.getSnapshot().getMedian()))); + EstimatedHistogramReservoirSnapshot partitionsPerLoggedBatchSnapshot = (EstimatedHistogramReservoirSnapshot) metrics.partitionsPerLoggedBatch.getSnapshot(); EstimatedHistogramReservoirSnapshot partitionsPerUnloggedBatchSnapshot = (EstimatedHistogramReservoirSnapshot) metrics.partitionsPerUnloggedBatch.getSnapshot(); EstimatedHistogramReservoirSnapshot partitionsPerCounterBatchSnapshot = (EstimatedHistogramReservoirSnapshot) metrics.partitionsPerCounterBatch.getSnapshot(); diff --git a/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java b/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java index c41ded37db00..31999a6931bd 100644 --- a/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java +++ b/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java @@ -18,13 +18,6 @@ package org.apache.cassandra.metrics; -import java.io.IOException; - -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - import com.datastax.driver.core.Cluster; import com.datastax.driver.core.PreparedStatement; import com.datastax.driver.core.Session; @@ -34,7 +27,15 @@ import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.service.EmbeddedCassandraService; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import static org.apache.cassandra.cql3.CQLTester.assertRowsContains; +import static org.apache.cassandra.cql3.CQLTester.row; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -85,6 +86,10 @@ public void testConnectionWithUseDisabled() { DatabaseDescriptor.setUseStatementsEnabled(true); } + + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.cql_group"), + row("org.apache.cassandra.metrics.CQL.UseStatementsExecuted", "unknown", "counter", + String.valueOf(useCountBefore))); } @Test @@ -96,6 +101,10 @@ public void testPreparedStatementsCount() Assert.assertEquals(useCountBefore + 1, QueryProcessor.metrics.useStatementsExecuted.getCount()); session.prepare("SELECT * FROM junit.metricstest WHERE id = ?"); assertEquals(n+2, (int) QueryProcessor.metrics.preparedStatementsCount.getValue()); + + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.cql_group"), + row("org.apache.cassandra.metrics.CQL.PreparedStatementsCount", "unknown", "gauge", + String.valueOf(QueryProcessor.metrics.preparedStatementsCount.getValue()))); } @Test @@ -128,6 +137,12 @@ public void testPreparedStatementsExecuted() assertEquals(10, QueryProcessor.metrics.preparedStatementsExecuted.getCount()); assertEquals(0, QueryProcessor.metrics.regularStatementsExecuted.getCount()); + + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.cql_group"), + row("org.apache.cassandra.metrics.CQL.RegularStatementsExecuted", "unknown", "counter", + String.valueOf(QueryProcessor.metrics.regularStatementsExecuted.getCount())), + row("org.apache.cassandra.metrics.CQL.PreparedStatementsExecuted", "unknown", "counter", + String.valueOf(QueryProcessor.metrics.preparedStatementsExecuted.getCount()))); } @Test @@ -145,6 +160,10 @@ public void testPreparedStatementsRatio() for (int i = 0; i < 10; i++) session.execute(String.format("INSERT INTO junit.metricstest (id, val) VALUES (%d, '%s')", i, "val" + i)); assertEquals(0.5, QueryProcessor.metrics.preparedStatementsRatio.getValue(), 0.0); + + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.cql_group"), + row("org.apache.cassandra.metrics.CQL.PreparedStatementsRatio", "unknown", "gauge", + String.valueOf(QueryProcessor.metrics.preparedStatementsRatio.getValue()))); } private void clearMetrics() diff --git a/test/unit/org/apache/cassandra/metrics/CassandraMetricsRegistryTest.java b/test/unit/org/apache/cassandra/metrics/CassandraMetricsRegistryTest.java index 02e4228c3f2c..b748d7127e1c 100644 --- a/test/unit/org/apache/cassandra/metrics/CassandraMetricsRegistryTest.java +++ b/test/unit/org/apache/cassandra/metrics/CassandraMetricsRegistryTest.java @@ -20,21 +20,35 @@ */ package org.apache.cassandra.metrics; -import static org.junit.Assert.*; - -import java.lang.management.ManagementFactory; -import java.util.Collection; -import java.util.concurrent.TimeUnit; - +import com.codahale.metrics.Meter; +import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricRegistryListener; import com.codahale.metrics.Timer; -import org.apache.cassandra.metrics.CassandraMetricsRegistry.MetricName; - -import org.junit.Test; - import com.codahale.metrics.jvm.BufferPoolMetricSet; import com.codahale.metrics.jvm.GarbageCollectorMetricSet; import com.codahale.metrics.jvm.MemoryUsageGaugeSet; +import org.apache.cassandra.metrics.CassandraMetricsRegistry.MetricName; import org.apache.cassandra.utils.EstimatedHistogram; +import org.junit.Before; +import org.junit.Test; + +import java.lang.management.ManagementFactory; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; public class CassandraMetricsRegistryTest @@ -44,6 +58,12 @@ private static class StrangeName$ { } + @Before + public void prepare() + { + CassandraMetricsRegistry.Metrics.getNames().forEach(CassandraMetricsRegistry.Metrics::remove); + } + @Test public void testChooseType() { @@ -90,6 +110,14 @@ public void testJvmMetricsRegistration() } } + @Test(expected = IllegalArgumentException.class) + public void testMetricSetRegistration() + { + CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics; + DefaultNameFactory factory = new DefaultNameFactory("ThreadPools", "TestMetricSetRegistration"); + registry.register(factory.createMetricName("jvm.buffers"), new BufferPoolMetricSet(ManagementFactory.getPlatformMBeanServer())); + } + @Test public void testDeltaBaseCase() { @@ -156,6 +184,83 @@ public void testTimer() assertEquals(expectedBucketsWithValues, bucketsWithValues); } + @Test + public void testMetricAliasesOrder() + { + String dummy = "defaultName"; + LinkedList aliases = new LinkedList<>(); + int size = ThreadLocalRandom.current().nextInt(10, 1000); + MetricName first = DefaultNameFactory.createMetricName("Table", "FirstTestMetricAliasesOrder", "FirstScope"); + for (int i = 0; i < size; i++) + aliases.add(DefaultNameFactory.createMetricName("Table", "FirstTestMetricAliasesOrder" + UUID.randomUUID(), UUID.randomUUID().toString())); + + Meter metric = CassandraMetricsRegistry.Metrics.meter(dummy); + LinkedList verify = new LinkedList<>(aliases); + verify.addFirst(first); + MetricRegistryListener listener; + CassandraMetricsRegistry.Metrics.addListener(listener = new MetricRegistryListener.Base() + { + @Override + public void onMeterAdded(String name, Meter meter) + { + if (dummy.equals(name)) + return; + + assertEquals(verify.removeFirst().getMetricName(), name); + } + }); + CassandraMetricsRegistry.Metrics.register(first, metric, aliases.toArray(new MetricName[size])); + CassandraMetricsRegistry.Metrics.removeListener(listener); + + List all = CassandraMetricsRegistry.Metrics.getMetrics().keySet(). + stream() + .filter(m -> m.contains("FirstTestMetricAliasesOrder")) + .collect(Collectors.toList()); + + assertNotNull(all); + assertEquals(size + 1, all.size()); + + CassandraMetricsRegistry.Metrics.remove(first.getMetricName()); + Map metrics = CassandraMetricsRegistry.Metrics.getMetrics(); + assertEquals(1, metrics.size()); + assertEquals(dummy, metrics.keySet().iterator().next()); + + CassandraMetricsRegistry.Metrics.remove(dummy); + assertTrue(CassandraMetricsRegistry.Metrics.getMetrics().isEmpty()); + } + + @Test + public void testMetricAliasesRegistration() + { + CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics; + MetricNameFactory factory = new DefaultNameFactory("Table", "MetricsTestScope"); + MetricNameFactory aliasFactory = new DefaultNameFactory("Table", "MetricsTestAliasScope"); + + MetricName meter = factory.createMetricName("TestMeter"); + MetricName meterAlias = aliasFactory.createMetricName("TestMeterAlias"); + registry.meter(meter, meterAlias); + + MetricName histogram = factory.createMetricName("TestHistogram"); + MetricName histogramAlias = aliasFactory.createMetricName("TestHistogramAlias"); + registry.histogram(histogram, histogramAlias, false); + + MetricName counter = factory.createMetricName("TestCounter"); + MetricName counterAlias = aliasFactory.createMetricName("TestCounterAlias"); + registry.counter(counter, counterAlias); + + MetricName timer = factory.createMetricName("TestTimer"); + MetricName timerAlias = aliasFactory.createMetricName("TestTimerAlias"); + registry.timer(timer, timerAlias); + + Set aliases = new HashSet<>(Arrays.asList(meter.getMetricName(), meterAlias.getMetricName(), + histogram.getMetricName(), histogramAlias.getMetricName(), + counter.getMetricName(), counterAlias.getMetricName(), + timer.getMetricName(), timerAlias.getMetricName())); + + assertTrue(registry.getMetrics().keySet().containsAll(aliases)); + assertTrue(registry.getNames().containsAll(aliases)); + } + private boolean inRange(long anchor, long input, double range) { return input / ((double) anchor) < range; diff --git a/test/unit/org/apache/cassandra/metrics/ClientRequestMetricsTest.java b/test/unit/org/apache/cassandra/metrics/ClientRequestMetricsTest.java index b4e9b085d2f5..289872be1afa 100644 --- a/test/unit/org/apache/cassandra/metrics/ClientRequestMetricsTest.java +++ b/test/unit/org/apache/cassandra/metrics/ClientRequestMetricsTest.java @@ -18,12 +18,6 @@ package org.apache.cassandra.metrics; -import java.io.IOException; - -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - import com.codahale.metrics.Histogram; import com.datastax.driver.core.BatchStatement; import com.datastax.driver.core.BoundStatement; @@ -35,11 +29,17 @@ import org.apache.cassandra.exceptions.ConfigurationException; import org.apache.cassandra.service.EmbeddedCassandraService; import org.apache.cassandra.service.reads.range.RangeCommandIterator; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; +import java.io.IOException; import static com.datastax.driver.core.Cluster.builder; +import static org.apache.cassandra.cql3.CQLTester.assertRowsContains; +import static org.apache.cassandra.cql3.CQLTester.row; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; public class ClientRequestMetricsTest { @@ -99,6 +99,8 @@ public void testWriteStatement() assertEquals(0, readMetricsContainer.compareLocalRequest()); assertEquals(0, readMetricsContainer.compareRemoteRequest()); + + assertMetricsVirtualTable(writeMetricsContainer, readMetricsContainer); } @Test @@ -114,6 +116,8 @@ public void testPaxosStatement() assertEquals(1, writeMetricsContainer.compareLocalRequest()); assertEquals(0, writeMetricsContainer.compareRemoteRequest()); + + assertMetricsVirtualTable(writeMetricsContainer, readMetricsContainer); } @Test @@ -129,6 +133,8 @@ public void testBatchStatement() assertEquals(10, writeMetricsContainer.compareLocalRequest()); assertEquals(0, writeMetricsContainer.compareRemoteRequest()); + + assertMetricsVirtualTable(writeMetricsContainer, readMetricsContainer); } @Test @@ -146,6 +152,8 @@ public void testReadStatement() assertEquals(0, writeMetricsContainer.compareLocalRequest()); assertEquals(0, writeMetricsContainer.compareRemoteRequest()); + + assertMetricsVirtualTable(writeMetricsContainer, readMetricsContainer); } @Test @@ -163,6 +171,8 @@ public void testRangeStatement() assertEquals(0, writeMetricsContainer.compareLocalRequest()); assertEquals(0, writeMetricsContainer.compareRemoteRequest()); + + assertMetricsVirtualTable(writeMetricsContainer, readMetricsContainer); } @Test @@ -178,6 +188,19 @@ public void testRangeRead() throws Throwable assertThat(RangeCommandIterator.rangeMetrics.latency.getCount()).isEqualTo(latencyCount + 1); } + private void assertMetricsVirtualTable(ClientRequestMetricsContainer writeMetricsContainer, ClientRequestMetricsContainer readMetricsContainer) + { + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.client_request_group"), + row("org.apache.cassandra.metrics.ClientRequest.LocalRequests.Read", "Read", "meter", + String.valueOf(readMetricsContainer.getMetrics().localRequests.getCount())), + row("org.apache.cassandra.metrics.ClientRequest.LocalRequests.Write", "Write", "meter", + String.valueOf(writeMetricsContainer.getMetrics().localRequests.getCount())), + row("org.apache.cassandra.metrics.ClientRequest.RemoteRequests.Read", "Read", "meter", + String.valueOf(readMetricsContainer.getMetrics().remoteRequests.getCount())), + row("org.apache.cassandra.metrics.ClientRequest.RemoteRequests.Write", "Write", "meter", + String.valueOf(writeMetricsContainer.getMetrics().remoteRequests.getCount()))); + } + private void clearHistogram(Histogram histogram) { ((ClearableHistogram) histogram).clear(); @@ -206,6 +229,11 @@ public long compareRemoteRequest() { return metrics.remoteRequests.getCount() - remoteRequests; } + + public ClientRequestMetrics getMetrics() + { + return metrics; + } } private void executeWrite(int id, int ord, String val) diff --git a/test/unit/org/apache/cassandra/metrics/ClientRequestRowAndColumnMetricsTest.java b/test/unit/org/apache/cassandra/metrics/ClientRequestRowAndColumnMetricsTest.java index 80a3d3e0ceaa..51a3166193b7 100644 --- a/test/unit/org/apache/cassandra/metrics/ClientRequestRowAndColumnMetricsTest.java +++ b/test/unit/org/apache/cassandra/metrics/ClientRequestRowAndColumnMetricsTest.java @@ -48,6 +48,7 @@ public class ClientRequestRowAndColumnMetricsTest extends CQLTester public static void setup() { requireNetwork(); + addMetricsKeyspace(); } @Before @@ -73,6 +74,9 @@ public void shouldRecordReadMetricsForMultiRowPartitionSelection() assertEquals(2, ClientRequestSizeMetrics.totalRowsRead.getCount()); // The partition key is provided by the client in the request, so we don't consider those columns as read. assertEquals(4, ClientRequestSizeMetrics.totalColumnsRead.getCount()); + assertRowsContains(executeNet("SELECT * FROM system_metrics.client_request_size_group"), + row("org.apache.cassandra.metrics.ClientRequestSize.RowsRead", "unknown", "counter", "2"), + row("org.apache.cassandra.metrics.ClientRequestSize.ColumnsRead", "unknown", "counter", "4")); } @Test diff --git a/test/unit/org/apache/cassandra/metrics/JmxVirtualTableMetricsTest.java b/test/unit/org/apache/cassandra/metrics/JmxVirtualTableMetricsTest.java new file mode 100644 index 000000000000..018ece8d49bb --- /dev/null +++ b/test/unit/org/apache/cassandra/metrics/JmxVirtualTableMetricsTest.java @@ -0,0 +1,324 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.metrics; + +import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricRegistry; +import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.db.virtual.CollectionVirtualTableAdapter; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import javax.management.JMX; +import javax.management.ObjectName; +import java.util.EnumMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import static java.util.Objects.requireNonNull; +import static org.apache.cassandra.metrics.CassandraMetricsRegistry.METRICS_GROUP_POSTFIX; +import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_METRICS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Test compares JMX metrics to virtual table metrics values, basically all metric values must be equal and + * have the same representation in both places. + */ +public class JmxVirtualTableMetricsTest extends CQLTester +{ + private final Map metricToNameMap = new EnumMap<>(MetricType.class); + private final AtomicInteger gaugeValue = new AtomicInteger(123); + + @BeforeClass + public static void setup() throws Exception + { + startJMXServer(); + createMBeanServerConnection(); + addMetricsKeyspace(); + } + + @Before + public void beforeTest() + { + metricToNameMap.clear(); + MetricRegistry registry = new MetricRegistry(); + + metricToNameMap.put(MetricType.METER, registry.meter("meter")); + metricToNameMap.put(MetricType.COUNTER, registry.counter("counter")); + metricToNameMap.put(MetricType.HISTOGRAM, registry.histogram("histogram")); + metricToNameMap.put(MetricType.TIMER, registry.timer("timer")); + metricToNameMap.put(MetricType.GAUGE, registry.gauge("gauge", () -> gaugeValue::get)); + + CassandraMetricsRegistry.metricGroups.forEach(group -> { + MetricNameFactory factory = new DefaultNameFactory(group, "jmx.virtual"); + CassandraMetricsRegistry.Metrics.register(factory.createMetricName(MetricType.METER.metricName), metricToNameMap.get(MetricType.METER)); + CassandraMetricsRegistry.Metrics.register(factory.createMetricName(MetricType.COUNTER.metricName), metricToNameMap.get(MetricType.COUNTER)); + CassandraMetricsRegistry.Metrics.register(factory.createMetricName(MetricType.HISTOGRAM.metricName), metricToNameMap.get(MetricType.HISTOGRAM)); + CassandraMetricsRegistry.Metrics.register(factory.createMetricName(MetricType.TIMER.metricName), metricToNameMap.get(MetricType.TIMER)); + CassandraMetricsRegistry.Metrics.register(factory.createMetricName(MetricType.GAUGE.metricName), metricToNameMap.get(MetricType.GAUGE)); + }); + } + + @Test + public void testJmxEqualVirtualTableByMetricGroup() throws Exception + { + Map> mbeanByMetricGroup = jmxConnection.queryNames(null, null) + .stream() + .filter(this::isLocalMetric) + .collect(Collectors.groupingBy(on -> requireNonNull(on.getKeyPropertyList().get("type")))); + + for (Map.Entry> e : mbeanByMetricGroup.entrySet()) + { + assertRowsContains(executeNet(String.format("SELECT * FROM %s.%s", VIRTUAL_METRICS, + METRICS_GROUP_POSTFIX.apply(CollectionVirtualTableAdapter.virtualTableNameStyle(e.getKey())))), + e.getValue().stream().map(this::makeMetricRow).collect(Collectors.toList())); + } + } + + @Test + public void testJmxEqualVirtualTableByMetricType() throws Exception + { + Map> mbeanByMetricGroup = jmxConnection.queryNames(null, null) + .stream() + .filter(this::isLocalMetric) + .collect(Collectors.groupingBy(on -> MetricType.find(on.getKeyPropertyList().get("name")) + .orElseThrow())); + + for (Map.Entry> e : mbeanByMetricGroup.entrySet()) + { + switch (e.getKey()) + { + case METER: + assertRowsContains(executeNet(String.format("SELECT * FROM %s.type_meter", VIRTUAL_METRICS)), + e.getValue().stream().map(this::makeMeterRow).collect(Collectors.toList())); + break; + case COUNTER: + assertRowsContains(executeNet(String.format("SELECT * FROM %s.type_counter", VIRTUAL_METRICS)), + e.getValue().stream().map(this::makeCounterRow).collect(Collectors.toList())); + break; + case HISTOGRAM: + assertRowsContains(executeNet(String.format("SELECT * FROM %s.type_histogram", VIRTUAL_METRICS)), + e.getValue().stream().map(this::makeHistogramRow).collect(Collectors.toList())); + break; + case TIMER: + assertRowsContains(executeNet(String.format("SELECT * FROM %s.type_timer", VIRTUAL_METRICS)), + e.getValue().stream().map(this::makeTimerRow).collect(Collectors.toList())); + break; + case GAUGE: + assertRowsContains(executeNet(String.format("SELECT * FROM %s.type_gauge", VIRTUAL_METRICS)), + e.getValue().stream().map(this::makeGaugeRow).collect(Collectors.toList())); + break; + } + } + } + + @Test + public void testAliasesWithJmxVirtualTables() throws Exception + { + CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics; + MetricNameFactory factory = new DefaultNameFactory("Table", "AliasTestScopeFirst"); + MetricNameFactory aliasFactory = new DefaultNameFactory("Index", "AliasTestScopeSecond"); + + CassandraMetricsRegistry.MetricName meter = factory.createMetricName("TestMeter"); + CassandraMetricsRegistry.MetricName meterAlias = aliasFactory.createMetricName("TestMeterAlias"); + registry.meter(meter, meterAlias); + + assertRowsContains(executeNet(String.format("SELECT * FROM %s.type_meter", VIRTUAL_METRICS)), + row(meter.getMetricName(), 0L, 0.0, 0.0, 0.0, 0.0, "AliasTestScopeFirst"), + row(meterAlias.getMetricName(), 0L, 0.0, 0.0, 0.0, 0.0, "AliasTestScopeSecond")); + + CassandraMetricsRegistry.JmxMeterMBean bean = JMX.newMBeanProxy(jmxConnection, + meter.getMBeanName(), + CassandraMetricsRegistry.JmxMeterMBean.class); + + CassandraMetricsRegistry.JmxMeterMBean beanAlias = JMX.newMBeanProxy(jmxConnection, + meterAlias.getMBeanName(), + CassandraMetricsRegistry.JmxMeterMBean.class); + + assertNotNull(bean); + assertNotNull(beanAlias); + assertEquals(bean.getCount(), beanAlias.getCount()); + assertEquals(bean.getFifteenMinuteRate(), beanAlias.getFifteenMinuteRate(), 0.0); + assertEquals(bean.getFiveMinuteRate(), beanAlias.getFiveMinuteRate(), 0.0); + assertEquals(bean.getMeanRate(), beanAlias.getMeanRate(), 0.0); + assertEquals(bean.getOneMinuteRate(), beanAlias.getOneMinuteRate(), 0.0); + } + + private Object[] makeMetricRow(ObjectName objectName) + { + MetricType type = MetricType.find(objectName.getKeyPropertyList().get("name")).orElseThrow(); + String jmxValue; + switch (type) + { + case METER: + jmxValue = String.valueOf(JMX.newMBeanProxy(jmxConnection, + objectName, + CassandraMetricsRegistry.JmxMeterMBean.class).getCount()); + break; + case COUNTER: + jmxValue = String.valueOf(JMX.newMBeanProxy(jmxConnection, + objectName, + CassandraMetricsRegistry.JmxCounterMBean.class).getCount()); + break; + case HISTOGRAM: + jmxValue = String.valueOf(JMX.newMBeanProxy(jmxConnection, + objectName, + CassandraMetricsRegistry.JmxHistogramMBean.class).get50thPercentile()); + break; + case TIMER: + jmxValue = String.valueOf(JMX.newMBeanProxy(jmxConnection, + objectName, + CassandraMetricsRegistry.JmxTimerMBean.class).getCount()); + break; + case GAUGE: + jmxValue = String.valueOf(JMX.newMBeanProxy(jmxConnection, + objectName, + CassandraMetricsRegistry.JmxGaugeMBean.class).getValue()); + break; + default: + throw new RuntimeException("Unknown metric type: " + objectName.getKeyPropertyList().get("name")); + } + return CQLTester.row(getFullMetricName(objectName), + requireNonNull(objectName.getKeyPropertyList().get("scope")), + MetricType.find(objectName.getKeyPropertyList().get("name")) + .map(MetricType::name) + .map(String::toLowerCase) + .orElseThrow(), + jmxValue); + } + + private Object[] makeMeterRow(ObjectName objectName) + { + assertEquals(MetricType.METER.metricName, objectName.getKeyPropertyList().get("name")); + CassandraMetricsRegistry.JmxMeterMBean bean = JMX.newMBeanProxy(jmxConnection, + objectName, + CassandraMetricsRegistry.JmxMeterMBean.class); + + return CQLTester.row(getFullMetricName(objectName), + bean.getCount(), + bean.getFifteenMinuteRate(), + bean.getFiveMinuteRate(), + bean.getMeanRate(), + bean.getOneMinuteRate(), + requireNonNull(objectName.getKeyPropertyList().get("scope"))); + } + + private Object[] makeCounterRow(ObjectName objectName) + { + assertEquals(MetricType.COUNTER.metricName, objectName.getKeyPropertyList().get("name")); + CassandraMetricsRegistry.JmxCounterMBean bean = JMX.newMBeanProxy(jmxConnection, + objectName, + CassandraMetricsRegistry.JmxCounterMBean.class); + + return CQLTester.row(getFullMetricName(objectName), + requireNonNull(objectName.getKeyPropertyList().get("scope")), + bean.getCount()); + } + + private Object[] makeHistogramRow(ObjectName objectName) + { + assertEquals(MetricType.HISTOGRAM.metricName, objectName.getKeyPropertyList().get("name")); + CassandraMetricsRegistry.JmxHistogramMBean bean = JMX.newMBeanProxy(jmxConnection, + objectName, + CassandraMetricsRegistry.JmxHistogramMBean.class); + + return CQLTester.row(getFullMetricName(objectName), + bean.getMax(), + bean.getMean(), + bean.getMin(), + bean.get75thPercentile(), + bean.get95thPercentile(), + bean.get98thPercentile(), + bean.get999thPercentile(), + bean.get99thPercentile(), + requireNonNull(objectName.getKeyPropertyList().get("scope"))); + } + + private Object[] makeTimerRow(ObjectName objectName) + { + assertEquals(MetricType.TIMER.metricName, objectName.getKeyPropertyList().get("name")); + CassandraMetricsRegistry.JmxTimerMBean bean = JMX.newMBeanProxy(jmxConnection, + objectName, + CassandraMetricsRegistry.JmxTimerMBean.class); + + return CQLTester.row(getFullMetricName(objectName), + bean.getCount(), + bean.getFifteenMinuteRate(), + bean.getFiveMinuteRate(), + bean.getMeanRate(), + bean.getOneMinuteRate(), + requireNonNull(objectName.getKeyPropertyList().get("scope"))); + } + + private Object[] makeGaugeRow(ObjectName objectName) + { + assertEquals(MetricType.GAUGE.metricName, objectName.getKeyPropertyList().get("name")); + String jmxValue = String.valueOf(JMX.newMBeanProxy(jmxConnection, + objectName, + CassandraMetricsRegistry.JmxGaugeMBean.class).getValue()); + + return CQLTester.row(getFullMetricName(objectName), + requireNonNull(objectName.getKeyPropertyList().get("scope")), + jmxValue); + } + + private boolean isLocalMetric(ObjectName mBean) + { + MetricType type = MetricType.find(mBean.getKeyPropertyList().get("name")).orElse(null); + return mBean.toString().startsWith(DefaultNameFactory.GROUP_NAME) && metricToNameMap.containsKey(type); + } + + private static String getFullMetricName(ObjectName objectName) + { + Map props = objectName.getKeyPropertyList(); + return new CassandraMetricsRegistry.MetricName(DefaultNameFactory.GROUP_NAME, requireNonNull(props.get("type")), + requireNonNull(props.get("name")), requireNonNull(props.get("scope"))) + .getMetricName(); + } + + private enum MetricType + { + METER("MeterTestMetric"), + COUNTER("CounterTestMetric"), + HISTOGRAM("HistogramTestMetric"), + TIMER("TimerTestMetric"), + GAUGE("GaugeTestMetric"); + + private final String metricName; + + MetricType(String metricName) + { + this.metricName = metricName; + } + + public static Optional find(String metricName) + { + for (MetricType type : values()) + if (type.metricName.equals(metricName)) + return Optional.of(type); + + return Optional.empty(); + } + } +} diff --git a/test/unit/org/apache/cassandra/metrics/KeyspaceMetricsTest.java b/test/unit/org/apache/cassandra/metrics/KeyspaceMetricsTest.java index 7c00da581944..d1adc92bf973 100644 --- a/test/unit/org/apache/cassandra/metrics/KeyspaceMetricsTest.java +++ b/test/unit/org/apache/cassandra/metrics/KeyspaceMetricsTest.java @@ -23,6 +23,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import com.codahale.metrics.Metric; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Row; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -35,6 +38,7 @@ import org.apache.cassandra.service.EmbeddedCassandraService; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class KeyspaceMetricsTest @@ -71,6 +75,33 @@ public void testMetricsCleanupOnDrop() assertEquals(metrics.get().collect(Collectors.joining(",")), 0, metrics.get().count()); } + @Test + public void testKeyspaceVirtualTable() + { + String keyspace = "uniquemetricskeyspace1"; + session.execute(String.format("CREATE KEYSPACE %s WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 };", keyspace)); + + assertTrue(CassandraMetricsRegistry.Metrics.getNames().stream().anyMatch(m -> m.endsWith(keyspace))); + ResultSet resultSet = session.execute("SELECT * FROM system_metrics.keyspace_group WHERE scope = '" + keyspace + "';"); + + int count = 0; + for (Row row : resultSet) + { + String metricName = row.getString("name"); + if (!metricName.endsWith(keyspace)) + continue; + + Metric metric = CassandraMetricsRegistry.Metrics.getMetrics().get(metricName); + assertEquals(CassandraMetricsRegistry.getValueAsString(metric), row.getString("value")); + count++; + } + + assertTrue("Keyspace " + keyspace + " metrics was not found", count > 0); + + session.execute(String.format("DROP KEYSPACE %s;", keyspace)); + assertFalse(CassandraMetricsRegistry.Metrics.getNames().stream().anyMatch(m -> m.endsWith(keyspace))); + } + @AfterClass public static void tearDown() { diff --git a/test/unit/org/apache/cassandra/metrics/LatencyMetricsTest.java b/test/unit/org/apache/cassandra/metrics/LatencyMetricsTest.java index 3a4dc8af9838..20fc79c6495d 100644 --- a/test/unit/org/apache/cassandra/metrics/LatencyMetricsTest.java +++ b/test/unit/org/apache/cassandra/metrics/LatencyMetricsTest.java @@ -29,13 +29,13 @@ public class LatencyMetricsTest { private final MetricNameFactory factory = new TestMetricsNameFactory(); - private class TestMetricsNameFactory implements MetricNameFactory + private static class TestMetricsNameFactory implements MetricNameFactory { @Override public CassandraMetricsRegistry.MetricName createMetricName(String metricName) { - return new CassandraMetricsRegistry.MetricName(TestMetricsNameFactory.class, metricName); + return new CassandraMetricsRegistry.MetricName(DefaultNameFactory.GROUP_NAME, ClientRequestMetrics.TYPE_NAME, metricName); } } @@ -45,7 +45,7 @@ public CassandraMetricsRegistry.MetricName createMetricName(String metricName) @Test public void testGetRecentLatency() { - final LatencyMetrics l = new LatencyMetrics("test", "test"); + final LatencyMetrics l = new LatencyMetrics(ClientRequestMetrics.TYPE_NAME, "test"); Runnable r = () -> { for (int i = 0; i < 10000; i++) { @@ -67,7 +67,7 @@ public void testGetRecentLatency() @Test public void testReadMerging() { - final LatencyMetrics parent = new LatencyMetrics("testMerge", "testMerge"); + final LatencyMetrics parent = new LatencyMetrics(ClientRequestMetrics.TYPE_NAME, "testMerge"); final LatencyMetrics child = new LatencyMetrics(factory, "testChild", parent); for (int i = 0; i < 100; i++) @@ -86,7 +86,7 @@ public void testReadMerging() @Test public void testRelease() { - final LatencyMetrics parent = new LatencyMetrics("testRelease", "testRelease"); + final LatencyMetrics parent = new LatencyMetrics(ClientRequestMetrics.TYPE_NAME, "testRelease"); final LatencyMetrics child = new LatencyMetrics(factory, "testChildRelease", parent); for (int i = 0; i < 100; i++) diff --git a/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java b/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java index bbcced43dbff..a0691d8ccdbc 100644 --- a/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java +++ b/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java @@ -40,6 +40,8 @@ import org.apache.cassandra.service.EmbeddedCassandraService; import org.apache.cassandra.service.StorageService; +import static org.apache.cassandra.cql3.CQLTester.assertRowsContains; +import static org.apache.cassandra.cql3.CQLTester.row; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -140,6 +142,12 @@ public void testRegularStatementsExecuted() assertEquals(10, cfs.metric.coordinatorWriteLatency.getCount()); assertGreaterThan(cfs.metric.coordinatorWriteLatency.getMeanRate(), 0); + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.table_group"), + row("org.apache.cassandra.metrics.Table.CoordinatorWriteLatency.junit.tablemetricstest", "junit.tablemetricstest", "timer", + String.valueOf(cfs.metric.coordinatorWriteLatency.getCount()))); + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.column_family_group"), + row("org.apache.cassandra.metrics.ColumnFamily.CoordinatorWriteLatency.junit.tablemetricstest", "junit.tablemetricstest", "timer", + String.valueOf(cfs.metric.coordinatorWriteLatency.getCount()))); } @Test @@ -156,6 +164,13 @@ public void testMaxSSTableSize() throws Exception StorageService.instance.forceKeyspaceFlush(KEYSPACE); assertGreaterThan(cfs.metric.maxSSTableSize.getValue().doubleValue(), 0); + + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.table_group"), + row("org.apache.cassandra.metrics.Table.MaxSSTableSize.junit.tablemetricstest", "junit.tablemetricstest", "gauge", + String.valueOf(cfs.metric.maxSSTableSize.getValue()))); + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.column_family_group"), + row("org.apache.cassandra.metrics.ColumnFamily.MaxSSTableSize.junit.tablemetricstest", "junit.tablemetricstest", "gauge", + String.valueOf(cfs.metric.maxSSTableSize.getValue()))); } @Test @@ -171,6 +186,12 @@ public void testMaxSSTableDuration() throws Exception StorageService.instance.forceKeyspaceFlush(KEYSPACE); assertGreaterThan(cfs.metric.maxSSTableDuration.getValue().doubleValue(), 0); + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.table_group"), + row("org.apache.cassandra.metrics.Table.MaxSSTableDuration.junit.tablemetricstesttwcs", "junit.tablemetricstesttwcs", "gauge", + String.valueOf(cfs.metric.maxSSTableDuration.getValue()))); + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.column_family_group"), + row("org.apache.cassandra.metrics.ColumnFamily.MaxSSTableDuration.junit.tablemetricstesttwcs", "junit.tablemetricstesttwcs", "gauge", + String.valueOf(cfs.metric.maxSSTableDuration.getValue()))); } @Test diff --git a/test/unit/org/apache/cassandra/metrics/TrieMemtableMetricsTest.java b/test/unit/org/apache/cassandra/metrics/TrieMemtableMetricsTest.java index 24fdc8b47524..511a02ead719 100644 --- a/test/unit/org/apache/cassandra/metrics/TrieMemtableMetricsTest.java +++ b/test/unit/org/apache/cassandra/metrics/TrieMemtableMetricsTest.java @@ -18,21 +18,6 @@ package org.apache.cassandra.metrics; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import com.datastax.driver.core.Cluster; import com.datastax.driver.core.Session; import org.apache.cassandra.SchemaLoader; @@ -45,9 +30,25 @@ import org.jboss.byteman.contrib.bmunit.BMRule; import org.jboss.byteman.contrib.bmunit.BMRules; import org.jboss.byteman.contrib.bmunit.BMUnitRunner; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.cassandra.config.CassandraRelevantProperties.MEMTABLE_SHARD_COUNT; -import static org.hamcrest.Matchers.*; +import static org.apache.cassandra.cql3.CQLTester.assertRowsContains; +import static org.apache.cassandra.cql3.CQLTester.row; +import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -59,6 +60,8 @@ public class TrieMemtableMetricsTest extends SchemaLoader private static final Logger logger = LoggerFactory.getLogger(TrieMemtableMetricsTest.class); private static Session session; + private static Cluster cluster; + private static EmbeddedCassandraService cassandra; private static final String KEYSPACE = "triememtable"; private static final String TABLE = "metricstest"; @@ -78,10 +81,10 @@ public static void setup() throws ConfigurationException, IOException }); MEMTABLE_SHARD_COUNT.setInt(NUM_SHARDS); - EmbeddedCassandraService cassandra = new EmbeddedCassandraService(); + cassandra = new EmbeddedCassandraService(); cassandra.start(); - Cluster cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build(); + cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build(); session = cluster.connect(); session.execute(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 };", KEYSPACE)); @@ -114,6 +117,11 @@ public void testRegularStatementsAreCounted() long allPuts = metrics.contendedPuts.getCount() + metrics.uncontendedPuts.getCount(); assertEquals(10, allPuts); + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.trie_memtable_group"), + row("org.apache.cassandra.metrics.TrieMemtable.Contended memtable puts.triememtable.metricstest", + "triememtable.metricstest", "counter", String.valueOf(metrics.contendedPuts.getCount())), + row("org.apache.cassandra.metrics.TrieMemtable.Uncontended memtable puts.triememtable.metricstest", + "triememtable.metricstest", "counter", String.valueOf(metrics.uncontendedPuts.getCount()))); } @Test @@ -155,6 +163,9 @@ public void testContentionMetrics() throws IOException, ExecutionException, Inte assertEquals(100, metrics.contendedPuts.getCount() + metrics.uncontendedPuts.getCount()); assertThat(metrics.contendedPuts.getCount(), greaterThan(0L)); assertThat(metrics.contentionTime.totalLatency.getCount(), greaterThan(0L)); + assertRowsContains(cluster, session.execute("SELECT * FROM system_metrics.trie_memtable_group"), + row("org.apache.cassandra.metrics.TrieMemtable.Contention timeTotalLatency.triememtable.metricstest", + "triememtable.metricstest", "counter", String.valueOf(metrics.contentionTime.totalLatency.getCount()))); } @Test @@ -203,6 +214,9 @@ private void writeAndFlush(int rows) throws IOException, ExecutionException, Int @AfterClass public static void teardown() { - session.close(); + if (cluster != null) + cluster.close(); + if (cassandra != null) + cassandra.stop(); } } diff --git a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java index 17b59400e702..65dc279ec547 100644 --- a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java +++ b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java @@ -251,7 +251,8 @@ public void testCamelToSnake() AssertionError error = null; for (Pair a : Arrays.asList(Pair.create("Testing", "testing"), Pair.create("fooBarBaz", "foo_bar_baz"), - Pair.create("foo_bar_baz", "foo_bar_baz") + Pair.create("foo_bar_baz", "foo_bar_baz"), + Pair.create("TCM", "tcm") )) { try