You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2020/06/12 01:00:44 UTC

[GitHub] [cassandra] stillalex opened a new pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

stillalex opened a new pull request #629:
URL: https://github.com/apache/cassandra/pull/629


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #629:
URL: https://github.com/apache/cassandra/pull/629#discussion_r439182343



##########
File path: src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
##########
@@ -166,186 +168,84 @@ public KeyspaceMetrics(final Keyspace ks)
     {
         factory = new KeyspaceMetricNameFactory(ks);
         keyspace = ks;
-        memtableColumnsCount = createKeyspaceGauge("MemtableColumnsCount", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.memtableColumnsCount.getValue();
-            }
-        });
-        memtableLiveDataSize = createKeyspaceGauge("MemtableLiveDataSize", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.memtableLiveDataSize.getValue();
-            }
-        });
-        memtableOnHeapDataSize = createKeyspaceGauge("MemtableOnHeapDataSize", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.memtableOnHeapSize.getValue();
-            }
-        });
-        memtableOffHeapDataSize = createKeyspaceGauge("MemtableOffHeapDataSize", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.memtableOffHeapSize.getValue();
-            }
-        });
-        allMemtablesLiveDataSize = createKeyspaceGauge("AllMemtablesLiveDataSize", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.allMemtablesLiveDataSize.getValue();
-            }
-        });
-        allMemtablesOnHeapDataSize = createKeyspaceGauge("AllMemtablesOnHeapDataSize", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.allMemtablesOnHeapSize.getValue();
-            }
-        });
-        allMemtablesOffHeapDataSize = createKeyspaceGauge("AllMemtablesOffHeapDataSize", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.allMemtablesOffHeapSize.getValue();
-            }
-        });
-        memtableSwitchCount = createKeyspaceGauge("MemtableSwitchCount", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.memtableSwitchCount.getCount();
-            }
-        });
-        pendingCompactions = createKeyspaceGauge("PendingCompactions", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return (long) metric.pendingCompactions.getValue();
-            }
-        });
-        pendingFlushes = createKeyspaceGauge("PendingFlushes", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return (long) metric.pendingFlushes.getCount();
-            }
-        });
-        liveDiskSpaceUsed = createKeyspaceGauge("LiveDiskSpaceUsed", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.liveDiskSpaceUsed.getCount();
-            }
-        });
-        totalDiskSpaceUsed = createKeyspaceGauge("TotalDiskSpaceUsed", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.totalDiskSpaceUsed.getCount();
-            }
-        });
-        bloomFilterDiskSpaceUsed = createKeyspaceGauge("BloomFilterDiskSpaceUsed", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.bloomFilterDiskSpaceUsed.getValue();
-            }
-        });
-        bloomFilterOffHeapMemoryUsed = createKeyspaceGauge("BloomFilterOffHeapMemoryUsed", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.bloomFilterOffHeapMemoryUsed.getValue();
-            }
-        });
-        indexSummaryOffHeapMemoryUsed = createKeyspaceGauge("IndexSummaryOffHeapMemoryUsed", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.indexSummaryOffHeapMemoryUsed.getValue();
-            }
-        });
-        compressionMetadataOffHeapMemoryUsed = createKeyspaceGauge("CompressionMetadataOffHeapMemoryUsed", new MetricValue()
-        {
-            public Long getValue(TableMetrics metric)
-            {
-                return metric.compressionMetadataOffHeapMemoryUsed.getValue();
-            }
-        });
+        memtableColumnsCount = createKeyspaceGauge("MemtableColumnsCount",
+                metric -> metric.memtableColumnsCount.getValue());
+        memtableLiveDataSize = createKeyspaceGauge("MemtableLiveDataSize",
+                metric -> metric.memtableLiveDataSize.getValue());
+        memtableOnHeapDataSize = createKeyspaceGauge("MemtableOnHeapDataSize",
+                metric -> metric.memtableOnHeapSize.getValue());
+        memtableOffHeapDataSize = createKeyspaceGauge("MemtableOffHeapDataSize",
+                metric -> metric.memtableOffHeapSize.getValue());
+        allMemtablesLiveDataSize = createKeyspaceGauge("AllMemtablesLiveDataSize",
+                metric -> metric.allMemtablesLiveDataSize.getValue());
+        allMemtablesOnHeapDataSize = createKeyspaceGauge("AllMemtablesOnHeapDataSize",
+                metric -> metric.allMemtablesOnHeapSize.getValue());
+        allMemtablesOffHeapDataSize = createKeyspaceGauge("AllMemtablesOffHeapDataSize",
+                metric -> metric.allMemtablesOffHeapSize.getValue());
+        memtableSwitchCount = createKeyspaceGauge("MemtableSwitchCount",
+                metric -> metric.memtableSwitchCount.getCount());
+        pendingCompactions = createKeyspaceGauge("PendingCompactions", metric -> metric.pendingCompactions.getValue());
+        pendingFlushes = createKeyspaceGauge("PendingFlushes", metric -> metric.pendingFlushes.getCount());
+        liveDiskSpaceUsed = createKeyspaceGauge("LiveDiskSpaceUsed", metric -> metric.liveDiskSpaceUsed.getCount());
+        totalDiskSpaceUsed = createKeyspaceGauge("TotalDiskSpaceUsed", metric -> metric.totalDiskSpaceUsed.getCount());
+        bloomFilterDiskSpaceUsed = createKeyspaceGauge("BloomFilterDiskSpaceUsed",
+                metric -> metric.bloomFilterDiskSpaceUsed.getValue());
+        bloomFilterOffHeapMemoryUsed = createKeyspaceGauge("BloomFilterOffHeapMemoryUsed",
+                metric -> metric.bloomFilterOffHeapMemoryUsed.getValue());
+        indexSummaryOffHeapMemoryUsed = createKeyspaceGauge("IndexSummaryOffHeapMemoryUsed",
+                metric -> metric.indexSummaryOffHeapMemoryUsed.getValue());
+        compressionMetadataOffHeapMemoryUsed = createKeyspaceGauge("CompressionMetadataOffHeapMemoryUsed",
+                metric -> metric.compressionMetadataOffHeapMemoryUsed.getValue());
+
         // latency metrics for TableMetrics to update
