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

[cassandra] branch trunk updated: Make Table/Keyspace Metric Names Consistent With Each Other

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

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


The following commit(s) were added to refs/heads/trunk by this push:
     new b0b8ab2  Make Table/Keyspace Metric Names Consistent With Each Other
b0b8ab2 is described below

commit b0b8ab2ea2f185d1f433b071d196f40dc2a2968e
Author: Caleb Rackliffe <ca...@gmail.com>
AuthorDate: Wed Sep 9 10:53:28 2020 -0700

    Make Table/Keyspace Metric Names Consistent With Each Other
    
    patch by Caleb Rackliffe, Stephen Mallette; reviewed by David Capwell, Dinesh Joshi for CASSANDRA-15909
---
 CHANGES.txt                                        |   1 +
 .../metrics/CassandraMetricsRegistry.java          |  17 +-
 .../apache/cassandra/metrics/ClientMetrics.java    |  20 +-
 .../apache/cassandra/metrics/KeyspaceMetrics.java  |  12 +-
 .../org/apache/cassandra/metrics/TableMetrics.java | 295 ++++++++++-----------
 src/java/org/apache/cassandra/repair/SyncTask.java |   2 +-
 .../service/reads/ReplicaFilteringProtection.java  |   2 +-
 .../cassandra/metrics/KeyspaceMetricsTest.java     |  10 +-
 .../apache/cassandra/metrics/TableMetricsTest.java |  10 +-
 9 files changed, 202 insertions(+), 167 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 4efccd7..afa03ac 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,7 @@
  * Fix yaml format and alignment in tpstats (CASSANDRA-11402)
  * Avoid trying to keep track of RTs for endpoints we won't write to during read repair (CASSANDRA-16084)
  * When compaction gets interrupted, the exception should include the compactionId (CASSANDRA-15954)
+ * Make Table/Keyspace Metric Names Consistent With Each Other (CASSANDRA-15909)
 
 4.0-beta2
  * Add addition incremental repair visibility to nodetool repair_admin (CASSANDRA-14939)
diff --git a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java
index 74c3367..0b5bac5 100644
--- a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java
+++ b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java
@@ -154,6 +154,16 @@ public class CassandraMetricsRegistry extends MetricRegistry
         return ret;
     }
 
+    public <T extends Metric> T register(MetricName name, T metric, MetricName... aliases)
+    {
+        T ret = register(name, metric);
+        for (MetricName aliasName : aliases)
+        {
+            registerAlias(name, aliasName);
+        }
+        return ret;
+    }
+
     public boolean remove(MetricName name)
     {
         boolean removed = remove(name.getMetricName());
@@ -162,11 +172,14 @@ public class CassandraMetricsRegistry extends MetricRegistry
         return removed;
     }
 
