junrao commented on code in PR #14629:
URL: https://github.com/apache/kafka/pull/14629#discussion_r1376651533
##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -864,6 +778,111 @@ class ReplicaManager(val config: KafkaConfig,
}
}
+ /*
+ * Note: This method can be used as a callback in a different request
thread. Ensure that correct RequestLocal
+ * is passed when executing this method. Accessing non-thread-safe data
structures should be avoided if possible.
+ */
+ private def appendEntries(allEntries: Map[TopicPartition, MemoryRecords],
+ internalTopicsAllowed: Boolean,
+ origin: AppendOrigin,
+ requiredAcks: Short,
+ verificationGuards: Map[TopicPartition,
VerificationGuard],
Review Comment:
1. On the server side, some of the callbacks (Purgatory, the one introduced
in KAFKA-14561, etc) are called in a different thread than the caller. I am not
sure if it's possible/desirable to change all of those callbacks to be run on
the same thread. Given that model, I agree that the wide exposure of
ThreadLocal seems potentially dangerous. I pinged
https://github.com/apache/kafka/pull/9220 to see if we could consider an
alternative approach. It might be easier to fix the `ThreadLocal` thing first
before fixing this issue.
2. In general, callbacks on different threads are tricky to get right. For
example, we spend a lot of time to fix the deadlock issues related to Purgatory
(https://issues.apache.org/jira/browse/KAFKA-8334). For this newly introduced
callback, I am wondering about the locking model. We have the following path
that's called under the GroupMetadata lock. `GroupCoordinator.doSyncGroup (hold
GroupMetadata lock) => groupManager.storeGroup => appendForGroup =>
replicaManager.appendRecords => replicaManager.appendEntries`. However, if we
register the callback, `replicaManager.appendEntries` will be called without
holding the `GroupMetadata` lock. Is that safe?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]