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 2022/01/24 14:52:22 UTC

[GitHub] [cassandra] skoppu22 opened a new pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

skoppu22 opened a new pull request #1421:
URL: https://github.com/apache/cassandra/pull/1421


   Exposing CQL metrics to a virtual table. 
   
   Below are CQL metrics from https://cassandra.apache.org/doc/latest/cassandra/operating/metrics.html#:~:text=org.apache.cassandra.metrics%3Atype%3DCQL,that%20are%20prepared%20vs%20unprepared.
   
   Name | Type | Description
   -- | -- | --
   PreparedStatementsCount | Gauge<Integer> | Number of cached prepared statements.
   PreparedStatementsEvicted | Counter | Number of prepared statements evicted from the prepared statement cache
   PreparedStatementsExecuted | Counter | Number of prepared statements executed.
   RegularStatementsExecuted | Counter | Number of non prepared statements executed.
   PreparedStatementsRatio | Gauge<Double> | Percentage of statements that are prepared vs unprepared.
   
   
   Here is the virtual table created with this diff, output from cqlsh:
   
   cqlsh> select * from system_views.cql_metrics ;
   
    name  | prepared_statements_count | prepared_statements_evicted | prepared_statements_executed | prepared_statements_ratio | regular_statements_executed
   -------+---------------------------+-----------------------------+------------------------------+---------------------------+-----------------------------
    value |                         0 |                           0 |                            0 |                         0 |                          17
   


-- 
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] yifan-c commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #1421:
URL: https://github.com/apache/cassandra/pull/1421#discussion_r792171871



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       I would lean to use `cql` as the name, based on my understanding of the pattern of the tables. However, taking 1 step back, the `name` col is not important at all in this case. 
   
   There is only 1 metric set and we do not really need the column to distinguish the rows. (Because there is only one row). 
   
   How about just getting rid of the `name` column and using the first metric column as the PK? 




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)

Review comment:
       done




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       Updated doc as well @yifan-c @ekaterinadimitrova2 please have a look




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       PS @skoppu22 do you mind to update also the docs? We weren't doing it lately due to the docs migration and now we have backlog but there is no reason to postpone new doc updates anymore. I can also help with that if there is a need. Just ping me




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns
+        assertEquals(6, vtsRows.getColumnDefinitions().size());
+
+        AtomicInteger rowCount = new AtomicInteger(0);
+        vtsRows.forEach(r -> {
+            assertEquals(expectedMetrics.preparedStatementsCount.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_COUNT_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsEvicted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EVICTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EXECUTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsRatio.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_RATIO_COL), 0.01);
+            assertEquals(expectedMetrics.regularStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.REGULAR_STATEMENTS_EXECUTED_COL), 0);
+            rowCount.getAndIncrement();
+        });
+
+        // validate - number of rows
+        assertEquals(1, rowCount.get());
+    }
+
+    @Test
+    public void testUsingPrepareStmts() throws Throwable
+    {
+        CQLMetricsTable table = new CQLMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));

Review comment:
       This was my initial code. But as we want to inject our own cqlMetrics in other test case, this doesn't work for both test cases.




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;

Review comment:
       With inverted table code, this comment no longer applicable




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       I am confused, do we agree to invert them then? 
   I don't see the `CachesTable` as a good comparison to the batch_metrics. I am curious to hear also what @blerer thinks as he was reviewing the second ticket so it is really a good topic for all of us to get aligned with, I totally agree on that. 




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       PS @skoppu22 do you mind to update also the docs? We weren't doing it lately due to the docs migration and now we have backlog but there is no reason to postpone doc updates anymore. I can also help with that if there is a need. Just ping me




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics

Review comment:
       `I feel comments are like chocolates, like to have as many as we can :-)`
   Wow, how did you know I am a chocoholic :D  Jokes aside, while I agree with you and unfortunately, we lack them often, in this case we just repeat things that are on the next line - like in this case you see it is Virtual Table and the CQL metrics is in the name. I don't mean to be super particular here but I know other engineers on the project are serious on that (talking from experience from my own reviews where I tended to put a lot of comments sometimes :D ) so just saying. 
   Also, in this case these comments are probably not at places that we expect a lot of code changes but according to the Clean Code rules - if we can make the code self explanatory that is the best (I think you already have that quite well done), better to skip them as in time with code changes, comments get pulled away from where they were meant to refer to and it becomes confusing. 




-- 
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] yifan-c commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #1421:
URL: https://github.com/apache/cassandra/pull/1421#discussion_r793094005



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       This is the PR referred: https://github.com/apache/cassandra/pull/1372
   To be consistent, I think its table format needs to be revised. 




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics

Review comment:
       I think we can skip this comment as the package and the class are already self-explanatory

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns
+        assertEquals(6, vtsRows.getColumnDefinitions().size());
+
+        AtomicInteger rowCount = new AtomicInteger(0);
+        vtsRows.forEach(r -> {
+            assertEquals(expectedMetrics.preparedStatementsCount.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_COUNT_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsEvicted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EVICTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EXECUTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsRatio.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_RATIO_COL), 0.01);
+            assertEquals(expectedMetrics.regularStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.REGULAR_STATEMENTS_EXECUTED_COL), 0);
+            rowCount.getAndIncrement();
+        });
+
+        // validate - number of rows
+        assertEquals(1, rowCount.get());
+    }
+
+    @Test
+    public void testUsingPrepareStmts() throws Throwable
+    {
+        CQLMetricsTable table = new CQLMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+
+        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        String tbl = createTable(ks, "CREATE TABLE %s (id int PRIMARY KEY, cid int, val text)");
+        Session session = sessionNet();
+
+        // prepare statements
+        String insertCQL = "INSERT INTO " + ks + "." + tbl + " (id, cid, val) VALUES (?, ?, ?)";
+        PreparedStatement preparedInsert = session.prepare(insertCQL);
+
+        String selectCQL = "Select * from " + ks + "." + tbl + " where id = ?";
+        PreparedStatement preparedSelect = session.prepare(selectCQL);
+
+        for (int i = 0; i < 10; i++)
+        {
+            // execute prepared statements

Review comment:
       I think we can skip this comment

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table

Review comment:
       I think we can skip this comment

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics

Review comment:
       I think we can skip this comment 

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns
+        assertEquals(6, vtsRows.getColumnDefinitions().size());
+
+        AtomicInteger rowCount = new AtomicInteger(0);
+        vtsRows.forEach(r -> {
+            assertEquals(expectedMetrics.preparedStatementsCount.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_COUNT_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsEvicted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EVICTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EXECUTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsRatio.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_RATIO_COL), 0.01);
+            assertEquals(expectedMetrics.regularStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.REGULAR_STATEMENTS_EXECUTED_COL), 0);
+            rowCount.getAndIncrement();
+        });
+
+        // validate - number of rows
+        assertEquals(1, rowCount.get());
+    }
+
+    @Test
+    public void testUsingPrepareStmts() throws Throwable
+    {
+        CQLMetricsTable table = new CQLMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));

Review comment:
       I think we can add something like that for clarity and to reduce the repetition of code below, WDYT?
   ```
       @Before
       public void config()
       {
           CQLMetricsTable table = new CQLMetricsTable(KS_NAME);
           VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
       }
   ```

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns
+        assertEquals(6, vtsRows.getColumnDefinitions().size());
+
+        AtomicInteger rowCount = new AtomicInteger(0);
+        vtsRows.forEach(r -> {
+            assertEquals(expectedMetrics.preparedStatementsCount.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_COUNT_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsEvicted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EVICTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EXECUTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsRatio.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_RATIO_COL), 0.01);
+            assertEquals(expectedMetrics.regularStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.REGULAR_STATEMENTS_EXECUTED_COL), 0);
+            rowCount.getAndIncrement();
+        });
+
+        // validate - number of rows
+        assertEquals(1, rowCount.get());
+    }
+
+    @Test
+    public void testUsingPrepareStmts() throws Throwable
+    {
+        CQLMetricsTable table = new CQLMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+
+        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        String tbl = createTable(ks, "CREATE TABLE %s (id int PRIMARY KEY, cid int, val text)");
+        Session session = sessionNet();
+
+        // prepare statements

Review comment:
       I think we can skip this comment

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;

Review comment:
       I don't think we really need `private final CQLMetrics cqlMetrics;`
   I would suggest 
   ```
       @Override
       public DataSet data()
       {
           SimpleDataSet result = new SimpleDataSet(metadata());
           CQLMetrics cqlMetrics = QueryProcessor.metrics;
           addRow(result, cqlMetrics);
   
           return result;
       }
   ```
   
   Then I will make another suggestion in the tests, please follow there :-) 

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name

Review comment:
       I think we can skip this comment

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns
+        assertEquals(6, vtsRows.getColumnDefinitions().size());
+
+        AtomicInteger rowCount = new AtomicInteger(0);
+        vtsRows.forEach(r -> {
+            assertEquals(expectedMetrics.preparedStatementsCount.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_COUNT_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsEvicted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EVICTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EXECUTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsRatio.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_RATIO_COL), 0.01);
+            assertEquals(expectedMetrics.regularStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.REGULAR_STATEMENTS_EXECUTED_COL), 0);
+            rowCount.getAndIncrement();
+        });
+
+        // validate - number of rows

Review comment:
       I think we can skip this comment

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns

Review comment:
       I think we can skip this comment

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       I am thinking maybe of `Metric`? (I have to warn you I am bad with names :-) )

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       We can also invert to table with two columns and 5 rows. I think this would be the format most tables follow actually




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics

Review comment:
       Agree, Thank you for explaining in detail




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       I just looked again in the tables and more specifically in the one that is about to get committed in CASSANDRA-17224 and reminded myself that one was the reason for me to think about the inversion. I think that one makes total sense to 
   be in that format and gives a direction for future metrics tables.. So I still feel we need to invert. What do others think? @yifan-c , please, consider the batch_metrics table below:
   ```
                    name                          | max | p50th | p999th | p99th
   -------------------------------+-----+-------+--------+-------
     partitions_per_counter_batch |   0 |     0 |      0 |     0
      partitions_per_logged_batch |   0 |     0 |      0 |     0
    partitions_per_unlogged_batch |   0 |     0 |      0 |     0
   ```




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics

Review comment:
       Removed comments




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics

Review comment:
       I feel comments are like chocolates, like to have as many as we can :-)




-- 
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] frankgh commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable

Review comment:
       Let's make this class `final` consistent with other virtual tables
   ```suggestion
   final class CQLMetricsTable extends AbstractVirtualTable
   ```

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       Maybe "cql" makes more sense for the value of the name column ?

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)

Review comment:
       Maybe match the type of the value without needing to cast it to double?
   ```suggestion
                              .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, Int32Type.instance)
                              .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, LongType.instance)
                              .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, LongType.instance)
                              .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
                              .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, LongType.instance)
   ```

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;

Review comment:
       minor nit: it might be worth considering inlining since we don't call `addRow` anywhere else?
   ```suggestion
           return new SimpleDataSet(metadata()).row(VALUE_ROW)
                                               .column(PREPARED_STATEMENTS_COUNT_COL, cqlMetrics.preparedStatementsCount.getValue())
                                               .column(PREPARED_STATEMENTS_EVICTED_COL, cqlMetrics.preparedStatementsEvicted.getCount())
                                               .column(PREPARED_STATEMENTS_EXECUTED_COL, cqlMetrics.preparedStatementsExecuted.getCount())
                                               .column(PREPARED_STATEMENTS_RATIO_COL, cqlMetrics.preparedStatementsRatio.getValue())
                                               .column(REGULAR_STATEMENTS_EXECUTED_COL, cqlMetrics.regularStatementsExecuted.getCount());
   ```




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       I am thinking maybe of `Metric`? (I have to warn you I am bad with naming things :-) )




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;

Review comment:
       Yes, using cqlMetrics to be able to test injected values. Especially testing evicted count was not possible in unit tests without this injection.




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable

Review comment:
       done




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: doc/modules/cassandra/pages/new/virtualtables.adoc
##########
@@ -74,6 +74,8 @@ recent_hit_rate_per_second, recent_request_rate_per_second, request_count, and s
 
 |coordinator_write_latency |Records counts, keyspace_name, table_name, max, median, and per_second for coordinator writes.
 
+|cql_metrcs |Lists metrics for CQL prepared statements.

Review comment:
       I would make it consistent with the original metrics doc here - https://cassandra.apache.org/doc/latest/cassandra/operating/metrics.html
   `Metrics specific to CQL prepared statement caching.`




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics

Review comment:
       I feel comments are like chocolates, like to have as many as we can :-)

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       'cql' is already in the table name, and doesn't describe what kind of data this row has. Metric makes sense to me, I can change it to 'Metric' if we agree.

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;

Review comment:
       Yes, using cqlMetrics to be able to test injected values. Especially testing evicted count was not possible in unit tests without this injection.

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)

Review comment:
       Without explicit conversion cql select query on this virtual table failing with ClassCastException. Same error for Long to Double as well.
   c```
   qlsh> select * from system_views.cql_metrics ;
   NoHostAvailable: ('Unable to complete the operation against any hosts', {<Host: 127.0.0.1:9042 datacenter1>: <Error from server: code=0000 [Server error] message="java.lang.ClassCastException: class java.lang.Integer cannot be cast to class java.lang.Double (java.lang.Integer and java.lang.Double are in module java.base of loader 'bootstrap')">})
   
   
   ```

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)

Review comment:
       Without explicit conversion cql select query on this virtual table failing with ClassCastException. Same error for Long to Double as well.
   
   ```
   qlsh> select * from system_views.cql_metrics ;
   NoHostAvailable: ('Unable to complete the operation against any hosts', {<Host: 127.0.0.1:9042 datacenter1>: <Error from server: code=0000 [Server error] message="java.lang.ClassCastException: class java.lang.Integer cannot be cast to class java.lang.Double (java.lang.Integer and java.lang.Double are in module java.base of loader 'bootstrap')">})
   ```
   
   

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;

Review comment:
       This is the pattern used in all virtual table classes, so like to keep inline with them.

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       Agree, me too like to have 5 rows format.
   @yifan-c to comment

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns
+        assertEquals(6, vtsRows.getColumnDefinitions().size());
+
+        AtomicInteger rowCount = new AtomicInteger(0);
+        vtsRows.forEach(r -> {
+            assertEquals(expectedMetrics.preparedStatementsCount.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_COUNT_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsEvicted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EVICTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EXECUTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsRatio.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_RATIO_COL), 0.01);
+            assertEquals(expectedMetrics.regularStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.REGULAR_STATEMENTS_EXECUTED_COL), 0);
+            rowCount.getAndIncrement();
+        });
+
+        // validate - number of rows
+        assertEquals(1, rowCount.get());
+    }
+
+    @Test
+    public void testUsingPrepareStmts() throws Throwable
+    {
+        CQLMetricsTable table = new CQLMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));