-    public boolean remove(MetricName name, MetricName alias)
+    public boolean remove(MetricName name, MetricName... aliases)
     {
         if (remove(name))
         {
-            removeAlias(alias);
+            for (MetricName alias : aliases)
+            {
+                removeAlias(alias);
+            }
             return true;
         }
         return false;
diff --git a/src/java/org/apache/cassandra/metrics/ClientMetrics.java b/src/java/org/apache/cassandra/metrics/ClientMetrics.java
index 0a4ae1a..6464a17 100644
--- a/src/java/org/apache/cassandra/metrics/ClientMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ClientMetrics.java
@@ -42,7 +42,10 @@ public final class ClientMetrics
     private Meter authFailure;
 
     private AtomicInteger pausedConnections;
+    
+    @SuppressWarnings({ "unused", "FieldCanBeLocal" })
     private Gauge<Integer> pausedConnectionsGauge;
+    
     private Meter requestDiscarded;
 
     private ClientMetrics()
@@ -81,10 +84,11 @@ public final class ClientMetrics
 
         this.servers = servers;
 
-        registerGauge("connectedNativeClients",       this::countConnectedClients);
-        registerGauge("connectedNativeClientsByUser", this::countConnectedClientsByUser);
-        registerGauge("connections",                  this::connectedClients);
-        registerGauge("clientsByProtocolVersion",     this::recentClientStats);
+        // deprecated the lower-cased initial letter metric names in 4.0
+        registerGauge("ConnectedNativeClients", "connectedNativeClients", this::countConnectedClients);
+        registerGauge("ConnectedNativeClientsByUser", "connectedNativeClientsByUser", this::countConnectedClientsByUser);
+        registerGauge("Connections", "connections", this::connectedClients);
+        registerGauge("ClientsByProtocolVersion", "clientsByProtocolVersion", this::recentClientStats);
 
         authSuccess = registerMeter("AuthSuccess");
         authFailure = registerMeter("AuthFailure");
@@ -136,7 +140,7 @@ public final class ClientMetrics
 
         for (Server server : servers)
             for (ClientStat stat : server.recentClientStats())
-                stats.add(new HashMap(stat.asMap())); // asMap returns guava, so need to convert to java for jmx
+                stats.add(new HashMap<>(stat.asMap())); // asMap returns guava, so need to convert to java for jmx
 
         stats.sort(Comparator.comparing(map -> map.get(ClientStat.PROTOCOL_VERSION)));
 
@@ -147,6 +151,12 @@ public final class ClientMetrics
     {
         return Metrics.register(factory.createMetricName(name), gauge);
     }
+    
+    private void registerGauge(String name, String deprecated, Gauge<?> gauge)
+    {
+        Gauge<?> registeredGauge = registerGauge(name, gauge);
+        Metrics.registerMBean(registeredGauge, factory.createMetricName(deprecated).getMBeanName());
+    }
 
     private Meter registerMeter(String name)
     {
diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
index 4af26c0..dadbe47 100644
--- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
@@ -173,15 +173,15 @@ public class KeyspaceMetrics
         memtableLiveDataSize = createKeyspaceGauge("MemtableLiveDataSize",
                 metric -> metric.memtableLiveDataSize.getValue());
         memtableOnHeapDataSize = createKeyspaceGauge("MemtableOnHeapDataSize",
-                metric -> metric.memtableOnHeapSize.getValue());
+                metric -> metric.memtableOnHeapDataSize.getValue());
         memtableOffHeapDataSize = createKeyspaceGauge("MemtableOffHeapDataSize",
-                metric -> metric.memtableOffHeapSize.getValue());
+                metric -> metric.memtableOffHeapDataSize.getValue());
         allMemtablesLiveDataSize = createKeyspaceGauge("AllMemtablesLiveDataSize",
                 metric -> metric.allMemtablesLiveDataSize.getValue());
         allMemtablesOnHeapDataSize = createKeyspaceGauge("AllMemtablesOnHeapDataSize",
-                metric -> metric.allMemtablesOnHeapSize.getValue());
+                metric -> metric.allMemtablesOnHeapDataSize.getValue());
         allMemtablesOffHeapDataSize = createKeyspaceGauge("AllMemtablesOffHeapDataSize",
-                metric -> metric.allMemtablesOffHeapSize.getValue());
+                metric -> metric.allMemtablesOffHeapDataSize.getValue());
         memtableSwitchCount = createKeyspaceGauge("MemtableSwitchCount",
                 metric -> metric.memtableSwitchCount.getCount());
         pendingCompactions = createKeyspaceGauge("PendingCompactions", metric -> metric.pendingCompactions.getValue());
@@ -233,8 +233,8 @@ public class KeyspaceMetrics
         confirmedRepairedInconsistencies = createKeyspaceMeter("RepairedDataInconsistenciesConfirmed");
         unconfirmedRepairedInconsistencies = createKeyspaceMeter("RepairedDataInconsistenciesUnconfirmed");
 
