[
https://issues.apache.org/jira/browse/CASSANDRA-15766?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17099431#comment-17099431
]
Jon Meredith commented on CASSANDRA-15766:
------------------------------------------
I've pushed up my WIP,
[branch|https://github.com/jonmeredith/cassandra/tree/15766] [and
PR|https://github.com/apache/cassandra/pull/582]
Disassembling generated code, it looks like we trade eagerly constructing a new
object array for an invoke dynamic call. I haven't had a chance to dig any more
to compare the costs. I'm interested what you think.
{code:java}
public void simpleLog();
Code:
0: aload_0
1: getfield #7 // Field mock:Lorg/slf4j/Logger;
4: getstatic #11 // Field
org/apache/cassandra/utils/NoSpamLogger$Level.INFO:Lorg/apache/cassandra/utils/NoSpamLogger$Level;
7: lconst_1
8: getstatic #23 // Field
java/util/concurrent/TimeUnit.NANOSECONDS:Ljava/util/concurrent/TimeUnit;
11: ldc #90 // String {}
13: iconst_1
14: anewarray #26 // class java/lang/Object
17: dup
18: iconst_0
19: ldc #91 // String param
21: aastore
22: invokestatic #28 // Method
org/apache/cassandra/utils/NoSpamLogger.log:(Lorg/slf4j/Logger;Lorg/apache/cassandra/utils/NoSpamLogger$Level;JLjava/util/concurrent/TimeUnit;Ljava/lang/String;[Ljava/lang/Object;)Z
25: pop
26: return
public void paramLog();
Code:
0: aload_0
1: getfield #7 // Field mock:Lorg/slf4j/Logger;
4: getstatic #11 // Field
org/apache/cassandra/utils/NoSpamLogger$Level.INFO:Lorg/apache/cassandra/utils/NoSpamLogger$Level;
7: lconst_1
8: getstatic #23 // Field
java/util/concurrent/TimeUnit.NANOSECONDS:Ljava/util/concurrent/TimeUnit;
11: ldc #90 // String {}
13: invokedynamic #92, 0 // InvokeDynamic
#1:get:()Ljava/util/function/Supplier;
18: invokestatic #82 // Method
org/apache/cassandra/utils/NoSpamLogger.log:(Lorg/slf4j/Logger;Lorg/apache/cassandra/utils/NoSpamLogger$Level;JLjava/util/concurrent/TimeUnit;Ljava/lang/String;Ljava/util/function/Supplier;)Z
21: pop
22: return
{code}
> NoSpamLogger arguments building objects on hot paths
> ----------------------------------------------------
>
> Key: CASSANDRA-15766
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15766
> Project: Cassandra
> Issue Type: Bug
> Components: Observability/Logging
> Reporter: Jon Meredith
> Assignee: Jon Meredith
> Priority: Normal
> Fix For: 4.0-rc
>
>
> NoSpamLogger is used in hot logging paths to prevent logs being overrun. For
> that to be most effective the arguments to the logger need to be cheap to
> construct. During the internode messaging refactor CASSANDRA-15066,
> performance changes to BufferPool for CASSANDRA-14416
> were accidentally reverted in the merge up from 3.11.
> Reviewing other uses since, it looks like there are a few places where the
> arguments require some form of String building.
> org.apache.cassandra.net.InboundSink#accept
> org.apache.cassandra.net.InboundMessageHandler#processCorruptFrame
> org.apache.cassandra.net.InboundMessageHandler.LargeMessage#deserialize
> org.apache.cassandra.net.OutboundConnection#onOverloaded
> org.apache.cassandra.utils.memory.BufferPool.GlobalPool#allocateMoreChunks
> Formatting arguments should either be precomputed, or if expensive they
> should be computed after the decision on whether to noSpamLog has been made.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]