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:32:49 UTC

[cassandra] branch client-metrics created (now 6a05bbd)

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

rustyrazorblade pushed a change to branch client-metrics
in repository https://gitbox.apache.org/repos/asf/cassandra.git.


      at 6a05bbd  minor tweaks

This branch includes the following new commits:

     new 1fea9a5  Add client request size metrics
     new 1a98571  high res
     new 9d54470  Renamed metrics based on feedback
     new fc9fd13  reworked the metric into the frame encoder / decoder instead of the pipeline
     new d77fc6e  renamed class
     new 6a05bbd  minor tweaks

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



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


[cassandra] 05/06: renamed class

Posted by ru...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d77fc6ed4774acaf61e30ac4a829893dc19f8e32
Author: Jon Haddad jon@jonhaddad.com <jo...@jonhaddad.com>
AuthorDate: Thu Apr 9 10:03:58 2020 -0700

    renamed class
---
 ...lientRequestSizeMetricsHandlerTest.java => ServerMetricsTest.java} | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/test/unit/org/apache/cassandra/transport/ClientRequestSizeMetricsHandlerTest.java b/test/unit/org/apache/cassandra/transport/ServerMetricsTest.java
similarity index 94%
rename from test/unit/org/apache/cassandra/transport/ClientRequestSizeMetricsHandlerTest.java
rename to test/unit/org/apache/cassandra/transport/ServerMetricsTest.java
index 1a8be13..081da00 100644
--- a/test/unit/org/apache/cassandra/transport/ClientRequestSizeMetricsHandlerTest.java
+++ b/test/unit/org/apache/cassandra/transport/ServerMetricsTest.java
@@ -31,9 +31,9 @@ import org.apache.cassandra.metrics.ClientRequestSizeMetrics;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /**
- * Ensures we properly account for bytes read from and to clients
+ * Ensures we properly account for metrics tracked in the native protocol
  */
-public class ClientRequestSizeMetricsHandlerTest extends CQLTester
+public class ServerMetricsTest extends CQLTester
 {
     private long totalBytesReadStart;
     private long totalBytesWrittenStart;


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


[cassandra] 03/06: Renamed metrics based on feedback

Posted by ru...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 9d544700f64515ff02c866784a72515b0c0bea3d
Author: Jon Haddad jon@jonhaddad.com <jo...@jonhaddad.com>
AuthorDate: Wed Apr 8 14:14:19 2020 -0700

    Renamed metrics based on feedback
---
 src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java b/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java
index cf26e55..c1d497b 100644
--- a/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java
@@ -31,6 +31,6 @@ 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 bytesReadPerQueryHistogram = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesReadPerQuery", null), true);
-    public static final Histogram bytesWrittenPerQueryHistogram = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesWrittenPerQuery", null), true);
+    public static final Histogram bytesReadPerQueryHistogram = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesRecievedPerFrame", null), true);
+    public static final Histogram bytesWrittenPerQueryHistogram = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesTransmittedPerFrame", null), true);
 }


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


[cassandra] 01/06: Add client request size metrics

