You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2020/07/18 05:38:57 UTC

[GitHub] [cassandra] maedhroz opened a new pull request #685: CASSANDRA-15766 NoSpamLogger arguments building objects on hot paths

maedhroz opened a new pull request #685:
URL: https://github.com/apache/cassandra/pull/685


   


----------------------------------------------------------------
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



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #685: CASSANDRA-15766 NoSpamLogger arguments building objects on hot paths

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #685:
URL: https://github.com/apache/cassandra/pull/685#discussion_r457052017



##########
File path: src/java/org/apache/cassandra/net/OutboundConnection.java
##########
@@ -439,13 +452,14 @@ private void releaseCapacity(long count, long bytes)
     private void onOverloaded(Message<?> message)
     {
         overloadedCountUpdater.incrementAndGet(this);
-        overloadedBytesUpdater.addAndGet(this, canonicalSize(message));
+        
+        int canonicalSize = canonicalSize(message);
+        overloadedBytesUpdater.addAndGet(this, canonicalSize);
+        
         noSpamLogger.warn("{} overloaded; dropping {} message (queue: {} local, {} endpoint, {} global)",
-                          id(),
-                          FBUtilities.prettyPrintMemory(canonicalSize(message)),
-                          FBUtilities.prettyPrintMemory(pendingBytes()),
-                          FBUtilities.prettyPrintMemory(reserveCapacityInBytes.endpoint.using()),
-                          FBUtilities.prettyPrintMemory(reserveCapacityInBytes.global.using()));
+                          this, FBUtilities.prettyPrintMemory(canonicalSize),

Review comment:
       It didn't seem worth creating a more complex mechanism to lazily materialize the whole group of binding values.




----------------------------------------------------------------
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



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #685: CASSANDRA-15766 NoSpamLogger arguments building objects on hot paths

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #685:
URL: https://github.com/apache/cassandra/pull/685#discussion_r456753975



##########
File path: src/java/org/apache/cassandra/utils/memory/BufferPool.java
##########
@@ -187,22 +200,26 @@ public static void debug(Debug setDebug)
     {
         /** The size of a bigger chunk, 1 MiB, must be a multiple of NORMAL_CHUNK_SIZE */
         static final int MACRO_CHUNK_SIZE = 64 * NORMAL_CHUNK_SIZE;
+        private static final String READABLE_MACRO_CHUNK_SIZE = prettyPrintMemory(MACRO_CHUNK_SIZE);
 
         static
         {
             assert Integer.bitCount(NORMAL_CHUNK_SIZE) == 1; // must be a power of 2
             assert Integer.bitCount(MACRO_CHUNK_SIZE) == 1; // must be a power of 2
             assert MACRO_CHUNK_SIZE % NORMAL_CHUNK_SIZE == 0; // must be a multiple
 
-            logger.info("Global buffer pool limit is {}",
-                            prettyPrintMemory(MEMORY_USAGE_THRESHOLD));
+            logger.info("Global buffer pool limit is {}", prettyPrintMemory(MEMORY_USAGE_THRESHOLD));
         }
 
         private final Queue<Chunk> macroChunks = new ConcurrentLinkedQueue<>();
         // TODO (future): it would be preferable to use a CLStack to improve cache occupancy; it would also be preferable to use "CoreLocal" storage
         private final Queue<Chunk> chunks = new ConcurrentLinkedQueue<>();
         private final AtomicLong memoryUsage = new AtomicLong();
 
+        /** Used in logging statements to lazily build a human-readable current memory usage. */
+        private final Object readableSizeInBytes = 
+            new Object() { @Override public String toString() { return prettyPrintMemory(sizeInBytes()); } };

Review comment:
       The assumption here is that the pool instance will be long-lived and so the overhead of creating this `Object` will be negligible.




----------------------------------------------------------------
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



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


[GitHub] [cassandra] jasonstack commented on a change in pull request #685: CASSANDRA-15766 NoSpamLogger arguments building objects on hot paths

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #685:
URL: https://github.com/apache/cassandra/pull/685#discussion_r456938566



##########
File path: src/java/org/apache/cassandra/net/OutboundConnection.java
##########
@@ -439,13 +452,14 @@ private void releaseCapacity(long count, long bytes)
     private void onOverloaded(Message<?> message)
     {
         overloadedCountUpdater.incrementAndGet(this);
-        overloadedBytesUpdater.addAndGet(this, canonicalSize(message));
+        
+        int canonicalSize = canonicalSize(message);
+        overloadedBytesUpdater.addAndGet(this, canonicalSize);
+        
         noSpamLogger.warn("{} overloaded; dropping {} message (queue: {} local, {} endpoint, {} global)",
-                          id(),
-                          FBUtilities.prettyPrintMemory(canonicalSize(message)),
-                          FBUtilities.prettyPrintMemory(pendingBytes()),
-                          FBUtilities.prettyPrintMemory(reserveCapacityInBytes.endpoint.using()),
-                          FBUtilities.prettyPrintMemory(reserveCapacityInBytes.global.using()));
+                          this, FBUtilities.prettyPrintMemory(canonicalSize),

Review comment:
       I suppose the remaining one won't cause any issue.




----------------------------------------------------------------
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



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #685: CASSANDRA-15766 NoSpamLogger arguments building objects on hot paths

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #685:
URL: https://github.com/apache/cassandra/pull/685#discussion_r456753597



##########
File path: src/java/org/apache/cassandra/net/InboundMessageHandler.java
##########
@@ -822,7 +828,7 @@ private Message deserialize()
             catch (IncompatibleSchemaException e)
             {
                 callbacks.onFailedDeserialize(size, header, e);
-                noSpamLogger.info("{} incompatible schema encountered while deserializing a message", id(), e);
+                noSpamLogger.info("{} incompatible schema encountered while deserializing a message", InboundMessageHandler.this, e);

Review comment:
       self-nit: I thought it might be nice for readability (and future-proofing?) to use `InboundMessageHandler.this` even in the other cases where it's no necessary. Thoughts?




----------------------------------------------------------------
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



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #685: CASSANDRA-15766 NoSpamLogger arguments building objects on hot paths

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #685:
URL: https://github.com/apache/cassandra/pull/685#discussion_r456753673



##########
File path: src/java/org/apache/cassandra/net/OutboundConnection.java
##########
@@ -120,6 +121,18 @@
      *  we allocate from here whenever queueSize > queueCapacity */
     private final EndpointAndGlobal reserveCapacityInBytes;
 
+    /** Used in logging statements to lazily build a human-readable number of pending bytes. */
+    private final Object readablePendingBytes =
+        new Object() { @Override public String toString() { return prettyPrintMemory(pendingBytes()); } };
+
+    /** Used in logging statements to lazily build a human-readable number of reserve endpoint bytes in use. */
+    private final Object readableReserveEndpointUsing =
+        new Object() { @Override public String toString() { return prettyPrintMemory(reserveCapacityInBytes.endpoint.using()); } };
+
+    /** Used in logging statements to lazily build a human-readable number of reserve global bytes in use. */
+    private final Object readableReserveGlobalUsing =
+        new Object() { @Override public String toString() { return prettyPrintMemory(reserveCapacityInBytes.global.using()); } };

Review comment:
       The comments hopefully explain well enough what these do. My assumption is that we won't actually be creating enough `OutboundConnection` instances to make the overhead of creating these three objects matter.




----------------------------------------------------------------
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



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #685: CASSANDRA-15766 NoSpamLogger arguments building objects on hot paths

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #685:
URL: https://github.com/apache/cassandra/pull/685#discussion_r456754358



##########
File path: src/java/org/apache/cassandra/utils/memory/BufferPool.java
##########
@@ -252,9 +268,7 @@ private Chunk allocateMoreChunks()
                 noSpamLogger.error("Buffer pool failed to allocate chunk of {}, current size {} ({}). " +
                                    "Attempting to continue; buffers will be allocated in on-heap memory which can degrade performance. " +
                                    "Make sure direct memory size (-XX:MaxDirectMemorySize) is large enough to accommodate off-heap memtables and caches.",
-                                   prettyPrintMemory(MACRO_CHUNK_SIZE),
-                                   prettyPrintMemory(sizeInBytes()),
-                                   oom.toString());
+                                   READABLE_MACRO_CHUNK_SIZE, readableMemoryUsage, oom.getClass().getName());

Review comment:
       It didn't look like the `OutOfMemoryError` had much of a message accompanying it...really ever. If it did, it basically looked like the one already explicitly logged above. Without a message the class name is all we got before, so now we're just doing it cheaply.




----------------------------------------------------------------
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



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #685: CASSANDRA-15766 NoSpamLogger arguments building objects on hot paths

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #685:
URL: https://github.com/apache/cassandra/pull/685#discussion_r456753781



##########
File path: src/java/org/apache/cassandra/net/OutboundConnection.java
##########
@@ -439,13 +452,14 @@ private void releaseCapacity(long count, long bytes)
     private void onOverloaded(Message<?> message)
     {
         overloadedCountUpdater.incrementAndGet(this);
-        overloadedBytesUpdater.addAndGet(this, canonicalSize(message));
+        
+        int canonicalSize = canonicalSize(message);

Review comment:
       We were calculating the canonical size twice...so fixed that while I was here.




----------------------------------------------------------------
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



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


[GitHub] [cassandra] smiklosovic closed pull request #685: CASSANDRA-15766 NoSpamLogger arguments building objects on hot paths

Posted by GitBox <gi...@apache.org>.
smiklosovic closed pull request #685:
URL: https://github.com/apache/cassandra/pull/685


   


-- 
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.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [cassandra] maedhroz commented on a change in pull request #685: CASSANDRA-15766 NoSpamLogger arguments building objects on hot paths

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #685:
URL: https://github.com/apache/cassandra/pull/685#discussion_r456753504



##########
File path: src/java/org/apache/cassandra/net/InboundMessageHandler.java
##########
@@ -692,6 +692,12 @@ String id()
         return SocketFactory.channelId(peer, self, type, channel.id().asShortText());
     }
 
+    @Override
+    public String toString()
+    {
+        return id();
+    }

Review comment:
       Assuming there isn't something that actually relied on the default `Object#toString()`, this seemed like a reasonable way to solve the construction problem here. We get the laziness we want, and we get it from `this`, the already created enclosing instance.




----------------------------------------------------------------
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



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


[GitHub] [cassandra] jasonstack commented on a change in pull request #685: CASSANDRA-15766 NoSpamLogger arguments building objects on hot paths

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #685:
URL: https://github.com/apache/cassandra/pull/685#discussion_r456938356



##########
File path: src/java/org/apache/cassandra/net/InboundMessageHandler.java
##########
@@ -692,6 +692,12 @@ String id()
         return SocketFactory.channelId(peer, self, type, channel.id().asShortText());
     }
 
+    @Override
+    public String toString()
+    {
+        return id();
+    }

Review comment:
       smart idea

##########
File path: src/java/org/apache/cassandra/net/OutboundConnection.java
##########
@@ -439,13 +452,14 @@ private void releaseCapacity(long count, long bytes)
     private void onOverloaded(Message<?> message)
     {
         overloadedCountUpdater.incrementAndGet(this);
-        overloadedBytesUpdater.addAndGet(this, canonicalSize(message));
+        
+        int canonicalSize = canonicalSize(message);

Review comment:
       `message` caches its `canonicalSize`..




----------------------------------------------------------------
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



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