-        readLatency = new LatencyMetrics(factory, "Read");
-        writeLatency = new LatencyMetrics(factory, "Write");
-        rangeLatency = new LatencyMetrics(factory, "Range");
+        readLatency = createLatencyMetrics("Read");
+        writeLatency = createLatencyMetrics("Write");
+        rangeLatency = createLatencyMetrics("Range");
+
         // create histograms for TableMetrics to replicate updates to
-        sstablesPerReadHistogram = Metrics.histogram(factory.createMetricName("SSTablesPerReadHistogram"), true);
-        tombstoneScannedHistogram = Metrics.histogram(factory.createMetricName("TombstoneScannedHistogram"), false);
-        liveScannedHistogram = Metrics.histogram(factory.createMetricName("LiveScannedHistogram"), false);
-        colUpdateTimeDeltaHistogram = Metrics.histogram(factory.createMetricName("ColUpdateTimeDeltaHistogram"), false);
-        viewLockAcquireTime =  Metrics.timer(factory.createMetricName("ViewLockAcquireTime"));
-        viewReadTime = Metrics.timer(factory.createMetricName("ViewReadTime"));
-        // add manually since histograms do not use createKeyspaceGauge method
-        allMetrics.addAll(Lists.newArrayList("SSTablesPerReadHistogram", "TombstoneScannedHistogram", "LiveScannedHistogram"));
+        sstablesPerReadHistogram = createKeyspaceHistogram("SSTablesPerReadHistogram", true);
+        tombstoneScannedHistogram = createKeyspaceHistogram("TombstoneScannedHistogram", false);
+        liveScannedHistogram = createKeyspaceHistogram("LiveScannedHistogram", false);
+        colUpdateTimeDeltaHistogram = createKeyspaceHistogram("ColUpdateTimeDeltaHistogram", false);
+        viewLockAcquireTime = createKeyspaceTimer("ViewLockAcquireTime");
+        viewReadTime = createKeyspaceTimer("ViewReadTime");
 