Posted by ru...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1fea9a507079cde5c9e87f8c70c8f33cb2c7486c
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 for CASSANDRA-15704
---
 .../metrics/ClientRequestSizeMetrics.java          |  36 +++++++
 .../transport/ClientRequestSizeMetricsHandler.java |  58 +++++++++++
 .../org/apache/cassandra/transport/Server.java     |   5 +
 .../ClientRequestSizeMetricsHandlerTest.java       | 112 +++++++++++++++++++++
 4 files changed, 211 insertions(+)

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..cf26e55
--- /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 bytesReadPerQueryHistogram = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesReadPerQuery", null), true);
+    public static final Histogram bytesWrittenPerQueryHistogram = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesWrittenPerQuery", null), true);
+}
diff --git a/src/java/org/apache/cassandra/transport/ClientRequestSizeMetricsHandler.java b/src/java/org/apache/cassandra/transport/ClientRequestSizeMetricsHandler.java
new file mode 100644
index 0000000..33fe034
--- /dev/null
+++ b/src/java/org/apache/cassandra/transport/ClientRequestSizeMetricsHandler.java
@@ -0,0 +1,58 @@
+/*
+  * 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 io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.MessageToMessageCodec;
+import org.apache.cassandra.metrics.ClientRequestSizeMetrics;
+
+/**
+ * Records the number of bytes read off of and written to the network
+ * Normally we only use the MessageToMessageCodec to apply a transformation in the Netty messaging pipeline
+ * Here we need to ensure the ByteBuf sticks around past this Handler, so we need to remember to call retain()
+ */
+@ChannelHandler.Sharable
+public class ClientRequestSizeMetricsHandler extends MessageToMessageCodec<ByteBuf, ByteBuf>
+{
+    @Override
+    public void decode(ChannelHandlerContext ctx, ByteBuf buf, List<Object> results)
+    {
+        final long messageSize = buf.writerIndex() - buf.readerIndex();
+        ClientRequestSizeMetrics.totalBytesRead.inc(messageSize);
+        ClientRequestSizeMetrics.bytesReadPerQueryHistogram.update(messageSize);
+        // the buffer needs to be retained here due to Netty's internal requirements.  Without it the buffer may be freed
+        buf.retain();
+        results.add(buf);
+    }
+
+    @Override
+    public void encode(ChannelHandlerContext ctx, ByteBuf buf, List<Object> results)
+    {
+        final long messageSize = buf.writerIndex() - buf.readerIndex();
+        ClientRequestSizeMetrics.totalBytesWritten.inc(messageSize);
+        ClientRequestSizeMetrics.bytesWrittenPerQueryHistogram.update(messageSize);
+        // please see the comment above regarding retaining the ByteBuf + Netty
+        buf.retain();
+        results.add(buf);
+    }
+}
diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java
index 43b024f..64a110b 100644
--- a/src/java/org/apache/cassandra/transport/Server.java
+++ b/src/java/org/apache/cassandra/transport/Server.java
@@ -416,6 +416,7 @@ public class Server implements CassandraDaemon.Server
         private static final Frame.Encoder frameEncoder = new Frame.Encoder();
         private static final Message.ExceptionHandler exceptionHandler = new Message.ExceptionHandler();
         private static final ConnectionLimitHandler connectionLimitHandler = new ConnectionLimitHandler();
+        private static final ClientRequestSizeMetricsHandler clientRequestSizeMetricsHandler = new ClientRequestSizeMetricsHandler();
 
         private final Server server;
 
@@ -452,6 +453,10 @@ public class Server implements CassandraDaemon.Server
 
             //pipeline.addLast("debug", new LoggingHandler());
 
+            // Handler to log size of client requests and responses
+            // we need this to come after the connection limit handler so it can drop connections first
+            pipeline.addLast("requestMetricsHandler", clientRequestSizeMetricsHandler);
+
             pipeline.addLast("frameDecoder", new Frame.Decoder(server.connectionFactory));
             pipeline.addLast("frameEncoder", frameEncoder);
 