Review comment:
       This was my initial code. But as we want to inject our own cqlMetrics in other test case, this doesn't work for both test cases.




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       Agree, me too like to have 5 rows format.
   @yifan-c to comment




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       'cql' is already in the table name, and doesn't describe what kind of data this row has. Metric makes sense to me, I can change it to 'Metric' if we agree.




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       Btw now when the docs migration finished,  we need to consider updating the Virtual Tables page too. Also, we need to add NEWS.txt entry on commit (this one as a note to me and @yifan-c  as probably one of us will commit)




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       Happy that we are able to convince you. I have inverted the table format in the code, below is the output from updated code:
   ```
   
   cqlsh> select * from system_views.cql_metrics ;
   
    name                         | value
   ------------------------------+-------
       prepared_statements_count |     0
     prepared_statements_evicted |     0
    prepared_statements_executed |     0
       prepared_statements_ratio |     0
     regular_statements_executed |    17
   
   (5 rows)
   
   ```




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       I still think we need to invert the table, please, check my other comment




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       My view point, inversion of the table makes it to look better. Using metrics as columns makes us to scroll horizontally (columns wrap around to the next line) as they don't fit in screen width well, if you try to see the cql metrics output I pasted above you need to scroll to see all values. Instead if we just use metrics as rows, we will have only one column, so easy to view and interpret, that's why batch metrics output above feels comfortable to view.




-- 
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] yifan-c commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #1421:
URL: https://github.com/apache/cassandra/pull/1421#discussion_r793261212



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       Sorry for the confusion. 
   
   So.. I went through the current vtables. As you mentioned, there are 2 formats. The columns are either the metric names or the percentiles. The rows are always the belonging part that has the metrics. For example, 
   ```sql
   # Percentiles as column names. The metric name is the table name.
   cqlsh> SELECT * FROM system_views.rows_per_read;
   
    keyspace_name      | table_name                     | count | max | p50th | p99th
   --------------------+--------------------------------+-------+-----+-------+-------
                system |               available_ranges |     1 |   0 |     0 |     0
                system |            available_ranges_v2 |     0 |   0 |     0 |     0
                system |                        batches |    42 |   1 |     1 |     1 
   
   # metric name as column names 
   cqlsh> SELECT * FROM system_views.clients;
   
    address   | port  | connection_stage | driver_name            | driver_version | hostname  | protocol_version | request_count | ssl_cipher_suite | ssl_enabled | ssl_protocol | username
   -----------+-------+------------------+------------------------+----------------+-----------+------------------+---------------+------------------+-------------+--------------+-----------
    127.0.0.1 | 63678 |            ready | DataStax Python Driver |         3.25.0 | localhost |                5 |            28 |             null |       False |         null | anonymous
    127.0.0.1 | 63679 |            ready | DataStax Python Driver |         3.25.0 | localhost |                5 |            16 |             null |       False |         null | anonymous
   ```
   
   To me, the inversion is the third format, where the columns are the percentiles and the rows are the metrics. 
   ```sql
   name     |  value
   ---------+-------
   metric1  |  1
   metric2  |  0
   ```
   
   Maybe I was just being pedantic :| 
   
   What you (both Ekaterina and Shailaja) said make sense to me. I'd +1 to the proposed format. 
   Trying to sum up the discussion. For vtable styles, it is preferred to define less columns and have more rows to be better human readable. 




-- 
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] frankgh commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: doc/modules/cassandra/pages/new/virtualtables.adoc
##########
@@ -99,6 +99,9 @@ recent_hit_rate_per_second, recent_request_rate_per_second, request_count, and s
 |thread_pools |Lists metrics for each thread pool.
 
 |tombstones_per_read |Records counts, keyspace_name, tablek_name, max, and median for tombstones.
+
+|cql_metrcs |Lists metrics for CQL prepared statements.

Review comment:
       this should be sorted. Can you move it up?

##########
File path: doc/modules/cassandra/pages/new/virtualtables.adoc
##########
@@ -290,6 +293,22 @@ SELECT total - progress AS remaining
 FROM system_views.sstable_tasks;
 ....
 
+=== CQL metrics Virtual Table

Review comment:
       also, it looks like it needs to be moved up.




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       @blerer for the win :D 
   My point is I see batch_metrics more consistent with this new one being inverted than if we keep the current format. 
   Maybe it is me, let's see what others also think. Otherwise about consistency we are on the same page. 




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: doc/modules/cassandra/pages/new/virtualtables.adoc
##########
@@ -290,6 +293,22 @@ SELECT total - progress AS remaining
 FROM system_views.sstable_tasks;
 ....
 
+=== CQL metrics Virtual Table

Review comment:
       Looks like existing ones are not in order, moved anyhow




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+
+final class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_COL = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)

Review comment:
       I think I missed one thing here, let's add a table description, the way it is done here for example:
   https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/virtual/RolesCacheKeysTable.java#L33




-- 
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] frankgh commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)

Review comment:
       In the `addRow` method you need to remove the casting to double for the fields that don't need it. i.e. `Double.valueOf()`




-- 
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] yifan-c commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #1421:
URL: https://github.com/apache/cassandra/pull/1421#discussion_r794042382



##########
File path: doc/modules/cassandra/pages/new/virtualtables.adoc
##########
@@ -99,6 +99,9 @@ recent_hit_rate_per_second, recent_request_rate_per_second, request_count, and s
 |thread_pools |Lists metrics for each thread pool.
 
 |tombstones_per_read |Records counts, keyspace_name, tablek_name, max, and median for tombstones.
