cassandra-pr mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [cassandra] dcapwell commented on a change in pull request #524: CASSANDRA-15704 Add client request size metrics
Date Wed, 08 Apr 2020 01:51:50 GMT
dcapwell commented on a change in pull request #524: CASSANDRA-15704 Add client request size
metrics
URL: https://github.com/apache/cassandra/pull/524#discussion_r405210931
 
 

 ##########
 File path: 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();
 
 Review comment:
   sadly this isn't correct.  https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/transport/Frame.java#L302
has a buffer for the header and a separate buffer for the body; that should then cause this
method to get called twice per write.  
   
   This should be fine for totalBytesWritten, but bytesWrittenPerQueryHistogram will be off
since it will see N messages of size 9 and N messages of variable length (where N is the total
number of messages).

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


Mime
View raw message