diff --git a/test/unit/org/apache/cassandra/transport/ClientRequestSizeMetricsHandlerTest.java b/test/unit/org/apache/cassandra/transport/ClientRequestSizeMetricsHandlerTest.java
new file mode 100644
index 0000000..4d01763
--- /dev/null
+++ b/test/unit/org/apache/cassandra/transport/ClientRequestSizeMetricsHandlerTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.LinkedList;
+import java.util.List;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocator;
+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 bytes read from and to clients
+ */
+public class ClientRequestSizeMetricsHandlerTest extends CQLTester
+{
+    private ClientRequestSizeMetricsHandler handler;
+    private ByteBufAllocator alloc;
+    private ByteBuf buf;
+    private List<Object> result;
+    private long totalBytesReadStart;
+    private long totalBytesWrittenStart;
+
+    private long totalBytesReadHistoCount;
+    private long totalBytesWrittenHistoCount;
+
+    @Before
+    public void setUp()
+    {
+        handler = new ClientRequestSizeMetricsHandler();
+        alloc = PooledByteBufAllocator.DEFAULT;
+        buf = alloc.buffer(1024);
+        result = new LinkedList<>();
+        buf.writeInt(1);
+
+        totalBytesReadStart = ClientRequestSizeMetrics.totalBytesRead.getCount();
+        totalBytesWrittenStart = ClientRequestSizeMetrics.totalBytesWritten.getCount();
+
+        totalBytesReadHistoCount = ClientRequestSizeMetrics.bytesReadPerQueryHistogram.getCount();
+        totalBytesWrittenHistoCount = ClientRequestSizeMetrics.bytesWrittenPerQueryHistogram.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.bytesReadPerQueryHistogram.getCount()).isGreaterThan(totalBytesReadStart);
+        assertThat(ClientRequestSizeMetrics.bytesWrittenPerQueryHistogram.getCount()).isGreaterThan(totalBytesWrittenStart);
+    }
+
+    /**
+     * Ensures we work with the right metrics within the ClientRequestSizeMetricsHandler
+     */
+    @Test
+    public void testBytesRead()
+    {
+        int beforeRefCount = buf.refCnt();
+        handler.decode(null, buf, result);
+
+        assertThat(ClientRequestSizeMetrics.totalBytesRead.getCount()).isEqualTo(totalBytesReadStart + Integer.BYTES);
+        assertThat(ClientRequestSizeMetrics.bytesReadPerQueryHistogram.getCount()).isEqualTo(totalBytesReadHistoCount + 1);
+
+        // make sure we didn't touch the write metrics
+        assertThat(ClientRequestSizeMetrics.totalBytesWritten.getCount()).isEqualTo(totalBytesWrittenStart);
+        assertThat(ClientRequestSizeMetrics.bytesWrittenPerQueryHistogram.getCount()).isEqualTo(totalBytesWrittenHistoCount);
+
+        // we should have incremented the reference count (netty ByteBuf requirement)
+        assertThat(buf.refCnt()).isEqualTo(beforeRefCount + 1);
+    }
+
+    @Test
+    public void testBytesWritten()
+    {
+        int beforeRefCount = buf.refCnt();
+        handler.encode(null, buf, result);
+
+        assertThat(ClientRequestSizeMetrics.totalBytesWritten.getCount()).isEqualTo(totalBytesWrittenStart + Integer.BYTES);
+        assertThat(ClientRequestSizeMetrics.bytesWrittenPerQueryHistogram.getCount()).isEqualTo(totalBytesWrittenHistoCount + 1);
+
+        // make sure we didn't touch the read metrics
+        assertThat(ClientRequestSizeMetrics.totalBytesRead.getCount()).isEqualTo(totalBytesReadStart);
+        assertThat(ClientRequestSizeMetrics.bytesReadPerQueryHistogram.getCount()).isEqualTo(totalBytesReadHistoCount);
+
+        assertThat(buf.refCnt()).isEqualTo(beforeRefCount + 1);
+    }
+}


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


[cassandra] 02/06: high res

Posted by ru...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1a98571c1e2dd2a01fe01bd2115c27958253c7c8
Author: Jon Haddad jon@jonhaddad.com <jo...@jonhaddad.com>
AuthorDate: Tue Apr 7 18:14:38 2020 -0700

    high res
---
 .circleci/config.yml | 82 ++++++++++++++++++++++++++--------------------------
 1 file changed, 41 insertions(+), 41 deletions(-)

diff --git a/.circleci/config.yml b/.circleci/config.yml
index 3757f46..7fecdfe 100644
--- a/.circleci/config.yml
+++ b/.circleci/config.yml
@@ -3,10 +3,10 @@ jobs:
   j8_jvm_upgrade_dtests:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 1
+    parallelism: 2
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -94,10 +94,10 @@ jobs:
   j8_cqlsh-dtests-py2-with-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -171,10 +171,10 @@ jobs:
   j11_unit_tests:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11:20181210
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -263,10 +263,10 @@ jobs:
   j11_cqlsh-dtests-py3-with-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11:20181210
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -341,10 +341,10 @@ jobs:
   j11_cqlsh-dtests-py3-no-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11:20181210
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -419,10 +419,10 @@ jobs:
   j8_cqlsh-dtests-py3-with-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -496,10 +496,10 @@ jobs:
   j8_cqlsh-dtests-py2-no-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -573,10 +573,10 @@ jobs:
   j11_cqlsh-dtests-py2-with-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11:20181210
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -651,10 +651,10 @@ jobs:
   j11_dtests-with-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11:20181210
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -732,10 +732,10 @@ jobs:
   j8_dtests-no-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -790,10 +790,10 @@ jobs:
   j8_upgradetests-no-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -889,7 +889,7 @@ jobs:
   utests_stress:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
     parallelism: 1
