Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

CASSANDRA-17225 Add batch_metrics virtual table #1372

Closed
wants to merge 2 commits into from

Conversation

burmanm
Copy link
Contributor

@burmanm burmanm commented Dec 23, 2021

Allows to retrieve unlogged,, logged and counter batch metrics from a VT.

I'm here for the (rare, collectible) t-shirt of course ;)

Copy link
Contributor

@blerer blerer left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The patch looks good :-) I only have 2 suggestions.

Comment on lines 76 to 82
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();
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

} else {
snapshot = metrics.partitionsPerCounterBatch.getSnapshot();
}
assertEquals(r.getDouble("p50"), snapshot.getMedian(), 1.0);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Comment on lines 76 to 82
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();
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

}

@Test
void testSelectAll() throws Throwable
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

import org.junit.BeforeClass;
import org.junit.Test;

import com.codahale.metrics.Histogram;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
import com.codahale.metrics.Histogram;

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

unused import

…shot in the test to be a separate function, fix indentation and MAX() uses LongType instead of DoubleType
{
return metrics.partitionsPerLoggedBatch.getSnapshot();
}
else if ("partitions_per_unlogged_batch".equals(tableName))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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();

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor

@ekaterinadimitrova2 ekaterinadimitrova2 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

One formatting comment, otherwise the latest changes LGTM. +1, we need to add NEWS.txt and CHANGES.txt, this can be done also on commit IMHO

Copy link
Contributor

@blerer blerer left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The patch looks good to me

{
return metrics.partitionsPerLoggedBatch.getSnapshot();
}
else if ("partitions_per_unlogged_batch".equals(tableName))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

@ekaterinadimitrova2
Copy link
Contributor

Are we ready to wrap up this one? I can run CI when done

@ekaterinadimitrova2
Copy link
Contributor

In the meantime the docs migration was also finished. So we need to consider also updating the Virtual Tables page

Copy link
Contributor

@azotcsit azotcsit left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@burmanm
In general the code looks good. I added a few minor comments. Please, take a look. Feel free to ask if there is any confusion! And thanks for your contribution! 😄

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);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't we have p999th and max here tested as well?

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";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The ticket description mentions TableMetricTables as a reference. p999 is not used there. Do we really want to expose it?

@burmanm @ekaterinadimitrova2 WDYT?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

import org.apache.cassandra.metrics.BatchMetrics;
import org.apache.cassandra.schema.TableMetadata;

public class BatchMetricsTable extends AbstractVirtualTable
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

BatchMetricsTable(String keyspace)
{
super(TableMetadata.builder(keyspace, "batch_metrics")
.kind(TableMetadata.Kind.VIRTUAL)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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"

@blerer
Copy link
Contributor

blerer commented Feb 10, 2022

Patch merged manually.

@blerer blerer closed this Feb 10, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
4 participants