artemlivshits commented on code in PR #14629:
URL: https://github.com/apache/kafka/pull/14629#discussion_r1375355538
##########
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:
I think how much work is done in the callback is more a quantitative
difference rather than a qualitative difference -- how much work needs to be
done to treat it as a new pattern that requires some special care (and what is
that care, can we qualify)? I agree that if the functionality in the
callback just happened to be after the point of use of the RequestLocal then we
wouldn't hit the problem (even if we captured it in the callback), but it seems
to a matter of just random "luck". I don't think "don't be unlucky" is a
useful engineering guidance. On the other hand, saying "don't pass thread
local as an argument, access it at the point of use as a thread local instead"
would avoid these cases.
But maybe the first point to discuss is do we agree that the root cause is
the fact that RequestLocal is bound to the executing thread context
(effectively thread local passed as an argument) and it's not a typical pattern
in Java or do we think that it's something else? At some point I think there
was a theory that the machinery for re-scheduling the callback on the request
thread pool could be a culprit, but I think we all agree, that the problem
would be the same even if the callback was executed directly on the network
thread (or pretty much in any other way other than trying to schedule it on the
exact same thread, which is doable, btw, but would hinder performance).
--
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]