@@ -934,10 +934,10 @@ jobs:
   j8_unit_tests:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -1025,10 +1025,10 @@ jobs:
   j11_jvm_dtests:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11:20181210
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 1
+    parallelism: 2
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -1198,10 +1198,10 @@ jobs:
   j11_cqlsh-dtests-py2-no-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11:20181210
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -1276,10 +1276,10 @@ jobs:
   j8_dtests-with-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -1334,10 +1334,10 @@ jobs:
   j8_jvm_dtests:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 1
+    parallelism: 5
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -1505,10 +1505,10 @@ jobs:
   j8_cqlsh-dtests-py3-no-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -1582,7 +1582,7 @@ jobs:
   utests_long:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
     parallelism: 1
@@ -1627,7 +1627,7 @@ jobs:
   utests_fqltool:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
     parallelism: 1
@@ -1672,10 +1672,10 @@ jobs:
   j11_dtests-no-vnodes:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11:20181210
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra
@@ -1753,10 +1753,10 @@ jobs:
   utests_compression:
     docker:
     - image: spod/cassandra-testing-ubuntu1810-java11-w-dependencies:20190306
-    resource_class: medium
+    resource_class: xlarge
     working_directory: ~/
     shell: /bin/bash -eo pipefail -l
-    parallelism: 4
+    parallelism: 100
     steps:
     - attach_workspace:
         at: /home/cassandra


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


[cassandra] 06/06: minor tweaks

Posted by ru...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 6a05bbd5fd817c91cc709bae63179e94365cc58a
Author: Jon Haddad jon@jonhaddad.com <jo...@jonhaddad.com>
AuthorDate: Thu Apr 9 17:32:33 2020 -0700

    minor tweaks
---
 src/java/org/apache/cassandra/transport/Frame.java | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/src/java/org/apache/cassandra/transport/Frame.java b/src/java/org/apache/cassandra/transport/Frame.java
index 4a20a50..b597cc2 100644
--- a/src/java/org/apache/cassandra/transport/Frame.java
+++ b/src/java/org/apache/cassandra/transport/Frame.java
@@ -303,12 +303,12 @@ public class Frame
             header.writeByte(type.opcode);
             header.writeInt(frame.body.readableBytes());
 
-            results.add(header);
-            results.add(frame.body);
-
-            int messageSize = header.writerIndex() + frame.body.writerIndex();
+            int messageSize = header.readableBytes() + frame.body.readableBytes();
             ClientRequestSizeMetrics.totalBytesWritten.inc(messageSize);
             ClientRequestSizeMetrics.bytesTransmittedPerFrame.update(messageSize);
+
+            results.add(header);
+            results.add(frame.body);
         }
     }
 


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


[cassandra] 04/06: reworked the metric into the frame encoder / decoder instead of the pipeline

Posted by ru...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit fc9fd13bdf22d3ec91fe9e320d5c692193c8890e
Author: Jon Haddad jon@jonhaddad.com <jo...@jonhaddad.com>
AuthorDate: Wed Apr 8 14:50:18 2020 -0700

    reworked the metric into the frame encoder / decoder instead of the pipeline
---
 .../metrics/ClientRequestSizeMetrics.java          |  4 +-
 .../transport/ClientRequestSizeMetricsHandler.java | 58 ----------------------
 src/java/org/apache/cassandra/transport/Frame.java |  8 +++
 .../org/apache/cassandra/transport/Server.java     |  5 --
 .../ClientRequestSizeMetricsHandlerTest.java       | 55 ++------------------
 5 files changed, 14 insertions(+), 116 deletions(-)