-        casPrepare = new LatencyMetrics(factory, "CasPrepare");
-        casPropose = new LatencyMetrics(factory, "CasPropose");
-        casCommit = new LatencyMetrics(factory, "CasCommit");
-        writeFailedIdealCL = Metrics.counter(factory.createMetricName("WriteFailedIdealCL"));
-        idealCLWriteLatency = new LatencyMetrics(factory, "IdealCLWrite");
+        casPrepare = createLatencyMetrics("CasPrepare");
+        casPropose = createLatencyMetrics("CasPropose");
+        casCommit = createLatencyMetrics("CasCommit");
+        writeFailedIdealCL = createKeyspaceCounter("WriteFailedIdealCL");
+        idealCLWriteLatency = createLatencyMetrics("IdealCLWrite");
 
         speculativeRetries = createKeyspaceCounter("SpeculativeRetries", metric -> metric.speculativeRetries.getCount());
         speculativeFailedRetries = createKeyspaceCounter("SpeculativeFailedRetries", metric -> metric.speculativeFailedRetries.getCount());
         speculativeInsufficientReplicas = createKeyspaceCounter("SpeculativeInsufficientReplicas", metric -> metric.speculativeInsufficientReplicas.getCount());
         additionalWrites = createKeyspaceCounter("AdditionalWrites", metric -> metric.additionalWrites.getCount());
         repairsStarted = createKeyspaceCounter("RepairJobsStarted", metric -> metric.repairsStarted.getCount());
         repairsCompleted = createKeyspaceCounter("RepairJobsCompleted", metric -> metric.repairsCompleted.getCount());
-        repairTime = Metrics.timer(factory.createMetricName("RepairTime"));
-        repairPrepareTime = Metrics.timer(factory.createMetricName("RepairPrepareTime"));
-        anticompactionTime = Metrics.timer(factory.createMetricName("AntiCompactionTime"));
-        validationTime = Metrics.timer(factory.createMetricName("ValidationTime"));
-        repairSyncTime = Metrics.timer(factory.createMetricName("RepairSyncTime"));
-        partitionsValidated = Metrics.histogram(factory.createMetricName("PartitionsValidated"), false);
-        bytesValidated = Metrics.histogram(factory.createMetricName("BytesValidated"), false);
+        repairTime =createKeyspaceTimer("RepairTime");
+        repairPrepareTime = createKeyspaceTimer("RepairPrepareTime");
+        anticompactionTime = createKeyspaceTimer("AntiCompactionTime");
+        validationTime = createKeyspaceTimer("ValidationTime");
+        repairSyncTime = createKeyspaceTimer("RepairSyncTime");
+        partitionsValidated = createKeyspaceHistogram("PartitionsValidated", false);
+        bytesValidated = createKeyspaceHistogram("BytesValidated", false);
 
-        confirmedRepairedInconsistencies = Metrics.meter(factory.createMetricName("RepairedDataInconsistenciesConfirmed"));
-        unconfirmedRepairedInconsistencies = Metrics.meter(factory.createMetricName("RepairedDataInconsistenciesUnconfirmed"));
+        confirmedRepairedInconsistencies = createKeyspaceMeter("RepairedDataInconsistenciesConfirmed");
+        unconfirmedRepairedInconsistencies = createKeyspaceMeter("RepairedDataInconsistenciesUnconfirmed");
 