-        repairedDataTrackingOverreadRows = createKeyspaceHistogram("RepairedOverreadRows", false);
-        repairedDataTrackingOverreadTime = createKeyspaceTimer("RepairedOverreadTime");
+        repairedDataTrackingOverreadRows = createKeyspaceHistogram("RepairedDataTrackingOverreadRows", false);
+        repairedDataTrackingOverreadTime = createKeyspaceTimer("RepairedDataTrackingOverreadTime");
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java
index 24550b0..0bc66b9 100644
--- a/src/java/org/apache/cassandra/metrics/TableMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java
@@ -62,19 +62,18 @@ import com.codahale.metrics.RatioGauge;
  */
 public class TableMetrics
 {
-
     public static final long[] EMPTY = new long[0];
 
     /** Total amount of data stored in the memtable that resides on-heap, including column related overhead and partitions overwritten. */
-    public final Gauge<Long> memtableOnHeapSize;
+    public final Gauge<Long> memtableOnHeapDataSize;
     /** Total amount of data stored in the memtable that resides off-heap, including column related overhead and partitions overwritten. */
-    public final Gauge<Long> memtableOffHeapSize;
+    public final Gauge<Long> memtableOffHeapDataSize;
     /** Total amount of live data stored in the memtable, excluding any data structure overhead */
     public final Gauge<Long> memtableLiveDataSize;
     /** Total amount of data stored in the memtables (2i and pending flush memtables included) that resides on-heap. */
-    public final Gauge<Long> allMemtablesOnHeapSize;
+    public final Gauge<Long> allMemtablesOnHeapDataSize;
     /** Total amount of data stored in the memtables (2i and pending flush memtables included) that resides off-heap. */
-    public final Gauge<Long> allMemtablesOffHeapSize;
+    public final Gauge<Long> allMemtablesOffHeapDataSize;
     /** Total amount of live data stored in the memtables (2i and pending flush memtables included) that resides off-heap, excluding any data structure overhead */
     public final Gauge<Long> allMemtablesLiveDataSize;
     /** Total number of columns present in the memtable. */
@@ -184,7 +183,7 @@ public class TableMetrics
     /** time spent creating merkle trees */
     public final TableTimer validationTime;
     /** time spent syncing data in a repair */
-    public final TableTimer syncTime;
+    public final TableTimer repairSyncTime;
     /** approximate number of bytes read while creating merkle trees */
     public final TableHistogram bytesValidated;
     /** number of partitions read creating merkle trees */
@@ -300,31 +299,15 @@ public class TableMetrics
     });
 
     public static final Gauge<Long> globalBytesRepaired = Metrics.register(globalFactory.createMetricName("BytesRepaired"),
-                                                                           new Gauge<Long>()
-    {
-        public Long getValue()
-        {
-            return totalNonSystemTablesSize(SSTableReader::isRepaired).left;
-        }
-    });
+                                                                           () -> totalNonSystemTablesSize(SSTableReader::isRepaired).left);
 
-    public static final Gauge<Long> globalBytesUnrepaired = Metrics.register(globalFactory.createMetricName("BytesUnrepaired"),
-                                                                             new Gauge<Long>()
-    {
-        public Long getValue()
-        {
-            return totalNonSystemTablesSize(s -> !s.isRepaired() && !s.isPendingRepair()).left;
-        }
-    });
+    public static final Gauge<Long> globalBytesUnrepaired = 
+        Metrics.register(globalFactory.createMetricName("BytesUnrepaired"),
+                         () -> totalNonSystemTablesSize(s -> !s.isRepaired() && !s.isPendingRepair()).left);
 
-    public static final Gauge<Long> globalBytesPendingRepair = Metrics.register(globalFactory.createMetricName("BytesPendingRepair"),
-                                                                                new Gauge<Long>()
-    {
-        public Long getValue()
-        {
-            return totalNonSystemTablesSize(SSTableReader::isPendingRepair).left;
-        }
-    });
+    public static final Gauge<Long> globalBytesPendingRepair = 
+        Metrics.register(globalFactory.createMetricName("BytesPendingRepair"),
+                         () -> totalNonSystemTablesSize(SSTableReader::isPendingRepair).left);
 
     public final Meter readRepairRequests;
     public final Meter shortReadProtectionRequests;