diff --git a/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java b/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java
index c1d497b..41fb162 100644
--- a/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ClientRequestSizeMetrics.java
@@ -31,6 +31,6 @@ 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 bytesReadPerQueryHistogram = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesRecievedPerFrame", null), true);
-    public static final Histogram bytesWrittenPerQueryHistogram = Metrics.histogram(DefaultNameFactory.createMetricName(TYPE, "BytesTransmittedPerFrame", null), true);
+    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/ClientRequestSizeMetricsHandler.java b/src/java/org/apache/cassandra/transport/ClientRequestSizeMetricsHandler.java
deleted file mode 100644
index 33fe034..0000000
--- a/src/java/org/apache/cassandra/transport/ClientRequestSizeMetricsHandler.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
-  * 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 io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelHandler;
-import io.netty.channel.ChannelHandlerContext;
-import io.netty.handler.codec.MessageToMessageCodec;
-import org.apache.cassandra.metrics.ClientRequestSizeMetrics;
-
-/**
- * Records the number of bytes read off of and written to the network
- * Normally we only use the MessageToMessageCodec to apply a transformation in the Netty messaging pipeline
- * Here we need to ensure the ByteBuf sticks around past this Handler, so we need to remember to call retain()
- */
-@ChannelHandler.Sharable
-public class ClientRequestSizeMetricsHandler extends MessageToMessageCodec<ByteBuf, ByteBuf>
-{
-    @Override
-    public void decode(ChannelHandlerContext ctx, ByteBuf buf, List<Object> results)
-    {
-        final long messageSize = buf.writerIndex() - buf.readerIndex();
-        ClientRequestSizeMetrics.totalBytesRead.inc(messageSize);
-        ClientRequestSizeMetrics.bytesReadPerQueryHistogram.update(messageSize);
-        // the buffer needs to be retained here due to Netty's internal requirements.  Without it the buffer may be freed
-        buf.retain();
-        results.add(buf);
-    }
-
-    @Override
-    public void encode(ChannelHandlerContext ctx, ByteBuf buf, List<Object> results)
-    {
-        final long messageSize = buf.writerIndex() - buf.readerIndex();
-        ClientRequestSizeMetrics.totalBytesWritten.inc(messageSize);
-        ClientRequestSizeMetrics.bytesWrittenPerQueryHistogram.update(messageSize);
-        // please see the comment above regarding retaining the ByteBuf + Netty
-        buf.retain();
-        results.add(buf);
-    }
-}
diff --git a/src/java/org/apache/cassandra/transport/Frame.java b/src/java/org/apache/cassandra/transport/Frame.java
index 8163d7a..4a20a50 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();
@@ -301,6 +305,10 @@ public class Frame
 
             results.add(header);
             results.add(frame.body);
+
+            int messageSize = header.writerIndex() + frame.body.writerIndex();
+            ClientRequestSizeMetrics.totalBytesWritten.inc(messageSize);
+            ClientRequestSizeMetrics.bytesTransmittedPerFrame.update(messageSize);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java
index 64a110b..43b024f 100644
--- a/src/java/org/apache/cassandra/transport/Server.java
+++ b/src/java/org/apache/cassandra/transport/Server.java
@@ -416,7 +416,6 @@ public class Server implements CassandraDaemon.Server
         private static final Frame.Encoder frameEncoder = new Frame.Encoder();
         private static final Message.ExceptionHandler exceptionHandler = new Message.ExceptionHandler();
         private static final ConnectionLimitHandler connectionLimitHandler = new ConnectionLimitHandler();
-        private static final ClientRequestSizeMetricsHandler clientRequestSizeMetricsHandler = new ClientRequestSizeMetricsHandler();
 
         private final Server server;
 
@@ -453,10 +452,6 @@ public class Server implements CassandraDaemon.Server
 
             //pipeline.addLast("debug", new LoggingHandler());
 
-            // Handler to log size of client requests and responses
-            // we need this to come after the connection limit handler so it can drop connections first
-            pipeline.addLast("requestMetricsHandler", clientRequestSizeMetricsHandler);
-
             pipeline.addLast("frameDecoder", new Frame.Decoder(server.connectionFactory));
             pipeline.addLast("frameEncoder", frameEncoder);
 
diff --git a/test/unit/org/apache/cassandra/transport/ClientRequestSizeMetricsHandlerTest.java b/test/unit/org/apache/cassandra/transport/ClientRequestSizeMetricsHandlerTest.java
index 4d01763..1a8be13 100644
--- a/test/unit/org/apache/cassandra/transport/ClientRequestSizeMetricsHandlerTest.java
+++ b/test/unit/org/apache/cassandra/transport/ClientRequestSizeMetricsHandlerTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.transport;
 
-import java.util.LinkedList;
 import java.util.List;
 
 import org.junit.Before;
@@ -26,7 +25,6 @@ import org.junit.Test;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.PooledByteBufAllocator;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.metrics.ClientRequestSizeMetrics;
 