+
+|cql_metrcs |Lists metrics for CQL prepared statements.

Review comment:
       The metrics also include `regular_statements_executed`. The description can be 
   "Lists metrics for CQL statements."




-- 
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] smiklosovic closed pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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


   


-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: doc/modules/cassandra/pages/new/virtualtables.adoc
##########
@@ -170,6 +173,22 @@ counters |       26214400 |           0 |         0 |       NaN |
 (4 rows)
 ....
 
+=== CQL metrics Virtual Table
+
+Lists CQL prepared statements metrics. A query on CQL metrics virtual table lists below metrics.

Review comment:
       Same comment as before. If we opt out from duplication of certain info about the metrics (that we have already on the metrics page), I would say at least I prefer to say `Metrics specific to CQL prepared statement caching.`




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: doc/modules/cassandra/pages/new/virtualtables.adoc
##########
@@ -170,6 +173,22 @@ counters |       26214400 |           0 |         0 |       NaN |
 (4 rows)
 ....
 
+=== CQL metrics Virtual Table
+
+Lists CQL prepared statements metrics. A query on CQL metrics virtual table lists below metrics.

Review comment:
       Done




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: doc/modules/cassandra/pages/new/virtualtables.adoc
##########
@@ -99,6 +99,9 @@ recent_hit_rate_per_second, recent_request_rate_per_second, request_count, and s
 |thread_pools |Lists metrics for each thread pool.
 
 |tombstones_per_read |Records counts, keyspace_name, tablek_name, max, and median for tombstones.
+
+|cql_metrcs |Lists metrics for CQL prepared statements.

Review comment:
       > this should be sorted. Can you move it up?
   
   done




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: .circleci/config.yml
##########
@@ -21,10 +21,10 @@ jobs:
   j8_jvm_upgrade_dtests:

Review comment:
       Please ignore this file




-- 
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] yifan-c commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #1421:
URL: https://github.com/apache/cassandra/pull/1421#discussion_r793004247



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       To be clear. I do not think inversion is a good idea. It breaks consistency. And I think having a column for each metrics and rows for metrics sets organizes the data better, especially in the case of having multiple metrics sets. 
   
   I assumed the motivation of inversion was to place the percentile data. So I pointed out that there are existing tables that handles placing percentile data.




-- 
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] yifan-c commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #1421:
URL: https://github.com/apache/cassandra/pull/1421#discussion_r793038250



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       Alright. I have presented all my ideas and cannot repeat more. Consistency is the reason I argued to keep the current format. 
   
   Defer to @blerer for the final verdict :D




-- 
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] frankgh commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable

Review comment:
       Let's make this class `final` consistent with other virtual tables
   ```suggestion
   final class CQLMetricsTable extends AbstractVirtualTable
   ```

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       Maybe "cql" makes more sense for the value of the name column ?

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)

Review comment:
       Maybe match the type of the value without needing to cast it to double?
   ```suggestion
                              .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, Int32Type.instance)
                              .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, LongType.instance)
                              .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, LongType.instance)
                              .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
                              .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, LongType.instance)
   ```

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;

Review comment:
       minor nit: it might be worth considering inlining since we don't call `addRow` anywhere else?
   ```suggestion
           return new SimpleDataSet(metadata()).row(VALUE_ROW)
                                               .column(PREPARED_STATEMENTS_COUNT_COL, cqlMetrics.preparedStatementsCount.getValue())
                                               .column(PREPARED_STATEMENTS_EVICTED_COL, cqlMetrics.preparedStatementsEvicted.getCount())
                                               .column(PREPARED_STATEMENTS_EXECUTED_COL, cqlMetrics.preparedStatementsExecuted.getCount())
                                               .column(PREPARED_STATEMENTS_RATIO_COL, cqlMetrics.preparedStatementsRatio.getValue())
                                               .column(REGULAR_STATEMENTS_EXECUTED_COL, cqlMetrics.regularStatementsExecuted.getCount());
   ```

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)

Review comment:
       In the `addRow` method you need to remove the casting to double for the fields that don't need it. i.e. `Double.valueOf()`

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics

Review comment:
       maybe turn it into a javadoc instead?




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       Inverted table looks fine to me. @yifan-c to comment.




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       My view point, inversion of the table makes it to look better. Using metrics as columns makes us to scroll horizontally (columns wrap around to the next line) as they don't fit in screen width well, if you try to see the output I pasted above you need to scroll to see all values. If we just use metrics as rows, we will have only one column, so easy to view and interpret, that's why batch metrics output above feels comfortable to view.




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       > PS @skoppu22 do you mind to update also the docs? We weren't doing it lately due to the docs migration and now we have backlog but there is no reason to postpone new doc updates anymore. I can also help with that if there is a need. Just ping me
   
   Yes, I would like to update docs, can you please point me to docs, I am new to Cassandra




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)

Review comment:
       Without explicit conversion cql select query on this virtual table failing with ClassCastException. Same error for Long to Double as well.
   c```
   qlsh> select * from system_views.cql_metrics ;
   NoHostAvailable: ('Unable to complete the operation against any hosts', {<Host: 127.0.0.1:9042 datacenter1>: <Error from server: code=0000 [Server error] message="java.lang.ClassCastException: class java.lang.Integer cannot be cast to class java.lang.Double (java.lang.Integer and java.lang.Double are in module java.base of loader 'bootstrap')">})
   
   
   ```




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;

