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 2021/12/23 19:23:42 UTC

[GitHub] [cassandra] burmanm opened a new pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

burmanm opened a new pull request #1372:
URL: https://github.com/apache/cassandra/pull/1372


   Allows  to retrieve unlogged,, logged and counter batch metrics from a VT. 
   
   I'm here for the (rare, collectible) t-shirt of course ;)


-- 
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] blerer commented on a change in pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Snapshot;
+import com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.metrics.BatchMetrics;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+
+public class BatchMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private BatchMetricsTable table;
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void config()
+    {
+        table = new BatchMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+    }
+
+    @Test
+    void testSelectAll() throws Throwable
+    {
+        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        String tbl = createTable(ks, "CREATE TABLE %s (id int primary key, val int)");
+
+        String batch = format("BEGIN BATCH " +
+                              "INSERT INTO %s.%s (id, val) VALUES (0, 0) USING TTL %d; " +
+                              "INSERT INTO %s.%s (id, val) VALUES (1, 1) USING TTL %d; " +
+                              "APPLY BATCH;",
+                              ks, tbl, 1,
+                              ks, tbl, 1);
+        execute(batch);
+
+        BatchMetrics metrics = BatchStatement.metrics;
+        ResultSet result = executeNet("SELECT * FROM vts.batch_metrics");
+        assertEquals(5, result.getColumnDefinitions().size());
+        AtomicInteger rowCount = new AtomicInteger(0);
+        result.forEach(r -> {
+            Snapshot snapshot;
+            if(r.getString("name").equals("partitions_per_logged_batch")) {
+                snapshot = metrics.partitionsPerLoggedBatch.getSnapshot();
+            } else if(r.getString("name").equals("partitions_per_unlogged_batch")) {
+                snapshot = metrics.partitionsPerUnloggedBatch.getSnapshot();
+            } else {
+                snapshot = metrics.partitionsPerCounterBatch.getSnapshot();
+            }
+            assertEquals(r.getDouble("p50"), snapshot.getMedian(), 1.0);

Review comment:
       nit: I would switch the arguments as the expected value is the snapshot value and the actual value is what is returned by the resultset.

##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Snapshot;
+import com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.metrics.BatchMetrics;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+
+public class BatchMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private BatchMetricsTable table;
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void config()
+    {
+        table = new BatchMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+    }
+
+    @Test
+    void testSelectAll() throws Throwable
+    {
+        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        String tbl = createTable(ks, "CREATE TABLE %s (id int primary key, val int)");
+
+        String batch = format("BEGIN BATCH " +
+                              "INSERT INTO %s.%s (id, val) VALUES (0, 0) USING TTL %d; " +
+                              "INSERT INTO %s.%s (id, val) VALUES (1, 1) USING TTL %d; " +
+                              "APPLY BATCH;",
+                              ks, tbl, 1,
+                              ks, tbl, 1);
+        execute(batch);
+
+        BatchMetrics metrics = BatchStatement.metrics;
+        ResultSet result = executeNet("SELECT * FROM vts.batch_metrics");
+        assertEquals(5, result.getColumnDefinitions().size());
+        AtomicInteger rowCount = new AtomicInteger(0);
+        result.forEach(r -> {
+            Snapshot snapshot;
+            if(r.getString("name").equals("partitions_per_logged_batch")) {
+                snapshot = metrics.partitionsPerLoggedBatch.getSnapshot();
+            } else if(r.getString("name").equals("partitions_per_unlogged_batch")) {
+                snapshot = metrics.partitionsPerUnloggedBatch.getSnapshot();
+            } else {
+                snapshot = metrics.partitionsPerCounterBatch.getSnapshot();
+            }

Review comment:
       nit: It might make the code easier to read if we extract the logic in a separate method like:
   ```
   private Histogram getExpectedHistogram(BatchMetrics metrics, String name)
   {
       if ("partitions_per_logged_batch".equals(name))
           return metrics.partitionsPerLoggedBatch;
   
       if ("partitions_per_unlogged_batch".equals(name))
           return metrics.partitionsPerUnloggedBatch;
   
       return metrics.partitionsPerCounterBatch;
   } 
   ```




-- 
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] burmanm commented on a change in pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Snapshot;
+import com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.metrics.BatchMetrics;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+
+public class BatchMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private BatchMetricsTable table;
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void config()
+    {
+        table = new BatchMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+    }
+
+    @Test
+    void testSelectAll() throws Throwable
+    {
+        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        String tbl = createTable(ks, "CREATE TABLE %s (id int primary key, val int)");
+
+        String batch = format("BEGIN BATCH " +
+                              "INSERT INTO %s.%s (id, val) VALUES (0, 0) USING TTL %d; " +
+                              "INSERT INTO %s.%s (id, val) VALUES (1, 1) USING TTL %d; " +
+                              "APPLY BATCH;",
+                              ks, tbl, 1,
+                              ks, tbl, 1);
+        execute(batch);
+
+        BatchMetrics metrics = BatchStatement.metrics;
+        ResultSet result = executeNet("SELECT * FROM vts.batch_metrics");
+        assertEquals(5, result.getColumnDefinitions().size());
+        AtomicInteger rowCount = new AtomicInteger(0);
+        result.forEach(r -> {
+            Snapshot snapshot;
+            if(r.getString("name").equals("partitions_per_logged_batch")) {
+                snapshot = metrics.partitionsPerLoggedBatch.getSnapshot();
+            } else if(r.getString("name").equals("partitions_per_unlogged_batch")) {
+                snapshot = metrics.partitionsPerUnloggedBatch.getSnapshot();
+            } else {
+                snapshot = metrics.partitionsPerCounterBatch.getSnapshot();
+            }
+            assertEquals(r.getDouble("p50"), snapshot.getMedian(), 1.0);

Review comment:
       Yes, good catch (wonders of different test-libraries having these in different orders)




-- 
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] burmanm commented on a change in pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Snapshot;
+import com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.metrics.BatchMetrics;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+
+public class BatchMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private BatchMetricsTable table;
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void config()
+    {
+        table = new BatchMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+    }
+
+    @Test
+    void testSelectAll() throws Throwable
+    {
+        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        String tbl = createTable(ks, "CREATE TABLE %s (id int primary key, val int)");
+
+        String batch = format("BEGIN BATCH " +
+                              "INSERT INTO %s.%s (id, val) VALUES (0, 0) USING TTL %d; " +
+                              "INSERT INTO %s.%s (id, val) VALUES (1, 1) USING TTL %d; " +
+                              "APPLY BATCH;",
+                              ks, tbl, 1,
+                              ks, tbl, 1);
+        execute(batch);
+
+        BatchMetrics metrics = BatchStatement.metrics;
+        ResultSet result = executeNet("SELECT * FROM vts.batch_metrics");
+        assertEquals(5, result.getColumnDefinitions().size());
+        AtomicInteger rowCount = new AtomicInteger(0);
+        result.forEach(r -> {
+            Snapshot snapshot;
+            if(r.getString("name").equals("partitions_per_logged_batch")) {
+                snapshot = metrics.partitionsPerLoggedBatch.getSnapshot();
+            } else if(r.getString("name").equals("partitions_per_unlogged_batch")) {
+                snapshot = metrics.partitionsPerUnloggedBatch.getSnapshot();
+            } else {
+                snapshot = metrics.partitionsPerCounterBatch.getSnapshot();
+            }

Review comment:
       I'm too used to go fmt doing everything automatically on save, I'll fix the indentation (I wonder if IntelliJ has something similar when saving to auto-reformat code..)




-- 
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 #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: src/java/org/apache/cassandra/db/virtual/BatchMetricsTable.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.codahale.metrics.Snapshot;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.BatchMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+
+public class BatchMetricsTable extends AbstractVirtualTable
+{
+
+    private static final String PARTITIONS_PER_LOGGED_BATCH = "partitions_per_logged_batch";
+    private static final String PARTITIONS_PER_UNLOGGED_BATCH = "partitions_per_unlogged_batch";
+    private static final String PARTITIONS_PER_COUNTER_BATCH = "partitions_per_counter_batch";
+    private final static String P50 = "p50th";
+    private final static String P99 = "p99th";
+    private final static String P999 = "p999th";

Review comment:
       In my opinion `TableMetricTables` was mentioned to give an idea where to look in order to understand the requirement. In that sense I think p999 is fine here




-- 
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] azotcsit commented on a change in pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Snapshot;
+import com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.metrics.BatchMetrics;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+
+public class BatchMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void config()
+    {
+        BatchMetricsTable table = new BatchMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+    }
+
+    @Test
+    public void testSelectAll() throws Throwable
+    {
+        BatchMetrics metrics = BatchStatement.metrics;
+
+        for (int i = 0; i < 10; i++)
+        {
+            metrics.partitionsPerLoggedBatch.update(i);
+            metrics.partitionsPerUnloggedBatch.update(i + 10);
+            metrics.partitionsPerCounterBatch.update(i * 10);
+        }
+
+        ResultSet result = executeNet(format("SELECT * FROM %s.batch_metrics", KS_NAME));
+        assertEquals(5, result.getColumnDefinitions().size());
+        AtomicInteger rowCount = new AtomicInteger(0);
+        result.forEach(r -> {
+            Snapshot snapshot = getExpectedHistogramSnapshot(metrics, r.getString("name"));
+            assertEquals(snapshot.getMedian(), r.getDouble("p50th"), 0.0);
+            assertEquals(snapshot.get99thPercentile(), r.getDouble("p99th"), 0.0);

Review comment:
       Shouldn't we have `p999th` and `max` here tested as well?

##########
File path: src/java/org/apache/cassandra/db/virtual/BatchMetricsTable.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.codahale.metrics.Snapshot;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.BatchMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+
+public class BatchMetricsTable extends AbstractVirtualTable

Review comment:
       It would be nice to mention this new table in the documentation: https://github.com/apache/cassandra/blob/trunk/doc/modules/cassandra/pages/new/virtualtables.adoc

##########
File path: src/java/org/apache/cassandra/db/virtual/BatchMetricsTable.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.codahale.metrics.Snapshot;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.BatchMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+
+public class BatchMetricsTable extends AbstractVirtualTable
+{
+
+    private static final String PARTITIONS_PER_LOGGED_BATCH = "partitions_per_logged_batch";
+    private static final String PARTITIONS_PER_UNLOGGED_BATCH = "partitions_per_unlogged_batch";
+    private static final String PARTITIONS_PER_COUNTER_BATCH = "partitions_per_counter_batch";
+    private final static String P50 = "p50th";
+    private final static String P99 = "p99th";
+    private final static String P999 = "p999th";

Review comment:
       The ticket description mentions `TableMetricTables` as a reference. p999 is not used there. Do we really want to expose it?
   
   @burmanm @ekaterinadimitrova2 WDYT?

##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Snapshot;
+import com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.metrics.BatchMetrics;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+
+public class BatchMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void config()
+    {
+        BatchMetricsTable table = new BatchMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+    }
+
+    @Test
+    public void testSelectAll() throws Throwable
+    {
+        BatchMetrics metrics = BatchStatement.metrics;
+
+        for (int i = 0; i < 10; i++)
+        {
+            metrics.partitionsPerLoggedBatch.update(i);
+            metrics.partitionsPerUnloggedBatch.update(i + 10);
+            metrics.partitionsPerCounterBatch.update(i * 10);
+        }
+
+        ResultSet result = executeNet(format("SELECT * FROM %s.batch_metrics", KS_NAME));
+        assertEquals(5, result.getColumnDefinitions().size());
+        AtomicInteger rowCount = new AtomicInteger(0);
+        result.forEach(r -> {
+            Snapshot snapshot = getExpectedHistogramSnapshot(metrics, r.getString("name"));
+            assertEquals(snapshot.getMedian(), r.getDouble("p50th"), 0.0);

Review comment:
       My expectation was to see smth non zero here because you call `update` earlier in the test. What am I missing?

##########
File path: src/java/org/apache/cassandra/db/virtual/BatchMetricsTable.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.codahale.metrics.Snapshot;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.metrics.BatchMetrics;
+import org.apache.cassandra.schema.TableMetadata;
+
+public class BatchMetricsTable extends AbstractVirtualTable
+{
+
+    private static final String PARTITIONS_PER_LOGGED_BATCH = "partitions_per_logged_batch";
+    private static final String PARTITIONS_PER_UNLOGGED_BATCH = "partitions_per_unlogged_batch";
+    private static final String PARTITIONS_PER_COUNTER_BATCH = "partitions_per_counter_batch";
+    private final static String P50 = "p50th";
+    private final static String P99 = "p99th";
+    private final static String P999 = "p999th";
+    private final static String MAX = "max";
+
+    BatchMetricsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "batch_metrics")
+                           .kind(TableMetadata.Kind.VIRTUAL)

Review comment:
       Let's add a table description. Example: https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/virtual/RolesCacheKeysTable.java#L33
   
   Smth like: "metrics on batch 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] burmanm commented on a change in pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Snapshot;
+import com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.metrics.BatchMetrics;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+
+public class BatchMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void config()
+    {
+        BatchMetricsTable table = new BatchMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+    }
+
+    @Test
+    public void testSelectAll() throws Throwable
+    {
+        BatchMetrics metrics = BatchStatement.metrics;
+
+        for (int i = 0; i < 10; i++)
+        {
+            metrics.partitionsPerLoggedBatch.update(i);
+            metrics.partitionsPerUnloggedBatch.update(i + 10);
+            metrics.partitionsPerCounterBatch.update(i * 10);
+        }
+
+        ResultSet result = executeNet(format("SELECT * FROM %s.batch_metrics", KS_NAME));
+        assertEquals(5, result.getColumnDefinitions().size());
+        AtomicInteger rowCount = new AtomicInteger(0);
+        result.forEach(r -> {
+            Snapshot snapshot = getExpectedHistogramSnapshot(metrics, r.getString("name"));
+            assertEquals(snapshot.getMedian(), r.getDouble("p50th"), 0.0);

Review comment:
       The 0.0 is a floating point fluctuation. I guess it could be 0.1 also to ensure it doesn't fail to some rounding issue.




-- 
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] blerer closed pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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


   


-- 
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 pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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


   Are we ready to wrap up this one? I can run CI when 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] azotcsit commented on a change in pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Snapshot;
+import com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.metrics.BatchMetrics;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+
+public class BatchMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void config()
+    {
+        BatchMetricsTable table = new BatchMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+    }
+
+    @Test
+    public void testSelectAll() throws Throwable
+    {
+        BatchMetrics metrics = BatchStatement.metrics;
+
+        for (int i = 0; i < 10; i++)
+        {
+            metrics.partitionsPerLoggedBatch.update(i);
+            metrics.partitionsPerUnloggedBatch.update(i + 10);
+            metrics.partitionsPerCounterBatch.update(i * 10);
+        }
+
+        ResultSet result = executeNet(format("SELECT * FROM %s.batch_metrics", KS_NAME));
+        assertEquals(5, result.getColumnDefinitions().size());
+        AtomicInteger rowCount = new AtomicInteger(0);
+        result.forEach(r -> {
+            Snapshot snapshot = getExpectedHistogramSnapshot(metrics, r.getString("name"));
+            assertEquals(snapshot.getMedian(), r.getDouble("p50th"), 0.0);

Review comment:
       Oh sorry, I just missed that it fluctuation and not expected result. :+1: 




-- 
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] blerer commented on pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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


   Patch merged manually. 


-- 
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 #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -49,42 +47,45 @@ public static void setUpClass()
     @Before
     public void config()
     {
-        table = new BatchMetricsTable(KS_NAME);
+        BatchMetricsTable table = new BatchMetricsTable(KS_NAME);
         VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
     }
 
     @Test
-    void testSelectAll() throws Throwable
+    public void testSelectAll() throws Throwable
     {
-        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
-        String tbl = createTable(ks, "CREATE TABLE %s (id int primary key, val int)");
+        BatchMetrics metrics = BatchStatement.metrics;
 
-        String batch = format("BEGIN BATCH " +
-                              "INSERT INTO %s.%s (id, val) VALUES (0, 0) USING TTL %d; " +
-                              "INSERT INTO %s.%s (id, val) VALUES (1, 1) USING TTL %d; " +
-                              "APPLY BATCH;",
-                              ks, tbl, 1,
-                              ks, tbl, 1);
-        execute(batch);
+        for (int i = 0; i < 10; i++)
+        {
+            metrics.partitionsPerLoggedBatch.update(i);
+            metrics.partitionsPerUnloggedBatch.update(i + 10);
+            metrics.partitionsPerCounterBatch.update(i * 10);
+        }
 
-        BatchMetrics metrics = BatchStatement.metrics;
-        ResultSet result = executeNet("SELECT * FROM vts.batch_metrics");
+        ResultSet result = executeNet(format("SELECT * FROM %s.batch_metrics", KS_NAME));
         assertEquals(5, result.getColumnDefinitions().size());
         AtomicInteger rowCount = new AtomicInteger(0);
         result.forEach(r -> {
-            Snapshot snapshot;
-            if(r.getString("name").equals("partitions_per_logged_batch")) {
-                snapshot = metrics.partitionsPerLoggedBatch.getSnapshot();
-            } else if(r.getString("name").equals("partitions_per_unlogged_batch")) {
-                snapshot = metrics.partitionsPerUnloggedBatch.getSnapshot();
-            } else {
-                snapshot = metrics.partitionsPerCounterBatch.getSnapshot();
-            }
-            assertEquals(r.getDouble("p50"), snapshot.getMedian(), 1.0);
-            assertEquals(r.getDouble("p99"), snapshot.get99thPercentile(), 1.0);
+            Snapshot snapshot = getExpectedHistogramSnapshot(metrics, r.getString("name"));
+            assertEquals(snapshot.getMedian(), r.getDouble("p50th"), 0.0);
+            assertEquals(snapshot.get99thPercentile(), r.getDouble("p99th"), 0.0);
             rowCount.addAndGet(1);
         });
 
         assertEquals(3, rowCount.get());
     }
+
+    private Snapshot getExpectedHistogramSnapshot(BatchMetrics metrics, String tableName)
+    {
+        if ("partitions_per_logged_batch".equals(tableName))
+        {
+            return metrics.partitionsPerLoggedBatch.getSnapshot();
+        }
+        else if ("partitions_per_unlogged_batch".equals(tableName))

Review comment:
       I would probably skip the else :-) Just it reads better something like that:
   ```
           if ("partitions_per_logged_batch".equals(tableName))
               return metrics.partitionsPerLoggedBatch.getSnapshot();
           
           
           if ("partitions_per_unlogged_batch".equals(tableName))
               return metrics.partitionsPerUnloggedBatch.getSnapshot();
   
           return metrics.partitionsPerCounterBatch.getSnapshot();
   ```




-- 
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] burmanm commented on a change in pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -49,42 +47,45 @@ public static void setUpClass()
     @Before
     public void config()
     {
-        table = new BatchMetricsTable(KS_NAME);
+        BatchMetricsTable table = new BatchMetricsTable(KS_NAME);
         VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
     }
 
     @Test
-    void testSelectAll() throws Throwable
+    public void testSelectAll() throws Throwable
     {
-        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
-        String tbl = createTable(ks, "CREATE TABLE %s (id int primary key, val int)");
+        BatchMetrics metrics = BatchStatement.metrics;
 
-        String batch = format("BEGIN BATCH " +
-                              "INSERT INTO %s.%s (id, val) VALUES (0, 0) USING TTL %d; " +
-                              "INSERT INTO %s.%s (id, val) VALUES (1, 1) USING TTL %d; " +
-                              "APPLY BATCH;",
-                              ks, tbl, 1,
-                              ks, tbl, 1);
-        execute(batch);
+        for (int i = 0; i < 10; i++)
+        {
+            metrics.partitionsPerLoggedBatch.update(i);
+            metrics.partitionsPerUnloggedBatch.update(i + 10);
+            metrics.partitionsPerCounterBatch.update(i * 10);
+        }
 
-        BatchMetrics metrics = BatchStatement.metrics;
-        ResultSet result = executeNet("SELECT * FROM vts.batch_metrics");
+        ResultSet result = executeNet(format("SELECT * FROM %s.batch_metrics", KS_NAME));
         assertEquals(5, result.getColumnDefinitions().size());
         AtomicInteger rowCount = new AtomicInteger(0);
         result.forEach(r -> {
-            Snapshot snapshot;
-            if(r.getString("name").equals("partitions_per_logged_batch")) {
-                snapshot = metrics.partitionsPerLoggedBatch.getSnapshot();
-            } else if(r.getString("name").equals("partitions_per_unlogged_batch")) {
-                snapshot = metrics.partitionsPerUnloggedBatch.getSnapshot();
-            } else {
-                snapshot = metrics.partitionsPerCounterBatch.getSnapshot();
-            }
-            assertEquals(r.getDouble("p50"), snapshot.getMedian(), 1.0);
-            assertEquals(r.getDouble("p99"), snapshot.get99thPercentile(), 1.0);
+            Snapshot snapshot = getExpectedHistogramSnapshot(metrics, r.getString("name"));
+            assertEquals(snapshot.getMedian(), r.getDouble("p50th"), 0.0);
+            assertEquals(snapshot.get99thPercentile(), r.getDouble("p99th"), 0.0);
             rowCount.addAndGet(1);
         });
 
         assertEquals(3, rowCount.get());
     }
+
+    private Snapshot getExpectedHistogramSnapshot(BatchMetrics metrics, String tableName)
+    {
+        if ("partitions_per_logged_batch".equals(tableName))
+        {
+            return metrics.partitionsPerLoggedBatch.getSnapshot();
+        }
+        else if ("partitions_per_unlogged_batch".equals(tableName))

Review comment:
       I see both approaches used in the code base, I'm personally on the "else" camp usually - to me it makes it easier to follow the flow, with the added bonus of being able to hide the whole conditional flow in the editor with a single click (does not matter in this case, but in general). 




-- 
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 #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -49,42 +47,45 @@ public static void setUpClass()
     @Before
     public void config()
     {
-        table = new BatchMetricsTable(KS_NAME);
+        BatchMetricsTable table = new BatchMetricsTable(KS_NAME);
         VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
     }
 
     @Test
-    void testSelectAll() throws Throwable
+    public void testSelectAll() throws Throwable
     {
-        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
-        String tbl = createTable(ks, "CREATE TABLE %s (id int primary key, val int)");
+        BatchMetrics metrics = BatchStatement.metrics;
 
-        String batch = format("BEGIN BATCH " +
-                              "INSERT INTO %s.%s (id, val) VALUES (0, 0) USING TTL %d; " +
-                              "INSERT INTO %s.%s (id, val) VALUES (1, 1) USING TTL %d; " +
-                              "APPLY BATCH;",
-                              ks, tbl, 1,
-                              ks, tbl, 1);
-        execute(batch);
+        for (int i = 0; i < 10; i++)
+        {
+            metrics.partitionsPerLoggedBatch.update(i);
+            metrics.partitionsPerUnloggedBatch.update(i + 10);
+            metrics.partitionsPerCounterBatch.update(i * 10);
+        }
 
-        BatchMetrics metrics = BatchStatement.metrics;
-        ResultSet result = executeNet("SELECT * FROM vts.batch_metrics");
+        ResultSet result = executeNet(format("SELECT * FROM %s.batch_metrics", KS_NAME));
         assertEquals(5, result.getColumnDefinitions().size());
         AtomicInteger rowCount = new AtomicInteger(0);
         result.forEach(r -> {
-            Snapshot snapshot;
-            if(r.getString("name").equals("partitions_per_logged_batch")) {
-                snapshot = metrics.partitionsPerLoggedBatch.getSnapshot();
-            } else if(r.getString("name").equals("partitions_per_unlogged_batch")) {
-                snapshot = metrics.partitionsPerUnloggedBatch.getSnapshot();
-            } else {
-                snapshot = metrics.partitionsPerCounterBatch.getSnapshot();
-            }
-            assertEquals(r.getDouble("p50"), snapshot.getMedian(), 1.0);
-            assertEquals(r.getDouble("p99"), snapshot.get99thPercentile(), 1.0);
+            Snapshot snapshot = getExpectedHistogramSnapshot(metrics, r.getString("name"));
+            assertEquals(snapshot.getMedian(), r.getDouble("p50th"), 0.0);
+            assertEquals(snapshot.get99thPercentile(), r.getDouble("p99th"), 0.0);
             rowCount.addAndGet(1);
         });
 
         assertEquals(3, rowCount.get());
     }
+
+    private Snapshot getExpectedHistogramSnapshot(BatchMetrics metrics, String tableName)
+    {
+        if ("partitions_per_logged_batch".equals(tableName))
+        {
+            return metrics.partitionsPerLoggedBatch.getSnapshot();
+        }
+        else if ("partitions_per_unlogged_batch".equals(tableName))

Review comment:
       I would probably skip the else :-) Just it reads better something like that:
   ```
   if ("partitions_per_logged_batch".equals(tableName))
           {
               return metrics.partitionsPerLoggedBatch.getSnapshot();
           }
           
           if ("partitions_per_unlogged_batch".equals(tableName))
           {
               return metrics.partitionsPerUnloggedBatch.getSnapshot();
           }
   
           return metrics.partitionsPerCounterBatch.getSnapshot();
   ```




-- 
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 #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Histogram;

Review comment:
       unused import

##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Histogram;

Review comment:
       ```suggestion
   
   ```

##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Snapshot;
+import com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.metrics.BatchMetrics;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+
+public class BatchMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private BatchMetricsTable table;
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void config()
+    {
+        table = new BatchMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+    }
+
+    @Test
+    void testSelectAll() throws Throwable

Review comment:
       I suspect you accidentally deleted the public accessor, otherwise you can't run the test. I noticed It throws a class cast exception though. Can you, please check it? Thanks. Please let me know if you need help

##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.Snapshot;
+import com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.metrics.BatchMetrics;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+
+public class BatchMetricsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private BatchMetricsTable table;
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void config()
+    {
+        table = new BatchMetricsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+    }
+
+    @Test
+    void testSelectAll() throws Throwable
+    {
+        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        String tbl = createTable(ks, "CREATE TABLE %s (id int primary key, val int)");
+
+        String batch = format("BEGIN BATCH " +
+                              "INSERT INTO %s.%s (id, val) VALUES (0, 0) USING TTL %d; " +
+                              "INSERT INTO %s.%s (id, val) VALUES (1, 1) USING TTL %d; " +
+                              "APPLY BATCH;",
+                              ks, tbl, 1,
+                              ks, tbl, 1);
+        execute(batch);
+
+        BatchMetrics metrics = BatchStatement.metrics;
+        ResultSet result = executeNet("SELECT * FROM vts.batch_metrics");
+        assertEquals(5, result.getColumnDefinitions().size());
+        AtomicInteger rowCount = new AtomicInteger(0);
+        result.forEach(r -> {
+            Snapshot snapshot;
+            if(r.getString("name").equals("partitions_per_logged_batch")) {
+                snapshot = metrics.partitionsPerLoggedBatch.getSnapshot();
+            } else if(r.getString("name").equals("partitions_per_unlogged_batch")) {
+                snapshot = metrics.partitionsPerUnloggedBatch.getSnapshot();
+            } else {
+                snapshot = metrics.partitionsPerCounterBatch.getSnapshot();
+            }

Review comment:
       +1 on a new method. And I wanted to mention that the project follows some conventions you might want to get acquainted with - https://cassandra.apache.org/_/development/code_style.html (I noticed `{` are not on new lines)




-- 
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] blerer commented on a change in pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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



##########
File path: test/unit/org/apache/cassandra/db/virtual/BatchMetricsTableTest.java
##########
@@ -49,42 +47,45 @@ public static void setUpClass()
     @Before
     public void config()
     {
-        table = new BatchMetricsTable(KS_NAME);
+        BatchMetricsTable table = new BatchMetricsTable(KS_NAME);
         VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
     }
 
     @Test
-    void testSelectAll() throws Throwable
+    public void testSelectAll() throws Throwable
     {
-        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication={ 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
-        String tbl = createTable(ks, "CREATE TABLE %s (id int primary key, val int)");
+        BatchMetrics metrics = BatchStatement.metrics;
 
-        String batch = format("BEGIN BATCH " +
-                              "INSERT INTO %s.%s (id, val) VALUES (0, 0) USING TTL %d; " +
-                              "INSERT INTO %s.%s (id, val) VALUES (1, 1) USING TTL %d; " +
-                              "APPLY BATCH;",
-                              ks, tbl, 1,
-                              ks, tbl, 1);
-        execute(batch);
+        for (int i = 0; i < 10; i++)
+        {
+            metrics.partitionsPerLoggedBatch.update(i);
+            metrics.partitionsPerUnloggedBatch.update(i + 10);
+            metrics.partitionsPerCounterBatch.update(i * 10);
+        }
 
-        BatchMetrics metrics = BatchStatement.metrics;
-        ResultSet result = executeNet("SELECT * FROM vts.batch_metrics");
+        ResultSet result = executeNet(format("SELECT * FROM %s.batch_metrics", KS_NAME));
         assertEquals(5, result.getColumnDefinitions().size());
         AtomicInteger rowCount = new AtomicInteger(0);
         result.forEach(r -> {
-            Snapshot snapshot;
-            if(r.getString("name").equals("partitions_per_logged_batch")) {
-                snapshot = metrics.partitionsPerLoggedBatch.getSnapshot();
-            } else if(r.getString("name").equals("partitions_per_unlogged_batch")) {
-                snapshot = metrics.partitionsPerUnloggedBatch.getSnapshot();
-            } else {
-                snapshot = metrics.partitionsPerCounterBatch.getSnapshot();
-            }
-            assertEquals(r.getDouble("p50"), snapshot.getMedian(), 1.0);
-            assertEquals(r.getDouble("p99"), snapshot.get99thPercentile(), 1.0);
+            Snapshot snapshot = getExpectedHistogramSnapshot(metrics, r.getString("name"));
+            assertEquals(snapshot.getMedian(), r.getDouble("p50th"), 0.0);
+            assertEquals(snapshot.get99thPercentile(), r.getDouble("p99th"), 0.0);
             rowCount.addAndGet(1);
         });
 
         assertEquals(3, rowCount.get());
     }
+
+    private Snapshot getExpectedHistogramSnapshot(BatchMetrics metrics, String tableName)
+    {
+        if ("partitions_per_logged_batch".equals(tableName))
+        {
+            return metrics.partitionsPerLoggedBatch.getSnapshot();
+        }
+        else if ("partitions_per_unlogged_batch".equals(tableName))

Review comment:
       I also prefer avoiding any unecessary code as it can only create confusion and for simple condition braces are normally skipped.
   




-- 
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 pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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


   In the meantime the docs migration was also finished. So we need to consider also updating the Virtual Tables 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] ekaterinadimitrova2 commented on pull request #1372: CASSANDRA-17225 Add batch_metrics virtual table

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


   Are we ready to wrap up this one? I can run CI when 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