@@ -37,10 +35,6 @@ import static org.assertj.core.api.Assertions.assertThat;
  */
 public class ClientRequestSizeMetricsHandlerTest extends CQLTester
 {
-    private ClientRequestSizeMetricsHandler handler;
-    private ByteBufAllocator alloc;
-    private ByteBuf buf;
-    private List<Object> result;
     private long totalBytesReadStart;
     private long totalBytesWrittenStart;
 
@@ -50,17 +44,11 @@ public class ClientRequestSizeMetricsHandlerTest extends CQLTester
     @Before
     public void setUp()
     {
-        handler = new ClientRequestSizeMetricsHandler();
-        alloc = PooledByteBufAllocator.DEFAULT;
-        buf = alloc.buffer(1024);
-        result = new LinkedList<>();
-        buf.writeInt(1);
-
         totalBytesReadStart = ClientRequestSizeMetrics.totalBytesRead.getCount();
         totalBytesWrittenStart = ClientRequestSizeMetrics.totalBytesWritten.getCount();
 
-        totalBytesReadHistoCount = ClientRequestSizeMetrics.bytesReadPerQueryHistogram.getCount();
-        totalBytesWrittenHistoCount = ClientRequestSizeMetrics.bytesWrittenPerQueryHistogram.getCount();
+        totalBytesReadHistoCount = ClientRequestSizeMetrics.bytesRecievedPerFrame.getCount();
+        totalBytesWrittenHistoCount = ClientRequestSizeMetrics.bytesTransmittedPerFrame.getCount();
     }
 
     @Test
@@ -70,43 +58,8 @@ public class ClientRequestSizeMetricsHandlerTest extends CQLTester
 
         assertThat(ClientRequestSizeMetrics.totalBytesRead.getCount()).isGreaterThan(totalBytesReadStart);
         assertThat(ClientRequestSizeMetrics.totalBytesWritten.getCount()).isGreaterThan(totalBytesWrittenStart);
-        assertThat(ClientRequestSizeMetrics.bytesReadPerQueryHistogram.getCount()).isGreaterThan(totalBytesReadStart);
-        assertThat(ClientRequestSizeMetrics.bytesWrittenPerQueryHistogram.getCount()).isGreaterThan(totalBytesWrittenStart);
-    }
-
-    /**
-     * Ensures we work with the right metrics within the ClientRequestSizeMetricsHandler
-     */
-    @Test
-    public void testBytesRead()
-    {
-        int beforeRefCount = buf.refCnt();
-        handler.decode(null, buf, result);
-
-        assertThat(ClientRequestSizeMetrics.totalBytesRead.getCount()).isEqualTo(totalBytesReadStart + Integer.BYTES);
-        assertThat(ClientRequestSizeMetrics.bytesReadPerQueryHistogram.getCount()).isEqualTo(totalBytesReadHistoCount + 1);
-
-        // make sure we didn't touch the write metrics
-        assertThat(ClientRequestSizeMetrics.totalBytesWritten.getCount()).isEqualTo(totalBytesWrittenStart);
-        assertThat(ClientRequestSizeMetrics.bytesWrittenPerQueryHistogram.getCount()).isEqualTo(totalBytesWrittenHistoCount);
-
-        // we should have incremented the reference count (netty ByteBuf requirement)
-        assertThat(buf.refCnt()).isEqualTo(beforeRefCount + 1);
+        assertThat(ClientRequestSizeMetrics.bytesRecievedPerFrame.getCount()).isGreaterThan(totalBytesReadStart);
+        assertThat(ClientRequestSizeMetrics.bytesTransmittedPerFrame.getCount()).isGreaterThan(totalBytesWrittenStart);
     }
 
-    @Test
-    public void testBytesWritten()
-    {
-        int beforeRefCount = buf.refCnt();
-        handler.encode(null, buf, result);
-
-        assertThat(ClientRequestSizeMetrics.totalBytesWritten.getCount()).isEqualTo(totalBytesWrittenStart + Integer.BYTES);
-        assertThat(ClientRequestSizeMetrics.bytesWrittenPerQueryHistogram.getCount()).isEqualTo(totalBytesWrittenHistoCount + 1);
-
-        // make sure we didn't touch the read metrics
-        assertThat(ClientRequestSizeMetrics.totalBytesRead.getCount()).isEqualTo(totalBytesReadStart);
-        assertThat(ClientRequestSizeMetrics.bytesReadPerQueryHistogram.getCount()).isEqualTo(totalBytesReadHistoCount);
-
-        assertThat(buf.refCnt()).isEqualTo(beforeRefCount + 1);
-    }
 }


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