Review comment:
       This is the pattern used in all virtual table classes, so like to keep inline with them.




-- 
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] yifan-c commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #1421:
URL: https://github.com/apache/cassandra/pull/1421#discussion_r791335829



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;

Review comment:
       I think it is used to inject the test `CQLMetrics` object for testing. 




-- 
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] yifan-c commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #1421:
URL: https://github.com/apache/cassandra/pull/1421#discussion_r792967114



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       Hi @ekaterinadimitrova2 , good point about the percentiles and thanks for bringing it up. 
   
   I would like to emphasize the importance of **consistency** for the benefit that you all are aware of.
   
   Now, regarding how to place the percentile data, there are existing examples. `CachesTable` is a straightforward one. `TableMetricTables` is another one but more complex. The corresponding columns are created, if we do need to expose the percentile data. Hope it clear things up.
   
   (btw, I am happy to commit this one.)




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       I think the only reason caches was done that way is that we can't make a three-dimensional table




-- 
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] yifan-c commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #1421:
URL: https://github.com/apache/cassandra/pull/1421#discussion_r791335829



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;

Review comment:
       I think it is used to inject the test `CQLMetrics` object for testing. 




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       We just talked with @yifan-c and we agree that the current version is a better and more consistent one actually. Normally we list on the left what we provide value for but indeed metrics are on top as he pointed an example. 

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;

Review comment:
       I see, ok

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       SGTM

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns
+        assertEquals(6, vtsRows.getColumnDefinitions().size());
+
+        AtomicInteger rowCount = new AtomicInteger(0);
+        vtsRows.forEach(r -> {
+            assertEquals(expectedMetrics.preparedStatementsCount.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_COUNT_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsEvicted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EVICTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EXECUTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsRatio.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_RATIO_COL), 0.01);
+            assertEquals(expectedMetrics.regularStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.REGULAR_STATEMENTS_EXECUTED_COL), 0);
+            rowCount.getAndIncrement();
+        });
+
+        // validate - number of rows
+        assertEquals(1, rowCount.get());
+    }
+
+    @Test
+    public void testUsingPrepareStmts() throws Throwable
+    {
+        CQLMetricsTable table = new CQLMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));

Review comment:
       I see, yes, sorry for the noise, I missed that during the first pass

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics

Review comment:
       `I feel comments are like chocolates, like to have as many as we can :-)`
   Wow, how did you know I am a chocoholic :D  Jokes aside, while I agree with you and unfortunately, we lack them often, in this case we just repeat things that are on the next line - like in this case you see it is Virtual Table and the CQL metrics is in the name. I don't mean to be super particular here but I know other engineers on the project are serious on that (talking from experience from my own reviews where I tended to put a lot of comments sometimes :D ) so just saying. 
   Also, in this case these comments are probably not at places that we expect to change a lot but according to the Clean Code rules - if we can make the code self explanatory that is the best (I think you already have that quite well done), better to skip them as in time code changes, comments get pulled away from where they were meant to refer to and it becomes confusing. 




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+
+final class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_COL = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)

Review comment:
       Done




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       Not very sure on the correct/expected format or row/col names for this virtual table, this diff creating below format:
   ```
   
   cqlsh> select * from system_views.cql_metrics ;
   
    name  | prepared_statements_count | prepared_statements_evicted | prepared_statements_executed | prepared_statements_ratio | regular_statements_executed
   -------+---------------------------+-----------------------------+------------------------------+---------------------------+-----------------------------
    value |                         0 |                           0 |                            0 |                         0 |                          16
   
   (1 rows)
   
   ```




-- 
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] frankgh commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics

Review comment:
       maybe turn it into a javadoc instead?




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)

Review comment:
       Without explicit conversion cql select query on this virtual table failing with ClassCastException. Same error for Long to Double as well.
   
   ```
   qlsh> select * from system_views.cql_metrics ;
   NoHostAvailable: ('Unable to complete the operation against any hosts', {<Host: 127.0.0.1:9042 datacenter1>: <Error from server: code=0000 [Server error] message="java.lang.ClassCastException: class java.lang.Integer cannot be cast to class java.lang.Double (java.lang.Integer and java.lang.Double are in module java.base of loader 'bootstrap')">})
   ```
   
   




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics

Review comment:
       I think we can skip this comment as the package and the class are already self-explanatory

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns
+        assertEquals(6, vtsRows.getColumnDefinitions().size());
+
+        AtomicInteger rowCount = new AtomicInteger(0);
+        vtsRows.forEach(r -> {
+            assertEquals(expectedMetrics.preparedStatementsCount.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_COUNT_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsEvicted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EVICTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EXECUTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsRatio.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_RATIO_COL), 0.01);
+            assertEquals(expectedMetrics.regularStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.REGULAR_STATEMENTS_EXECUTED_COL), 0);
+            rowCount.getAndIncrement();
+        });
+
+        // validate - number of rows
+        assertEquals(1, rowCount.get());
+    }
+
+    @Test
+    public void testUsingPrepareStmts() throws Throwable
+    {
+        CQLMetricsTable table = new CQLMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+
+        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        String tbl = createTable(ks, "CREATE TABLE %s (id int PRIMARY KEY, cid int, val text)");
+        Session session = sessionNet();
+
+        // prepare statements
+        String insertCQL = "INSERT INTO " + ks + "." + tbl + " (id, cid, val) VALUES (?, ?, ?)";
+        PreparedStatement preparedInsert = session.prepare(insertCQL);
+
+        String selectCQL = "Select * from " + ks + "." + tbl + " where id = ?";
+        PreparedStatement preparedSelect = session.prepare(selectCQL);
+
+        for (int i = 0; i < 10; i++)
+        {
+            // execute prepared statements

Review comment:
       I think we can skip this comment

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table

Review comment:
       I think we can skip this comment

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics

Review comment:
       I think we can skip this comment 

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns
+        assertEquals(6, vtsRows.getColumnDefinitions().size());
+
+        AtomicInteger rowCount = new AtomicInteger(0);
+        vtsRows.forEach(r -> {
+            assertEquals(expectedMetrics.preparedStatementsCount.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_COUNT_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsEvicted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EVICTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EXECUTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsRatio.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_RATIO_COL), 0.01);
+            assertEquals(expectedMetrics.regularStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.REGULAR_STATEMENTS_EXECUTED_COL), 0);
+            rowCount.getAndIncrement();
+        });
+
+        // validate - number of rows
+        assertEquals(1, rowCount.get());
+    }
+
+    @Test
+    public void testUsingPrepareStmts() throws Throwable
+    {
+        CQLMetricsTable table = new CQLMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));

Review comment:
       I think we can add something like that for clarity and to reduce the repetition of code below, WDYT?
   ```
       @Before
       public void config()
       {
           CQLMetricsTable table = new CQLMetricsTable(KS_NAME);
           VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
       }
   ```

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns
+        assertEquals(6, vtsRows.getColumnDefinitions().size());
+
+        AtomicInteger rowCount = new AtomicInteger(0);
+        vtsRows.forEach(r -> {
+            assertEquals(expectedMetrics.preparedStatementsCount.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_COUNT_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsEvicted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EVICTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EXECUTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsRatio.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_RATIO_COL), 0.01);
+            assertEquals(expectedMetrics.regularStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.REGULAR_STATEMENTS_EXECUTED_COL), 0);
+            rowCount.getAndIncrement();
+        });
+
+        // validate - number of rows
+        assertEquals(1, rowCount.get());
+    }
+
+    @Test
+    public void testUsingPrepareStmts() throws Throwable
+    {
+        CQLMetricsTable table = new CQLMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+
+        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        String tbl = createTable(ks, "CREATE TABLE %s (id int PRIMARY KEY, cid int, val text)");
+        Session session = sessionNet();
+
+        // prepare statements

Review comment:
       I think we can skip this comment

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;

Review comment:
       I don't think we really need `private final CQLMetrics cqlMetrics;`
   I would suggest 
   ```
       @Override
       public DataSet data()
       {
           SimpleDataSet result = new SimpleDataSet(metadata());
           CQLMetrics cqlMetrics = QueryProcessor.metrics;
           addRow(result, cqlMetrics);
   
           return result;
       }
   ```
   
   Then I will make another suggestion in the tests, please follow there :-) 

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name

Review comment:
       I think we can skip this comment

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns
+        assertEquals(6, vtsRows.getColumnDefinitions().size());
+
+        AtomicInteger rowCount = new AtomicInteger(0);
+        vtsRows.forEach(r -> {
+            assertEquals(expectedMetrics.preparedStatementsCount.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_COUNT_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsEvicted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EVICTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_EXECUTED_COL), 0);
+            assertEquals(expectedMetrics.preparedStatementsRatio.getValue(), r.getDouble(CQLMetricsTable.PREPARED_STATEMENTS_RATIO_COL), 0.01);
+            assertEquals(expectedMetrics.regularStatementsExecuted.getCount(), r.getDouble(CQLMetricsTable.REGULAR_STATEMENTS_EXECUTED_COL), 0);
+            rowCount.getAndIncrement();
+        });
+
+        // validate - number of rows

Review comment:
       I think we can skip this comment

##########
File path: test/unit/org/apache/cassandra/db/virtual/CQLMetricsTableTest.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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 java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableList;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.BeforeClass;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.metrics.CQLMetrics;
+
+public class CQLMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    private void queryAndValidateMetrics(CQLMetrics expectedMetrics) throws Throwable
+    {
+        String getMetricsQuery = "SELECT * FROM " + KS_NAME + "." + CQLMetricsTable.TABLE_NAME;
+        ResultSet vtsRows = executeNet(getMetricsQuery);
+
+        // validate - number of columns

Review comment:
       I think we can skip this comment

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       I am thinking maybe of `Metric`? (I have to warn you I am bad with names :-) )

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       We can also invert to table with two columns and 5 rows. I think this would be the format most tables follow actually

##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       I am thinking maybe of `Metric`? (I have to warn you I am bad with naming things :-) )




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";