@@ -332,10 +315,10 @@ public class TableMetrics
     public final Meter replicaFilteringProtectionRequests;
     
     /**
-     * This histogram records the maximum number of rows {@link org.apache.cassandra.service.ReplicaFilteringProtection}
+     * This histogram records the maximum number of rows {@link org.apache.cassandra.service.reads.ReplicaFilteringProtection}
      * caches at a point in time per query. With no replica divergence, this is equivalent to the maximum number of
      * cached rows in a single partition during a query. It can be helpful when choosing appropriate values for the
-     * replica_filtering_protection thresholds in cassandra.yaml. 
+     * replica_filtering_protection thresholds in cassandra.yaml.
      */
     public final Histogram rfpRowsCachedPerQuery;
 
@@ -443,35 +426,24 @@ public class TableMetrics
         samplers.put(SamplerType.CAS_CONTENTIONS, topCasPartitionContention);
         samplers.put(SamplerType.LOCAL_READ_TIME, topLocalReadQueryTime);
 
-        memtableColumnsCount = createTableGauge("MemtableColumnsCount", new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return cfs.getTracker().getView().getCurrentMemtable().getOperations();
-            }
-        });
-        memtableOnHeapSize = createTableGauge("MemtableOnHeapSize", new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return cfs.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
-            }
-        });
-        memtableOffHeapSize = createTableGauge("MemtableOffHeapSize", new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return cfs.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
-            }
-        });
-        memtableLiveDataSize = createTableGauge("MemtableLiveDataSize", new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize();
-            }
-        });
-        allMemtablesOnHeapSize = createTableGauge("AllMemtablesHeapSize", new Gauge<Long>()
+        memtableColumnsCount = createTableGauge("MemtableColumnsCount", 
+                                                () -> cfs.getTracker().getView().getCurrentMemtable().getOperations());
+
+        // MemtableOnHeapSize naming deprecated in 4.0
+        memtableOnHeapDataSize = createTableGaugeWithDeprecation("MemtableOnHeapDataSize", "MemtableOnHeapSize", 
+                                                                 () -> cfs.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns(), 
+                                                                 new GlobalTableGauge("MemtableOnHeapDataSize"));
+
+        // MemtableOffHeapSize naming deprecated in 4.0
+        memtableOffHeapDataSize = createTableGaugeWithDeprecation("MemtableOffHeapDataSize", "MemtableOffHeapSize", 
+                                                                  () -> cfs.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns(), 
+                                                                  new GlobalTableGauge("MemtableOnHeapDataSize"));
+        
+        memtableLiveDataSize = createTableGauge("MemtableLiveDataSize", 
+                                                () -> cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize());
+
+        // AllMemtablesHeapSize naming deprecated in 4.0
+        allMemtablesOnHeapDataSize = createTableGaugeWithDeprecation("AllMemtablesOnHeapDataSize", "AllMemtablesHeapSize", new Gauge<Long>()
         {
             public Long getValue()
             {
@@ -480,8 +452,10 @@ public class TableMetrics
                     size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
                 return size;
             }
-        });
-        allMemtablesOffHeapSize = createTableGauge("AllMemtablesOffHeapSize", new Gauge<Long>()
+        }, new GlobalTableGauge("AllMemtablesOnHeapDataSize"));
+
+        // AllMemtablesOffHeapSize naming deprecated in 4.0
+        allMemtablesOffHeapDataSize = createTableGaugeWithDeprecation("AllMemtablesOffHeapDataSize", "AllMemtablesOffHeapSize", new Gauge<Long>()
         {
             public Long getValue()
             {
@@ -490,7 +464,7 @@ public class TableMetrics
                     size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
                 return size;
             }
-        });
+        }, new GlobalTableGauge("AllMemtablesOffHeapDataSize"));
         allMemtablesLiveDataSize = createTableGauge("AllMemtablesLiveDataSize", new Gauge<Long>()
         {
             public Long getValue()
@@ -502,19 +476,10 @@ public class TableMetrics
             }
         });
         memtableSwitchCount = createTableCounter("MemtableSwitchCount");
