Log message size in trace message in OutboundTcpConnection patch by aweisberg; reviewed by jasobrown for CASSANDRA-13028
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/07f990bc Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/07f990bc Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/07f990bc Branch: refs/heads/trunk Commit: 07f990bc2d786fb2693369eed1a77c37fa0fd953 Parents: c0e7110 Author: Ariel Weisberg <aweisb...@apple.com> Authored: Fri Dec 9 17:14:35 2016 -0500 Committer: Jason Brown <jasedbr...@gmail.com> Committed: Mon Dec 12 10:30:00 2016 -0800 ---------------------------------------------------------------------- CHANGES.txt | 1 + src/java/org/apache/cassandra/net/OutboundTcpConnection.java | 8 +++++++- 2 files changed, 8 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/07f990bc/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index c4f4c1f..ed0fc14 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.12 + * Log message size in trace message in OutboundTcpConnection (CASSANDRA-13028) * Add timeUnit Days for cassandra-stress (CASSANDRA-13029) * Add mutation size and batch metrics (CASSANDRA-12649) * Add method to get size of endpoints to TokenMetadata (CASSANDRA-12999) http://git-wip-us.apache.org/repos/asf/cassandra/blob/07f990bc/src/java/org/apache/cassandra/net/OutboundTcpConnection.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java index 1f47334..683aed2 100644 --- a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java +++ b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java @@ -82,6 +82,9 @@ public class OutboundTcpConnection extends FastThreadLocalThread private static final String BUFFER_SIZE_PROPERTY = PREFIX + "otc_buffer_size"; private static final int BUFFER_SIZE = Integer.getInteger(BUFFER_SIZE_PROPERTY, 1024 * 64); + //Size of 3 elements added to every message + private static final int PROTOCOL_MAGIC_ID_TIMESTAMP_SIZE = 12; + private static CoalescingStrategy newCoalescingStrategy(String displayName) { return CoalescingStrategies.newCoalescingStrategy(DatabaseDescriptor.getOtcCoalescingStrategy(), @@ -280,7 +283,9 @@ public class OutboundTcpConnection extends FastThreadLocalThread { UUID sessionId = UUIDGen.getUUID(ByteBuffer.wrap(sessionBytes)); TraceState state = Tracing.instance.get(sessionId); - String message = String.format("Sending %s message to %s", qm.message.verb, poolReference.endPoint()); + String message = String.format("Sending %s message to %s message size %d bytes", qm.message.verb, + poolReference.endPoint(), + qm.message.serializedSize(targetVersion) + PROTOCOL_MAGIC_ID_TIMESTAMP_SIZE); // session may have already finished; see CASSANDRA-5668 if (state == null) { @@ -335,6 +340,7 @@ public class OutboundTcpConnection extends FastThreadLocalThread private void writeInternal(MessageOut message, int id, long timestamp) throws IOException { + //If you add/remove fields before the message don't forget to update PROTOCOL_MAGIC_ID_TIMESTAMP_SIZE out.writeInt(MessagingService.PROTOCOL_MAGIC); if (targetVersion < MessagingService.VERSION_20)