You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ru...@apache.org on 2020/04/10 00:53:58 UTC

[cassandra] branch trunk updated: Add client request size metrics

This is an automated email from the ASF dual-hosted git repository.

rustyrazorblade pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 165081a  Add client request size metrics
165081a is described below

commit 165081afdaf82ff36a1ccf46f933faf6f76ec102
Author: Jon Haddad jon@jonhaddad.com <jo...@jonhaddad.com>
AuthorDate: Tue Apr 7 09:57:31 2020 -0700

    Add client request size metrics
    
    Patch by Jon Haddad; Reviewed by David Capwell for CASSANDRA-15704
---
 CHANGES.txt                                        |  1 +
 .../metrics/ClientRequestSizeMetrics.java          | 36 ++++++++++++
 src/java/org/apache/cassandra/transport/Frame.java |  8 +++
 .../cassandra/transport/ServerMetricsTest.java     | 65 ++++++++++++++++++++++
 4 files changed, 110 insertions(+)

diff --git a/CHANGES.txt b/CHANGES.txt
index 88ba6cb..878770b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0-alpha4
+ * Add client request size server metrics (CASSANDRA-15704)
  * Add additional logging around FileUtils and compaction leftover cleanup (CASSANDRA-15705)
  * Mark system_views/system_virtual_schema as non-alterable keyspaces in cqlsh (CASSANDRA-15711)
  * Fail incremental repair if an old version sstable is involved (CASSANDRA-15612)
diff --git a/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java b/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java
new file mode 100644
index 0000000..41fb162
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java
@@ -0,0 +1,36 @@
+/*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+package org.apache.cassandra.metrics;
+
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Histogram;
+
+/**
+ * Metrics to track the size of incoming and outgoing bytes at Cassandra server.
+ */
+public class ClientRequestSizeMetrics
+{
+    private static final String TYPE = "ClientRequestSize";
+    public static final Counter totalBytesRead = Metrics.counter(DefaultNameFactory.createMetricName(TYPE, "IncomingBytes", null));
+    public static final Counter totalBytesWritten = Metrics.counter(DefaultNameFactory.createMetricName(TYPE, "OutgoingBytes", null));
+    public static final Histogram bytesRecievedPerFrame = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesRecievedPerFrame", null), true);
+    public static final Histogram bytesTransmittedPerFrame = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesTransmittedPerFrame", null), true);
+}
diff --git a/src/java/org/apache/cassandra/transport/Frame.java b/src/java/org/apache/cassandra/transport/Frame.java
index 8163d7a..b597cc2 100644
--- a/src/java/org/apache/cassandra/transport/Frame.java
+++ b/src/java/org/apache/cassandra/transport/Frame.java
@@ -32,6 +32,7 @@ import io.netty.handler.codec.MessageToMessageEncoder;
 import io.netty.util.Attribute;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.metrics.ClientRequestSizeMetrics;
 import org.apache.cassandra.transport.frame.FrameBodyTransformer;
 import org.apache.cassandra.transport.messages.ErrorMessage;
 
@@ -222,6 +223,9 @@ public class Frame
             if (buffer.readableBytes() < frameLength)
                 return null;
 
+            ClientRequestSizeMetrics.totalBytesRead.inc(frameLength);
+            ClientRequestSizeMetrics.bytesRecievedPerFrame.update(frameLength);
+
             // extract body
             ByteBuf body = buffer.slice(idx, (int) bodyLength);
             body.retain();
@@ -299,6 +303,10 @@ public class Frame
             header.writeByte(type.opcode);
             header.writeInt(frame.body.readableBytes());
 
+            int messageSize = header.readableBytes() + frame.body.readableBytes();
+            ClientRequestSizeMetrics.totalBytesWritten.inc(messageSize);
+            ClientRequestSizeMetrics.bytesTransmittedPerFrame.update(messageSize);
+
             results.add(header);
             results.add(frame.body);
         }
diff --git a/test/unit/org/apache/cassandra/transport/ServerMetricsTest.java b/test/unit/org/apache/cassandra/transport/ServerMetricsTest.java
new file mode 100644
index 0000000..081da00
--- /dev/null
+++ b/test/unit/org/apache/cassandra/transport/ServerMetricsTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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.transport;
+
+import java.util.List;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.metrics.ClientRequestSizeMetrics;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Ensures we properly account for metrics tracked in the native protocol
+ */
+public class ServerMetricsTest extends CQLTester
+{
+    private long totalBytesReadStart;
+    private long totalBytesWrittenStart;
+
+    private long totalBytesReadHistoCount;
+    private long totalBytesWrittenHistoCount;
+
+    @Before
+    public void setUp()
+    {
+        totalBytesReadStart = ClientRequestSizeMetrics.totalBytesRead.getCount();
+        totalBytesWrittenStart = ClientRequestSizeMetrics.totalBytesWritten.getCount();
+
+        totalBytesReadHistoCount = ClientRequestSizeMetrics.bytesRecievedPerFrame.getCount();
+        totalBytesWrittenHistoCount = ClientRequestSizeMetrics.bytesTransmittedPerFrame.getCount();
+    }
+
+    @Test
+    public void testReadAndWriteMetricsAreRecordedDuringNativeRequests() throws Throwable
+    {
+        executeNet("SELECT * from system.peers");
+
+        assertThat(ClientRequestSizeMetrics.totalBytesRead.getCount()).isGreaterThan(totalBytesReadStart);
+        assertThat(ClientRequestSizeMetrics.totalBytesWritten.getCount()).isGreaterThan(totalBytesWrittenStart);
+        assertThat(ClientRequestSizeMetrics.bytesRecievedPerFrame.getCount()).isGreaterThan(totalBytesReadStart);
+        assertThat(ClientRequestSizeMetrics.bytesTransmittedPerFrame.getCount()).isGreaterThan(totalBytesWrittenStart);
+    }
+
+}


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