-        estimatedPartitionSizeHistogram = createTableGauge("EstimatedPartitionSizeHistogram", "EstimatedRowSizeHistogram", new Gauge<long[]>()
-        {
-            public long[] getValue()
-            {
-                return combineHistograms(cfs.getSSTables(SSTableSet.CANONICAL), new GetHistogram()
-                {
-                    public EstimatedHistogram getHistogram(SSTableReader reader)
-                    {
-                        return reader.getEstimatedPartitionSize();
-                    }
-                });
-            }
-        }, null);
+        estimatedPartitionSizeHistogram = createTableGauge("EstimatedPartitionSizeHistogram", "EstimatedRowSizeHistogram",
+                                                           () -> combineHistograms(cfs.getSSTables(SSTableSet.CANONICAL),
+                                                                                   SSTableReader::getEstimatedPartitionSize), null);
+        
         estimatedPartitionCount = createTableGauge("EstimatedPartitionCount", "EstimatedRowCount", new Gauge<Long>()
         {
             public Long getValue()
@@ -528,19 +493,10 @@ public class TableMetrics
                 }
             }
         }, null);
-        estimatedColumnCountHistogram = createTableGauge("EstimatedColumnCountHistogram", "EstimatedColumnCountHistogram", new Gauge<long[]>()
-        {
-            public long[] getValue()
-            {
-                return combineHistograms(cfs.getSSTables(SSTableSet.CANONICAL), new GetHistogram()
-                {
-                    public EstimatedHistogram getHistogram(SSTableReader reader)
-                    {
-                        return reader.getEstimatedCellPerPartitionCount();
-                    }
-                });
-            }
-        }, null);
+        estimatedColumnCountHistogram = createTableGauge("EstimatedColumnCountHistogram", "EstimatedColumnCountHistogram",
+                                                         () -> combineHistograms(cfs.getSSTables(SSTableSet.CANONICAL), 
+                                                                                 SSTableReader::getEstimatedCellPerPartitionCount), null);
+        
         sstablesPerReadHistogram = createTableHistogram("SSTablesPerReadHistogram", cfs.keyspace.metric.sstablesPerReadHistogram, true);
         compressionRatio = createTableGauge("CompressionRatio", new Gauge<Double>()
         {
@@ -621,20 +577,8 @@ public class TableMetrics
         bytesFlushed = createTableCounter("BytesFlushed");
 
         compactionBytesWritten = createTableCounter("CompactionBytesWritten");
-        pendingCompactions = createTableGauge("PendingCompactions", new Gauge<Integer>()
-        {
-            public Integer getValue()
-            {
-                return cfs.getCompactionStrategyManager().getEstimatedRemainingTasks();
-            }
-        });
-        liveSSTableCount = createTableGauge("LiveSSTableCount", new Gauge<Integer>()
-        {
-            public Integer getValue()
-            {
-                return cfs.getTracker().getView().liveSSTables().size();
-            }
-        });
+        pendingCompactions = createTableGauge("PendingCompactions", () -> cfs.getCompactionStrategyManager().getEstimatedRemainingTasks());
+        liveSSTableCount = createTableGauge("LiveSSTableCount", () -> cfs.getTracker().getView().liveSSTables().size());
         oldVersionSSTableCount = createTableGauge("OldVersionSSTableCount", new Gauge<Integer>()
         {
             public Integer getValue()
@@ -909,13 +853,7 @@ public class TableMetrics
             viewReadTime = createTableTimer("ViewReadTime", cfs.keyspace.metric.viewReadTime);
         }
 
-        trueSnapshotsSize = createTableGauge("SnapshotsSize", new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                return cfs.trueSnapshotsSize();
-            }
-        });
+        trueSnapshotsSize = createTableGauge("SnapshotsSize", cfs::trueSnapshotsSize);
         rowCacheHitOutOfRange = createTableCounter("RowCacheHitOutOfRange");
         rowCacheHit = createTableCounter("RowCacheHit");
         rowCacheMiss = createTableCounter("RowCacheMiss");
@@ -934,7 +872,7 @@ public class TableMetrics
 
         anticompactionTime = createTableTimer("AnticompactionTime", cfs.keyspace.metric.anticompactionTime);
         validationTime = createTableTimer("ValidationTime", cfs.keyspace.metric.validationTime);
-        syncTime = createTableTimer("SyncTime", cfs.keyspace.metric.repairSyncTime);
+        repairSyncTime = createTableTimer("RepairSyncTime", cfs.keyspace.metric.repairSyncTime);
 
         bytesValidated = createTableHistogram("BytesValidated", cfs.keyspace.metric.bytesValidated, false);
         partitionsValidated = createTableHistogram("PartitionsValidated", cfs.keyspace.metric.partitionsValidated, false);
@@ -987,25 +925,13 @@ public class TableMetrics
         }
     }
 
