Skip to content

CNDB-7237: add metrics to track replica response sizes #1858

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

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.cassandra.metrics;

import com.codahale.metrics.Counter;
import com.codahale.metrics.Histogram;

import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;

/**
* Metrics for tracking result sizes coming from replicas/writers to coordinators.
*/
public class ReplicaResponseSizeMetrics
{
private static final String TYPE = "ReplicaResponseSize";

/** Total bytes received from replicas in response messages */
public static final Counter totalBytesReceived = Metrics.counter(DefaultNameFactory.createMetricName(TYPE, "TotalBytesReceived", null));

/** Histogram of response sizes from replicas */
public static final Histogram bytesReceivedPerResponse = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesReceivedPerResponse", null), true);

/** Total bytes received from replicas in read responses */
public static final Counter readResponseBytesReceived = Metrics.counter(DefaultNameFactory.createMetricName(TYPE, "ReadResponseBytesReceived", null));

/** Histogram of read response sizes from replicas */
public static final Histogram readResponseBytesPerResponse = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "ReadResponseBytesPerResponse", null), true);

/** Total bytes received from replicas in write responses */
public static final Counter writeResponseBytesReceived = Metrics.counter(DefaultNameFactory.createMetricName(TYPE, "WriteResponseBytesReceived", null));

/** Histogram of write response sizes from replicas */
public static final Histogram writeResponseBytesPerResponse = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "WriteResponseBytesPerResponse", null), true);

/**
* Record the size of a response received from a replica
* @param responseSize the size of the response in bytes
* @param isReadResponse true if this is a read response, false for write response
*/
public static void recordReplicaResponseSize(int responseSize, boolean isReadResponse)
{
totalBytesReceived.inc(responseSize);
bytesReceivedPerResponse.update(responseSize);

if (isReadResponse)
{
readResponseBytesReceived.inc(responseSize);
readResponseBytesPerResponse.update(responseSize);
}
else
{
writeResponseBytesReceived.inc(responseSize);
writeResponseBytesPerResponse.update(responseSize);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,8 @@
import org.apache.cassandra.sensors.RequestSensors;
import org.apache.cassandra.sensors.RequestTracker;
import org.apache.cassandra.utils.concurrent.SimpleCondition;
import org.apache.cassandra.metrics.ReplicaResponseSizeMetrics;
import org.apache.cassandra.net.MessagingService;

import static java.util.concurrent.TimeUnit.NANOSECONDS;
import static org.apache.cassandra.locator.Replicas.countInOurDc;
Expand Down Expand Up @@ -262,6 +264,19 @@ public boolean waitingFor(InetAddressAndPort from)
* null message means "response from local write"
*/
public abstract void onResponse(Message<T> msg);

/**
* Track the size of a response message from a replica
* @param msg the response message
*/
protected void trackReplicaResponseSize(Message<T> msg)
{
if (msg != null)
{
int responseSize = msg.payloadSize(MessagingService.current_version);
ReplicaResponseSizeMetrics.recordReplicaResponseSize(responseSize, false);
}
}

public void signal()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@ public DatacenterSyncWriteResponseHandler(ReplicaPlan.ForTokenWrite replicaPlan,

public void onResponse(Message<T> message)
{
trackReplicaResponseSize(message);
try
{
String dataCenter = message == null
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.cassandra.net.Message;
import org.apache.cassandra.db.WriteType;
import org.apache.cassandra.net.Verb;
import org.apache.cassandra.metrics.ReplicaResponseSizeMetrics;

/**
* Handles blocking writes for ONE, ANY, TWO, THREE, QUORUM, and ALL consistency levels.
Expand Down Expand Up @@ -64,6 +65,7 @@ public boolean trackLatencyForSnitch(Verb responseVerb, boolean isTimeout)
@Override
public void onResponse(Message<T> m)
{
trackReplicaResponseSize(m);
if (responsesUpdater.decrementAndGet(this) == 0)
signal();
//Must be last after all subclass processing
Expand Down
7 changes: 7 additions & 0 deletions src/java/org/apache/cassandra/service/reads/ReadCallback.java
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@
import org.apache.cassandra.sensors.RequestTracker;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.utils.concurrent.SimpleCondition;
import org.apache.cassandra.metrics.ReplicaResponseSizeMetrics;
import org.apache.cassandra.net.MessagingService;

import static java.util.concurrent.TimeUnit.MILLISECONDS;

Expand Down Expand Up @@ -159,6 +161,11 @@ public int blockFor()
public void onResponse(Message<ReadResponse> message)
{
assertWaitingFor(message.from());

// Track the response size from replica to coordinator
int responseSize = message.payloadSize(MessagingService.current_version);
ReplicaResponseSizeMetrics.recordReplicaResponseSize(responseSize, true);

resolver.preprocess(message);

/*
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.cassandra.metrics;

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

import com.codahale.metrics.Snapshot;

import static org.junit.Assert.*;

public class ReplicaResponseSizeMetricsTest
{
@BeforeClass
public static void setup()
{
// Note: Counter metrics cannot be reset, so tests track deltas
}

@Test
public void testReadResponseMetrics()
{
long initialTotal = ReplicaResponseSizeMetrics.totalBytesReceived.getCount();
long initialRead = ReplicaResponseSizeMetrics.readResponseBytesReceived.getCount();

// Record a read response
int responseSize = 1024;
ReplicaResponseSizeMetrics.recordReplicaResponseSize(responseSize, true);

// Verify counters
assertEquals(initialTotal + responseSize, ReplicaResponseSizeMetrics.totalBytesReceived.getCount());
assertEquals(initialRead + responseSize, ReplicaResponseSizeMetrics.readResponseBytesReceived.getCount());

// Verify histogram recorded the value
Snapshot readSnapshot = ReplicaResponseSizeMetrics.readResponseBytesPerResponse.getSnapshot();
assertTrue(readSnapshot.size() > 0);
// Check that the histogram contains values in the expected range
assertTrue(readSnapshot.getMax() >= responseSize);
}

@Test
public void testWriteResponseMetrics()
{
long initialTotal = ReplicaResponseSizeMetrics.totalBytesReceived.getCount();
long initialWrite = ReplicaResponseSizeMetrics.writeResponseBytesReceived.getCount();

// Record a write response
int responseSize = 256;
ReplicaResponseSizeMetrics.recordReplicaResponseSize(responseSize, false);

// Verify counters
assertEquals(initialTotal + responseSize, ReplicaResponseSizeMetrics.totalBytesReceived.getCount());
assertEquals(initialWrite + responseSize, ReplicaResponseSizeMetrics.writeResponseBytesReceived.getCount());

// Verify histogram
Snapshot writeSnapshot = ReplicaResponseSizeMetrics.writeResponseBytesPerResponse.getSnapshot();
assertTrue(writeSnapshot.size() > 0);
}

@Test
public void testMultipleResponses()
{
long initialTotal = ReplicaResponseSizeMetrics.totalBytesReceived.getCount();

// Record multiple responses
int[] sizes = {100, 200, 300, 400, 500};
int expectedTotal = 0;

for (int size : sizes)
{
ReplicaResponseSizeMetrics.recordReplicaResponseSize(size, size % 2 == 0);
expectedTotal += size;
}

// Verify total
assertEquals(initialTotal + expectedTotal, ReplicaResponseSizeMetrics.totalBytesReceived.getCount());

// Verify histogram captures all sizes
Snapshot totalSnapshot = ReplicaResponseSizeMetrics.bytesReceivedPerResponse.getSnapshot();
assertTrue(totalSnapshot.size() >= sizes.length);
assertTrue(totalSnapshot.getMean() > 0);
}
}
Loading