Review comment:
       Yes, not having name col makes sense and as other columns are self explanatory




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: doc/modules/cassandra/pages/new/virtualtables.adoc
##########
@@ -74,6 +74,8 @@ recent_hit_rate_per_second, recent_request_rate_per_second, request_count, and s
 
 |coordinator_write_latency |Records counts, keyspace_name, table_name, max, median, and per_second for coordinator writes.
 
+|cql_metrcs |Lists metrics for CQL prepared statements.

Review comment:
       Also, there is a typo - should be `cql_metrics`




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: doc/modules/cassandra/pages/new/virtualtables.adoc
##########
@@ -74,6 +74,8 @@ recent_hit_rate_per_second, recent_request_rate_per_second, request_count, and s
 
 |coordinator_write_latency |Records counts, keyspace_name, table_name, max, median, and per_second for coordinator writes.
 
+|cql_metrcs |Lists metrics for CQL prepared statements.

Review comment:
       Done




-- 
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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       I just looked again in the tables and more specifically in the one that is about to get committed in CASSANDRA-17224 and reminded myself that one was the reason for me to think about the inversion. I think that one makes total sense to 
   be in that format and gives a direction for future metrics tables.. So I still feel we need to invert. What do others think? @yifan-c , please, consider the batch_metrics table below:
   ```
     name                          | max | p50th | p999th | p99th
   -------------------------------+-----+-------+--------+-------
     partitions_per_counter_batch |   0 |     0 |      0 |     0
      partitions_per_logged_batch |   0 |     0 |      0 |     0
    partitions_per_unlogged_batch |   0 |     0 |      0 |     0
   ```




-- 
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] skoppu22 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: doc/modules/cassandra/pages/new/virtualtables.adoc
##########
@@ -99,6 +99,9 @@ recent_hit_rate_per_second, recent_request_rate_per_second, request_count, and s
 |thread_pools |Lists metrics for each thread pool.
 
 |tombstones_per_read |Records counts, keyspace_name, tablek_name, max, and median for tombstones.
+
+|cql_metrcs |Lists metrics for CQL prepared statements.

Review comment:
       https://cassandra.apache.org/doc/latest/cassandra/operating/metrics.html#:~:text=org.apache.cassandra.metrics%3Atype%3DCQL,that%20are%20prepared%20vs%20unprepared doc mentions as cql prepared statements metric, so trying to be inline with 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.

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] ekaterinadimitrova2 commented on a change in pull request #1421: CASSANDRA-17224: Virtual table for exposing CQL metrics

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



##########
File path: src/java/org/apache/cassandra/db/virtual/CQLMetricsTable.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.CQLMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.cql3.QueryProcessor;
+
+// Virtual table for CQL metrics
+public class CQLMetricsTable extends AbstractVirtualTable
+{
+    public static final String TABLE_NAME = "cql_metrics";
+    public static final String PREPARED_STATEMENTS_COUNT_COL = "prepared_statements_count";
+    public static final String PREPARED_STATEMENTS_EVICTED_COL = "prepared_statements_evicted";
+    public static final String PREPARED_STATEMENTS_EXECUTED_COL = "prepared_statements_executed";
+    public static final String PREPARED_STATEMENTS_RATIO_COL = "prepared_statements_ratio";
+    public static final String REGULAR_STATEMENTS_EXECUTED_COL = "regular_statements_executed";
+    public static final String NAME_COL = "name";
+    public static final String VALUE_ROW = "value";
+
+    private final CQLMetrics cqlMetrics;
+
+    // Default constructor references query processor metrics
+    CQLMetricsTable(String keyspace)
+    {
+        this(keyspace, QueryProcessor.metrics);
+    }
+
+    // For dependency injection
+    @VisibleForTesting
+    CQLMetricsTable(String keyspace, CQLMetrics cqlMetrics)
+    {
+        // create virtual table with this name
+        super(TableMetadata.builder(keyspace, TABLE_NAME)
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(NAME_COL, UTF8Type.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_COUNT_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EVICTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .addRegularColumn(PREPARED_STATEMENTS_RATIO_COL, DoubleType.instance)
+                           .addRegularColumn(REGULAR_STATEMENTS_EXECUTED_COL, DoubleType.instance)
+                           .build());
+        this.cqlMetrics = cqlMetrics;
+    }
+
+    @Override
+    public DataSet data()
+    {
+        // populate metrics in the virtual table
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        addRow(result, VALUE_ROW, cqlMetrics);
+
+        return result;
+    }
+
+    private void addRow(SimpleDataSet dataSet, String name, CQLMetrics cqlMetrics)
+    {
+        dataSet.row(name)
+               .column(PREPARED_STATEMENTS_COUNT_COL, Double.valueOf(cqlMetrics.preparedStatementsCount.getValue()))

Review comment:
       That is what confused me honestly. As we said percentiles in columns but you are against it. Do you suggest all of them in a long header and one row for values?
   This is absolutely not needed and user friendly IMHO :( 
   I think these two can be done in the proposed format and it will be way more clear and still consistent. 
   We can have the format for set of metrics (caches) and the user friendly format for pure one time metric appearance as suggested for batch_metrics and this one. If we follow this rule across new metrics tables, this shouldn't be inconsistent, WDYT?
   




-- 
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