-
     /**
      * Create a gauge that will be part of a merged version of all column families.  The global gauge
      * will merge each CF gauge by adding their values
      */
     protected <T extends Number> Gauge<T> createTableGauge(final String name, Gauge<T> gauge)
     {
-        return createTableGauge(name, gauge, new Gauge<Long>()
-        {
-            public Long getValue()
-            {
-                long total = 0;
-                for (Metric cfGauge : allTableMetrics.get(name))
-                {
-                    total = total + ((Gauge<? extends Number>) cfGauge).getValue().longValue();
-                }
-                return total;
-            }
-        });
+        return createTableGauge(name, gauge, new GlobalTableGauge(name));
     }
 
     /**
@@ -1028,6 +954,35 @@ public class TableMetrics
     }
 
     /**
+     * Same as {@link #createTableGauge(String, Gauge, Gauge)} but accepts a deprecated
+     * name for a table {@code Gauge}. Prefer that method when deprecation is not necessary.
+     *
+     * @param name the name of the metric registered with the "Table" type
+     * @param deprecated the deprecated name for the metric registered with the "Table" type
+     */
+    protected <G,T> Gauge<T> createTableGaugeWithDeprecation(String name, String deprecated, Gauge<T> gauge, Gauge<G> globalGauge)
+    {
+        assert deprecated != null : "no deprecated metric name provided";
+        assert globalGauge != null : "no global Gauge metric provided";
+        
+        Gauge<T> cfGauge = Metrics.register(factory.createMetricName(name), 
+                                            gauge,
+                                            aliasFactory.createMetricName(name),
+                                            factory.createMetricName(deprecated),
+                                            aliasFactory.createMetricName(deprecated));
+        
+        if (register(name, name, deprecated, cfGauge))
+        {
+            Metrics.register(globalFactory.createMetricName(name), 
+                             globalGauge,
+                             globalAliasFactory.createMetricName(name),
+                             globalFactory.createMetricName(deprecated),
+                             globalAliasFactory.createMetricName(deprecated));
+        }
+        return cfGauge;
+    }
+
+    /**
      * Creates a counter that will also have a global counter thats the sum of all counters across
      * different column families
      */
@@ -1138,28 +1093,17 @@ public class TableMetrics
 
     protected TableTimer createTableTimer(String name, Timer keyspaceTimer)
     {
-        return createTableTimer(name, name, keyspaceTimer);
-    }
+        Timer cfTimer = Metrics.timer(factory.createMetricName(name), aliasFactory.createMetricName(name));
+        register(name, name, keyspaceTimer);
+        Timer global = Metrics.timer(globalFactory.createMetricName(name), globalAliasFactory.createMetricName(name));
 
