[ 
https://issues.apache.org/jira/browse/CASSANDRA-15766?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17100139#comment-17100139
 ] 

Jon Meredith commented on CASSANDRA-15766:
------------------------------------------

Thanks for the feedback. I was concerned it wasn't particularly idiomatic. 

My original design was to modify the actual log call to check if the statement 
should be logged, then checked if any of the format parameters were instances 
of {{Supplier<Object> and if so rebuild the parameter objects array calling 
}}{{get()}} where needed to retrieve the actual object before printing.

I backed off from that as I thought that would be multiple invocations of 
lambdas rather than a single invocation for all arguments as well as 
copying/updating the objects array (although I suppose it would be safe to 
update in place assuming it was always called as a varargs). If you're going to 
pay the cost of the lambda, maybe only worth it once. I need to play with JMH 
to understand costs better.

I'll have a go at something more idiomatic, but for cases like this it seems 
like a single lambda supplying all of the parameters would be more efficient.
{code:java}
private void onOverloaded(Message<?> message)
{
    overloadedCountUpdater.incrementAndGet(this);
    int serializedSize = canonicalSize(message);
    overloadedBytesUpdater.addAndGet(this, serializedSize);
    noSpamLogger.warn("{} overloaded; dropping {} message (queue: {} local, {} 
endpoint, {} global)",
                      lazyId,
                      FBUtilities.prettyPrintMemory(serializedSize),
                      FBUtilities.prettyPrintMemory(pendingBytes()),
                      
FBUtilities.prettyPrintMemory(reserveCapacityInBytes.endpoint.using()),
                      
FBUtilities.prettyPrintMemory(reserveCapacityInBytes.global.using()));
    callbacks.onOverloaded(message, template.to);
} {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: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to