You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by gw...@apache.org on 2015/12/16 02:06:15 UTC

kafka git commit: Kafka-2992: Guard trace statements in the inner loop of the replica fetcher

Repository: kafka
Updated Branches:
  refs/heads/trunk fdac34e6a -> 5df783f18


Kafka-2992: Guard trace statements in the inner loop of the replica fetcher

We're seeing some GC pause issues in production, and during our investigation found that the thunks created during invocation of three trace statements guarded in this PR were responsible for ~98% of all allocations by object count and ~90% by size. While I'm not sure that this was actually the cause of our issue, it seems prudent to avoid useless allocations in a tight loop.

I realize that the trace() call does its own guarding internally, however it's insufficient to prevent allocation of the thunk.

This is my original work, and I license it to the Kafka project under the project's Apache license.

Author: Cory Kolbeck <co...@urbanairship.com>

Reviewers: Gwen Shapira

Closes #682 from ckolbeck/guard-trace-statements


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/5df783f1
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/5df783f1
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/5df783f1

Branch: refs/heads/trunk
Commit: 5df783f18e9a6a6e17960e3ef4b0a66f4d10d65c
Parents: fdac34e
Author: Cory Kolbeck <co...@urbanairship.com>
Authored: Tue Dec 15 17:05:46 2015 -0800
Committer: Gwen Shapira <cs...@gmail.com>
Committed: Tue Dec 15 17:05:46 2015 -0800

----------------------------------------------------------------------
 .../scala/kafka/server/ReplicaFetcherThread.scala    | 15 +++++++++------
 1 file changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/5df783f1/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
index aa8e14f..8ce0c8f 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
@@ -108,18 +108,21 @@ class ReplicaFetcherThread(name: String,
 
       if (fetchOffset != replica.logEndOffset.messageOffset)
         throw new RuntimeException("Offset mismatch: fetched offset = %d, log end offset = %d.".format(fetchOffset, replica.logEndOffset.messageOffset))
-      trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d"
-            .format(replica.brokerId, replica.logEndOffset.messageOffset, topicAndPartition, messageSet.sizeInBytes, partitionData.highWatermark))
+      if (logger.isTraceEnabled)
+        trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d"
+          .format(replica.brokerId, replica.logEndOffset.messageOffset, topicAndPartition, messageSet.sizeInBytes, partitionData.highWatermark))
       replica.log.get.append(messageSet, assignOffsets = false)
-      trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s"
-            .format(replica.brokerId, replica.logEndOffset.messageOffset, messageSet.sizeInBytes, topicAndPartition))
+      if (logger.isTraceEnabled)
+        trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s"
+          .format(replica.brokerId, replica.logEndOffset.messageOffset, messageSet.sizeInBytes, topicAndPartition))
       val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark)
       // for the follower replica, we do not need to keep
       // its segment base offset the physical position,
       // these values will be computed upon making the leader
       replica.highWatermark = new LogOffsetMetadata(followerHighWatermark)
-      trace("Follower %d set replica high watermark for partition [%s,%d] to %s"
-            .format(replica.brokerId, topic, partitionId, followerHighWatermark))
+      if (logger.isTraceEnabled)
+        trace("Follower %d set replica high watermark for partition [%s,%d] to %s"
+          .format(replica.brokerId, topic, partitionId, followerHighWatermark))
     } catch {
       case e: KafkaStorageException =>
         fatal("Disk error while replicating data.", e)