-    protected TableTimer createTableTimer(String name, String alias, Timer keyspaceTimer)
-    {
-        Timer cfTimer = Metrics.timer(factory.createMetricName(name), aliasFactory.createMetricName(alias));
-        register(name, alias, cfTimer);
-        return new TableTimer(cfTimer,
-                              keyspaceTimer,
-                              Metrics.timer(globalFactory.createMetricName(name),
-                                            globalAliasFactory.createMetricName(alias)));
+        return new TableTimer(cfTimer, keyspaceTimer, global);
     }
 
     protected Timer createTableTimer(String name)
     {
-        return createTableTimer(name, name);
-    }
-
-    protected Timer createTableTimer(String name, String alias)
-    {
-        Timer tableTimer = Metrics.timer(factory.createMetricName(name), aliasFactory.createMetricName(alias));
-        register(name, alias, tableTimer);
+        Timer tableTimer = Metrics.timer(factory.createMetricName(name), aliasFactory.createMetricName(name));
+        register(name, name, tableTimer);
         return tableTimer;
     }
 
@@ -1181,7 +1125,7 @@ public class TableMetrics
     private LatencyMetrics createLatencyMetrics(String namePrefix, LatencyMetrics ... parents)
     {
         LatencyMetrics metric = new LatencyMetrics(factory, namePrefix, parents);
-        all.add(() -> metric.release());
+        all.add(metric::release);
         return metric;
     }
 
@@ -1191,21 +1135,45 @@ public class TableMetrics
      */
     private boolean register(String name, String alias, Metric metric)
     {
+        return register(name, alias, null, metric);
+    }
+
+    /**
+     * Registers a metric to be removed when unloading CF.
+     * 
+     * @param name the name of the metric registered with the "Table" type
+     * @param alias the name of the metric registered with the legacy "ColumnFamily" type
+     * @param deprecated an optionally null deprecated name for the metric registered with the "Table"
+     * 
+     * @return true if first time metric with that name has been registered
+     */
+    private boolean register(String name, String alias, String deprecated, Metric metric)
+    {
         boolean ret = allTableMetrics.putIfAbsent(name, ConcurrentHashMap.newKeySet()) == null;
         allTableMetrics.get(name).add(metric);
-        all.add(() -> releaseMetric(name, alias));
+        all.add(() -> releaseMetric(name, alias, deprecated));
         return ret;
     }
 
-    private void releaseMetric(String metricName, String metricAlias)
+    private void releaseMetric(String tableMetricName, String cfMetricName, String tableMetricAlias)
     {
-        CassandraMetricsRegistry.MetricName name = factory.createMetricName(metricName);
-        CassandraMetricsRegistry.MetricName alias = aliasFactory.createMetricName(metricAlias);
+        CassandraMetricsRegistry.MetricName name = factory.createMetricName(tableMetricName);
+
         final Metric metric = Metrics.getMetrics().get(name.getMetricName());
         if (metric != null)
-        {   // Metric will be null if we are releasing a view metric.  Views have null for ViewLockAcquireTime and ViewLockReadTime
-            allTableMetrics.get(metricName).remove(metric);
-            Metrics.remove(name, alias);
+        {
+            // Metric will be null if we are releasing a view metric.  Views have null for ViewLockAcquireTime and ViewLockReadTime
+            allTableMetrics.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);
+            }
         }
     }
 
