KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.#12627
KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.#12627akhileshchg wants to merge 1 commit intoapache:trunkfrom
Conversation
…nsistent ACL reads. The issue with StandardAuthorizer#authorize is, that it looks up aclsByResources (which is of type ConcurrentSkipListMap)twice for every authorize call and uses Iterator with weak consistency guarantees on top of aclsByResources. This can cause the authorize function call to process the concurrent writes out of order. Implemented ReadWrite lock at StandardAuthorizer level to make sure the reads are strongly consistent with write order.
hachikuji
left a comment
There was a problem hiding this comment.
Thanks @akhileshchg. I left some initial comments. I do wonder about the tradeoffs between the approach used here and what we do in AclAuthorizer. The advantage of the latter approach is that reads are never blocked. Since the authorizer needs to be accessed on very request, that is a tough advantage to overlook. On the other hand, to make it work, it looks like we would need to add batching methods for adding new ACLs since doing a copy for every ACL is probably a non-starter.
| @@ -534,49 +536,19 @@ Iterable<AclBinding> acls(AclBindingFilter filter) { | |||
| } | |||
|
|
|||
| class AclIterable implements Iterable<AclBinding> { | |||
There was a problem hiding this comment.
Do we still need this class since it is just wrapping a list iterator?
| results.add(result); | ||
| } | ||
| return results; | ||
| return inReadLock(() -> { |
There was a problem hiding this comment.
I know it does not look as pretty, but perhaps we should just do the try/finally blocks. Especially for the case of authorize, it is annoying to have the additional allocations just to pass the calback.
There was a problem hiding this comment.
Okay. I'll remove the functions.
| @Override | ||
| public void addAcl(Uuid id, StandardAcl acl) { | ||
| data.addAcl(id, acl); | ||
| inWriteLock(() -> data.addAcl(id, acl)); |
There was a problem hiding this comment.
I am not sure how much it matters, but it would probably be more efficient for addAcl to be a batched API. Perhaps it is ok since hopefully most of the time the brunt of the initialization is in loadSnapshot.
|
This was closed in favor of #12628 |
KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
The issue with StandardAuthorizer#authorize is, that it looks up aclsByResources (which is of type ConcurrentSkipListMap)twice for every authorize call and uses Iterator with weak consistency guarantees on top of aclsByResources. This can cause the authorize function call to process the concurrent writes out of order.
Implemented ReadWrite lock at StandardAuthorizer level to make sure the reads are strongly consistent with write order.