-        repairedDataTrackingOverreadRows = Metrics.histogram(factory.createMetricName("RepairedOverreadRows"), false);
-        repairedDataTrackingOverreadTime = Metrics.timer(factory.createMetricName("RepairedOverreadTime"));
+        repairedDataTrackingOverreadRows = createKeyspaceHistogram("RepairedOverreadRows", false);
+        repairedDataTrackingOverreadTime = createKeyspaceTimer("RepairedOverreadTime");
     }
 
     /**
      * Release all associated metrics.
      */
     public void release()
     {
-        for(String name : allMetrics)
+        for(ReleasableMetric metric : allMetrics)

Review comment:
       ```suggestion
           for (ReleasableMetric metric : allMetrics)
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] smiklosovic closed pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
smiklosovic closed pull request #629:
URL: https://github.com/apache/cassandra/pull/629


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] stillalex commented on a change in pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
stillalex commented on a change in pull request #629:
URL: https://github.com/apache/cassandra/pull/629#discussion_r439183924



##########
File path: test/unit/org/apache/cassandra/metrics/ViewWriteMetricsTest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.function.LongSupplier;
+
+import org.apache.cassandra.SchemaLoader;
+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.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+
+public class ViewWriteMetricsTest extends SchemaLoader
+{
+    private static Session session;
+
+    @BeforeClass()
+    public static void setup() throws ConfigurationException, IOException
+    {
+        Schema.instance.clear();
+
+        EmbeddedCassandraService cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+
+        Cluster cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build();
+        session = cluster.connect();
+    }
+
+    @Test
+    public void testMetricsCleanupOnDrop()
+    {
+        String keyspace = "junit";
+        String table = "viewwritemetrics";
+        String view = "viewwritemetrics_materialized_view";
+
+        CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics;
+        LongSupplier count = () -> registry.getNames().stream().filter(n -> n.contains(view)).count();
+
+        // no metrics before create
+        assertEquals(0, count.getAsLong());
+
+        // TODO the ViewWriteMetrics metrics are never created
+        session.execute(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 };", keyspace));
+        session.execute(String.format("CREATE TABLE IF NOT EXISTS %s.%s (id int, val text, PRIMARY KEY(id, val));", keyspace, table));
+        session.execute(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT id,val FROM %s.%s WHERE id IS NOT NULL AND val IS NOT NULL PRIMARY KEY (id,val);", keyspace, view, keyspace, table));
+
+        // TODO needed?
+        //session.execute(String.format("INSERT INTO %s.%s (id, val) VALUES (1, 'Apache Cassandra');", keyspace, table));
+
+        // some metrics
+        assertTrue(count.getAsLong() > 0);
+
+        session.execute(String.format("DROP MATERIALIZED VIEW %s.%s;", keyspace, view));
+        session.execute(String.format("DROP TABLE %s.%s;", keyspace, table));

Review comment:
       this was not part of the test, it was the cleanup




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #629:
URL: https://github.com/apache/cassandra/pull/629#discussion_r439181389



##########
File path: test/unit/org/apache/cassandra/metrics/ViewWriteMetricsTest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.function.LongSupplier;
+
+import org.apache.cassandra.SchemaLoader;
+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.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+
+public class ViewWriteMetricsTest extends SchemaLoader
+{
+    private static Session session;
+
+    @BeforeClass()
+    public static void setup() throws ConfigurationException, IOException
+    {
+        Schema.instance.clear();
+
+        EmbeddedCassandraService cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+
+        Cluster cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build();
+        session = cluster.connect();
+    }
+
+    @Test
+    public void testMetricsCleanupOnDrop()
+    {
+        String keyspace = "junit";
+        String table = "viewwritemetrics";
+        String view = "viewwritemetrics_materialized_view";
+
+        CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics;
+        LongSupplier count = () -> registry.getNames().stream().filter(n -> n.contains(view)).count();
+
+        // no metrics before create
+        assertEquals(0, count.getAsLong());
+
+        // TODO the ViewWriteMetrics metrics are never created

Review comment:
       @stillalex There are two kinds of metrics for MVs. The first represents the conditionally registered metrics in `TableMetrics`, for example `viewReadTime`. Then you have `ViewWriteMetrics` which is created on the `StorageProxy` and spans tables. (I think I mentioned the latter a bit too carelessly in one of my earlier comments on the Jira. Apologies.)
   
   In any case, all I think we need to do is drop this test at the end of `TableMetricsTest` (maybe renaming to `testViewMetricsCleanupOnDrop()`)and we're golden. That will ensure that even MVs-specific metrics in `TableMetrics` are removed. Does that make sense?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] stillalex commented on a change in pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
stillalex commented on a change in pull request #629:
URL: https://github.com/apache/cassandra/pull/629#discussion_r439184726



##########
File path: src/java/org/apache/cassandra/metrics/TableMetrics.java
##########
@@ -981,28 +971,10 @@ public void updateSSTableIterated(int count)
      */
     public void release()
     {
-        for(Map.Entry<String, String> entry : all.entrySet())
-        {
-            CassandraMetricsRegistry.MetricName name = factory.createMetricName(entry.getKey());
-            CassandraMetricsRegistry.MetricName alias = aliasFactory.createMetricName(entry.getValue());
-            final Metric metric = Metrics.getMetrics().get(name.getMetricName());
-            if (metric != null)
-            {   // Metric will be null if it's a view metric we are releasing. Views have null for ViewLockAcquireTime and ViewLockReadTime
-                allTableMetrics.get(entry.getKey()).remove(metric);
-                Metrics.remove(name, alias);
-            }
+        for(ReleasableMetric entry : all)

Review comment:
       same for KeyspaceMetrics




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #629:
URL: https://github.com/apache/cassandra/pull/629#discussion_r439180287



##########
File path: test/unit/org/apache/cassandra/metrics/ViewWriteMetricsTest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.function.LongSupplier;
+
+import org.apache.cassandra.SchemaLoader;
+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.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+
+public class ViewWriteMetricsTest extends SchemaLoader
+{
+    private static Session session;
+
+    @BeforeClass()
+    public static void setup() throws ConfigurationException, IOException
+    {
+        Schema.instance.clear();
+
+        EmbeddedCassandraService cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+
+        Cluster cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build();
+        session = cluster.connect();
+    }
+
+    @Test
+    public void testMetricsCleanupOnDrop()
+    {
+        String keyspace = "junit";
+        String table = "viewwritemetrics";
+        String view = "viewwritemetrics_materialized_view";
+
+        CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics;
+        LongSupplier count = () -> registry.getNames().stream().filter(n -> n.contains(view)).count();
+
+        // no metrics before create
+        assertEquals(0, count.getAsLong());
+
+        // TODO the ViewWriteMetrics metrics are never created
+        session.execute(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 };", keyspace));
+        session.execute(String.format("CREATE TABLE IF NOT EXISTS %s.%s (id int, val text, PRIMARY KEY(id, val));", keyspace, table));
+        session.execute(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT id,val FROM %s.%s WHERE id IS NOT NULL AND val IS NOT NULL PRIMARY KEY (id,val);", keyspace, view, keyspace, table));
+
+        // TODO needed?
+        //session.execute(String.format("INSERT INTO %s.%s (id, val) VALUES (1, 'Apache Cassandra');", keyspace, table));
+
+        // some metrics
+        assertTrue(count.getAsLong() > 0);
+
+        session.execute(String.format("DROP MATERIALIZED VIEW %s.%s;", keyspace, view));
+        session.execute(String.format("DROP TABLE %s.%s;", keyspace, table));

Review comment:
       You shouldn't have to drop the source table or the keyspace to get the MV's metrics removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #629:
URL: https://github.com/apache/cassandra/pull/629#discussion_r439181389



##########
File path: test/unit/org/apache/cassandra/metrics/ViewWriteMetricsTest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.function.LongSupplier;
+
+import org.apache.cassandra.SchemaLoader;
+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.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+
+public class ViewWriteMetricsTest extends SchemaLoader
+{
+    private static Session session;
+
+    @BeforeClass()
+    public static void setup() throws ConfigurationException, IOException
+    {
+        Schema.instance.clear();
+
+        EmbeddedCassandraService cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+
+        Cluster cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build();
+        session = cluster.connect();
+    }
+
+    @Test
+    public void testMetricsCleanupOnDrop()
+    {
+        String keyspace = "junit";
+        String table = "viewwritemetrics";
+        String view = "viewwritemetrics_materialized_view";
+
+        CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics;
+        LongSupplier count = () -> registry.getNames().stream().filter(n -> n.contains(view)).count();
+
+        // no metrics before create
+        assertEquals(0, count.getAsLong());
+
+        // TODO the ViewWriteMetrics metrics are never created

Review comment:
       @stillalex There are two kinds of metrics for MVs. The first represents the conditionally registered metrics in `TableMetrics`, for example `viewReadTime`. Then you have `ViewWriteMetrics` which is created on the `StorageProxy` and spans tables. (I think I mentioned the latter a bit too carelessly in one of my earlier comments on the Jira. Apologies.)
   
   In any case, all I think we need to do is drop this test at the end of `TableMetricsTest` and we're golden. That will ensure that even MVs-specific metrics in `TableMetrics` are removed. Does that make sense?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #629:
URL: https://github.com/apache/cassandra/pull/629#discussion_r439190544



##########
File path: test/unit/org/apache/cassandra/metrics/ViewWriteMetricsTest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.function.LongSupplier;
+
+import org.apache.cassandra.SchemaLoader;
+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.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+
+public class ViewWriteMetricsTest extends SchemaLoader
+{
+    private static Session session;
+
+    @BeforeClass()
+    public static void setup() throws ConfigurationException, IOException
+    {
+        Schema.instance.clear();
+
+        EmbeddedCassandraService cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+
+        Cluster cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build();
+        session = cluster.connect();
+    }
+
+    @Test
+    public void testMetricsCleanupOnDrop()
+    {
+        String keyspace = "junit";
+        String table = "viewwritemetrics";
+        String view = "viewwritemetrics_materialized_view";
+
+        CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics;
+        LongSupplier count = () -> registry.getNames().stream().filter(n -> n.contains(view)).count();
+
+        // no metrics before create
+        assertEquals(0, count.getAsLong());
+
+        // TODO the ViewWriteMetrics metrics are never created

Review comment:
       If you want to verify here that `ViewLockAcquireTime` and `ViewReadTime` are created only the base table and not an MV, you could filter the names on each of them, etc.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
maedhroz commented on pull request #629:
URL: https://github.com/apache/cassandra/pull/629#issuecomment-645422789


   @stillalex I think we're probably good to commit this, although squashing first might make it easier for the committer ;)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] stillalex commented on pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
stillalex commented on pull request #629:
URL: https://github.com/apache/cassandra/pull/629#issuecomment-643335193


   @maedhroz very good suggestion! I changed the tests to include the metrics in the error message. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] stillalex commented on a change in pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
stillalex commented on a change in pull request #629:
URL: https://github.com/apache/cassandra/pull/629#discussion_r439184219



##########
File path: test/unit/org/apache/cassandra/metrics/ViewWriteMetricsTest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.function.LongSupplier;
+
+import org.apache.cassandra.SchemaLoader;
+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.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+
+public class ViewWriteMetricsTest extends SchemaLoader
+{
+    private static Session session;
+
+    @BeforeClass()
+    public static void setup() throws ConfigurationException, IOException
+    {
+        Schema.instance.clear();
+
+        EmbeddedCassandraService cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+
+        Cluster cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build();
+        session = cluster.connect();
+    }
+
+    @Test
+    public void testMetricsCleanupOnDrop()
+    {
+        String keyspace = "junit";
+        String table = "viewwritemetrics";
+        String view = "viewwritemetrics_materialized_view";
+
+        CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics;
+        LongSupplier count = () -> registry.getNames().stream().filter(n -> n.contains(view)).count();
+
+        // no metrics before create
+        assertEquals(0, count.getAsLong());
+
+        // TODO the ViewWriteMetrics metrics are never created

Review comment:
       I'll move the test over, sure. the trouble is I'm trying to verify (manually) that those metrics are actually created, otherwise that part of the code is not covered, so the test is not actually testing anything :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] stillalex commented on pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
stillalex commented on pull request #629:
URL: https://github.com/apache/cassandra/pull/629#issuecomment-643046865


   @maedhroz updated the PR based on your suggestions, let me know if you want to dig deeper into the MV tests, or if this is sufficient.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #629:
URL: https://github.com/apache/cassandra/pull/629#discussion_r439182440



##########
File path: src/java/org/apache/cassandra/metrics/TableMetrics.java
##########
@@ -981,28 +971,10 @@ public void updateSSTableIterated(int count)
      */
     public void release()
     {
-        for(Map.Entry<String, String> entry : all.entrySet())
-        {
-            CassandraMetricsRegistry.MetricName name = factory.createMetricName(entry.getKey());
-            CassandraMetricsRegistry.MetricName alias = aliasFactory.createMetricName(entry.getValue());
-            final Metric metric = Metrics.getMetrics().get(name.getMetricName());
-            if (metric != null)
-            {   // Metric will be null if it's a view metric we are releasing. Views have null for ViewLockAcquireTime and ViewLockReadTime
-                allTableMetrics.get(entry.getKey()).remove(metric);
-                Metrics.remove(name, alias);
-            }
+        for(ReleasableMetric entry : all)

Review comment:
       ```suggestion
           for (ReleasableMetric entry : all)
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #629:
URL: https://github.com/apache/cassandra/pull/629#discussion_r439188732



##########
File path: test/unit/org/apache/cassandra/metrics/ViewWriteMetricsTest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.function.LongSupplier;
+
+import org.apache.cassandra.SchemaLoader;
+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.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+
+public class ViewWriteMetricsTest extends SchemaLoader
+{
+    private static Session session;
+
+    @BeforeClass()
+    public static void setup() throws ConfigurationException, IOException
+    {
+        Schema.instance.clear();
+
+        EmbeddedCassandraService cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+
+        Cluster cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build();
+        session = cluster.connect();
+    }
+
+    @Test
+    public void testMetricsCleanupOnDrop()
+    {
+        String keyspace = "junit";
+        String table = "viewwritemetrics";
+        String view = "viewwritemetrics_materialized_view";
+
+        CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics;
+        LongSupplier count = () -> registry.getNames().stream().filter(n -> n.contains(view)).count();
+
+        // no metrics before create
+        assertEquals(0, count.getAsLong());
+
+        // TODO the ViewWriteMetrics metrics are never created

Review comment:
       An MV actually has fewer metrics than a normal table, due to `ViewLockAcquireTime` and `ViewReadTime` being present only for the latter. This test is still valuable, as it makes sure removal still works when those aren't present (i.e. the conditional logic in `releaseMetric()`).
   
   Either way, I don't think we need to worry about `ViewWriteMetrics`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #629: CASSANDRA-14888 Several mbeans are not unregistered when dropping a keyspace and table

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #629:
URL: https://github.com/apache/cassandra/pull/629#discussion_r439180947



##########
File path: test/unit/org/apache/cassandra/metrics/ViewWriteMetricsTest.java
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.function.LongSupplier;
+
+import org.apache.cassandra.SchemaLoader;
+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.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+
+public class ViewWriteMetricsTest extends SchemaLoader
+{
+    private static Session session;
+
+    @BeforeClass()
+    public static void setup() throws ConfigurationException, IOException
+    {
+        Schema.instance.clear();
+
+        EmbeddedCassandraService cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+
+        Cluster cluster = Cluster.builder().addContactPoint("127.0.0.1").withPort(DatabaseDescriptor.getNativeTransportPort()).build();
+        session = cluster.connect();
+    }
+
+    @Test
+    public void testMetricsCleanupOnDrop()
+    {
+        String keyspace = "junit";
+        String table = "viewwritemetrics";
+        String view = "viewwritemetrics_materialized_view";
+
+        CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics;
+        LongSupplier count = () -> registry.getNames().stream().filter(n -> n.contains(view)).count();
+
+        // no metrics before create
+        assertEquals(0, count.getAsLong());
+
+        // TODO the ViewWriteMetrics metrics are never created
+        session.execute(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 };", keyspace));
+        session.execute(String.format("CREATE TABLE IF NOT EXISTS %s.%s (id int, val text, PRIMARY KEY(id, val));", keyspace, table));
+        session.execute(String.format("CREATE MATERIALIZED VIEW %s.%s AS SELECT id,val FROM %s.%s WHERE id IS NOT NULL AND val IS NOT NULL PRIMARY KEY (id,val);", keyspace, view, keyspace, table));
+
+        // TODO needed?
+        //session.execute(String.format("INSERT INTO %s.%s (id, val) VALUES (1, 'Apache Cassandra');", keyspace, table));

Review comment:
       Can confirm. We shouldn't need this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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