@@ -1213,9 +1181,12 @@ public class TableMetrics
     {
         public final Meter[] all;
         public final Meter table;
+        public final Meter global;
+
         private TableMeter(Meter table, Meter keyspace, Meter global)
         {
             this.table = table;
+            this.global = global;
             this.all = new Meter[]{table, keyspace, global};
         }
 
@@ -1232,9 +1203,12 @@ public class TableMetrics
     {
         public final Histogram[] all;
         public final Histogram cf;
+        public final Histogram global;
+
         private TableHistogram(Histogram cf, Histogram keyspace, Histogram global)
         {
             this.cf = cf;
+            this.global = global;
             this.all = new Histogram[]{cf, keyspace, global};
         }
 
@@ -1251,9 +1225,12 @@ public class TableMetrics
     {
         public final Timer[] all;
         public final Timer cf;
+        public final Timer global;
+
         private TableTimer(Timer cf, Timer keyspace, Timer global)
         {
             this.cf = cf;
+            this.global = global;
             this.all = new Timer[]{cf, keyspace, global};
         }
 
@@ -1345,4 +1322,24 @@ public class TableMetrics
     {
         void release();
     }
+
+    private static class GlobalTableGauge implements Gauge<Long>
+    {
+        private final String name;
+
+        public GlobalTableGauge(String name)
+        {
+            this.name = name;
+        }
+
+        public Long getValue()
+        {
+            long total = 0;
+            for (Metric cfGauge : allTableMetrics.get(name))
+            {
+                total = total + ((Gauge<? extends Number>) cfGauge).getValue().longValue();
+            }
+            return total;
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/repair/SyncTask.java b/src/java/org/apache/cassandra/repair/SyncTask.java
index d0f1eca..a63f037 100644
--- a/src/java/org/apache/cassandra/repair/SyncTask.java
+++ b/src/java/org/apache/cassandra/repair/SyncTask.java
@@ -97,6 +97,6 @@ public abstract class SyncTask extends AbstractFuture<SyncStat> implements Runna
     protected void finished()
     {
         if (startTime != Long.MIN_VALUE)
-            Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).metric.syncTime.update(System.currentTimeMillis() - startTime, TimeUnit.MILLISECONDS);
+            Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).metric.repairSyncTime.update(System.currentTimeMillis() - startTime, TimeUnit.MILLISECONDS);
     }
 }
diff --git a/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java b/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java
index 4f0ad4d..ed60dea 100644
--- a/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java
+++ b/src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java
@@ -85,7 +85,7 @@ import org.apache.cassandra.utils.btree.BTreeSet;
  * <p>
  * See CASSANDRA-8272, CASSANDRA-8273, and CASSANDRA-15907 for further details.
  */
-class ReplicaFilteringProtection<E extends Endpoints<E>>
+public class ReplicaFilteringProtection<E extends Endpoints<E>>
 {
     private static final Logger logger = LoggerFactory.getLogger(ReplicaFilteringProtection.class);
     private static final NoSpamLogger oneMinuteLogger = NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
diff --git a/test/unit/org/apache/cassandra/metrics/KeyspaceMetricsTest.java b/test/unit/org/apache/cassandra/metrics/KeyspaceMetricsTest.java
index ae92146..e941a84 100644
--- a/test/unit/org/apache/cassandra/metrics/KeyspaceMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/KeyspaceMetricsTest.java
@@ -31,6 +31,8 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.service.EmbeddedCassandraService;
+
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -41,7 +43,7 @@ public class KeyspaceMetricsTest extends SchemaLoader
 {
     private static Session session;
 
-    @BeforeClass()
+    @BeforeClass
     public static void setup() throws ConfigurationException, IOException
     {
         Schema.instance.clear();
@@ -71,4 +73,10 @@ public class KeyspaceMetricsTest extends SchemaLoader
         // no metrics after drop
         assertEquals(metrics.get().collect(Collectors.joining(",")), 0, metrics.get().count());
     }
+    
+    @AfterClass
+    public static void teardown()
+    {
+        session.close();
+    }
 }
diff --git a/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java b/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java
index 56ad401..3ce219b 100644
--- a/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java
@@ -23,6 +23,7 @@ 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;
@@ -45,14 +46,13 @@ import static org.junit.Assert.assertTrue;
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class TableMetricsTest extends SchemaLoader
 {
-
     private static Session session;
 
     private static final String KEYSPACE = "junit";
     private static final String TABLE = "tablemetricstest";
     private static final String COUNTER_TABLE = "tablemetricscountertest";
 
-    @BeforeClass()
+    @BeforeClass
     public static void setup() throws ConfigurationException, IOException
     {
         Schema.instance.clear();
@@ -278,4 +278,10 @@ public class TableMetricsTest extends SchemaLoader
         // no metrics after drop
         assertEquals(metrics.get().collect(Collectors.joining(",")), 0, metrics.get().count());
     }
+
+    @AfterClass
+    public static void teardown()
+    {
+        session.close();
+    }
 }


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