You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/09/12 22:59:44 UTC
[GitHub] [kafka] akhileshchg opened a new pull request, #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
akhileshchg opened a new pull request, #12628:
URL: https://github.com/apache/kafka/pull/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.
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] mumrah commented on a diff in pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
mumrah commented on code in PR #12628:
URL: https://github.com/apache/kafka/pull/12628#discussion_r969006371
##########
metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java:
##########
@@ -534,49 +535,19 @@ Iterable<AclBinding> acls(AclBindingFilter filter) {
}
class AclIterable implements Iterable<AclBinding> {
- private final AclBindingFilter filter;
+ private final List<AclBinding> aclBindingList;
AclIterable(AclBindingFilter filter) {
- this.filter = filter;
+ this.aclBindingList = aclsByResource
Review Comment:
Just to clarify my understanding.
Previously, we were wrapping the aclsByResource iterator. This was intended to be thread-safe, but as you mentioned offline, there was no guard against the underlying map getting modified during iteration (since the ConcurrentSkipListMap might show updated elements to the iterator).
Instead, we are now making a copy of the matching AclBinding and returning that's lists iterator the caller. So, basically trading off memory for consistency.
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] akhileshchg commented on pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
akhileshchg commented on PR #12628:
URL: https://github.com/apache/kafka/pull/12628#issuecomment-1251672840
Reopened the PR. Moved the ReadWrite lock from StandardAuthorizerData scope to StandardAuthorizer. I'm fine with most of the methods. with moving the lock, but it seems loadSnapshot can be quite heavy with this implementation. Let me know your thoughts @mumrah @cmccabe @hachikuji
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] showuon commented on pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
showuon commented on PR #12628:
URL: https://github.com/apache/kafka/pull/12628#issuecomment-1251787224
> Reopened the PR. Moved the ReadWrite lock from StandardAuthorizerData scope to StandardAuthorizer. I'm okay with most methods after moving the lock, but loadSnapshot can be pretty heavy with this implementation. Let me know your thoughts @mumrah @cmccabe @hachikuji
The `loadSnapshot` is indeed a concern. I'm wondering if we can remove lock for `loadSnapshot` method? After all, the authorizer won't be up and ready to use before `loadSnapshot` completed, right?
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] akhileshchg commented on a diff in pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
akhileshchg commented on code in PR #12628:
URL: https://github.com/apache/kafka/pull/12628#discussion_r969009047
##########
metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java:
##########
@@ -534,49 +535,19 @@ Iterable<AclBinding> acls(AclBindingFilter filter) {
}
class AclIterable implements Iterable<AclBinding> {
- private final AclBindingFilter filter;
+ private final List<AclBinding> aclBindingList;
AclIterable(AclBindingFilter filter) {
- this.filter = filter;
+ this.aclBindingList = aclsByResource
Review Comment:
Yes, you're right. I think there is no other way we can guarantee the consistency here other than giving an Iterable that stays constant to the client accessing `Authorizer#acls`.
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] akhileshchg commented on pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
akhileshchg commented on PR #12628:
URL: https://github.com/apache/kafka/pull/12628#issuecomment-1251763252
I reverted the benchmark changes.
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] mumrah commented on a diff in pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
mumrah commented on code in PR #12628:
URL: https://github.com/apache/kafka/pull/12628#discussion_r975497294
##########
metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java:
##########
@@ -530,54 +545,14 @@ static AuthorizationResult findResult(Action action,
}
Iterable<AclBinding> acls(AclBindingFilter filter) {
Review Comment:
Let's add a javadoc to this method indicating that we are returning a copy of the AclBinding-s
##########
metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java:
##########
@@ -193,15 +194,29 @@ StandardAuthorizerData copyWithNewAcls(Collection<Entry<Uuid, StandardAcl>> aclE
loadingComplete,
superUsers,
defaultRule.result,
- new ConcurrentSkipListSet<>(),
- new ConcurrentHashMap<>());
+ new TreeSet<>(),
+ new HashMap<>());
for (Entry<Uuid, StandardAcl> entry : aclEntries) {
newData.addAcl(entry.getKey(), entry.getValue());
}
log.info("Applied {} acl(s) from image.", aclEntries.size());
return newData;
}
+ StandardAuthorizerData copyWithNewAcls(TreeSet<StandardAcl> aclsByResource, HashMap<Uuid,
Review Comment:
Is the other `copyWithNewAcls` still used? Can we remove it?
##########
metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizer.java:
##########
@@ -58,28 +59,46 @@ public class StandardAuthorizer implements ClusterMetadataAuthorizer {
*/
private final CompletableFuture<Void> initialLoadFuture = new CompletableFuture<>();
+ private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
/**
- * The current data. Can be read without a lock. Must be written while holding the object lock.
+ * The current data. We use a read-write lock to synchronize reads and writes to the data.
Review Comment:
Let's mention that we expect a single writer and multiple readers for this class and that the locks are here to ensure consistency (i.e., the issue you found)
##########
metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizer.java:
##########
@@ -129,23 +167,37 @@ public synchronized void loadSnapshot(Map<Uuid, StandardAcl> acls) {
public List<AuthorizationResult> authorize(
AuthorizableRequestContext requestContext,
List<Action> actions) {
- StandardAuthorizerData curData = data;
List<AuthorizationResult> results = new ArrayList<>(actions.size());
- for (Action action: actions) {
- AuthorizationResult result = curData.authorize(requestContext, action);
- results.add(result);
+ lock.readLock().lock();
+ try {
+ for (Action action : actions) {
+ AuthorizationResult result = data.authorize(requestContext, action);
+ results.add(result);
+ }
+ } finally {
+ lock.readLock().unlock();
}
return results;
}
@Override
public Iterable<AclBinding> acls(AclBindingFilter filter) {
- return data.acls(filter);
+ lock.readLock().lock();
+ try {
+ return data.acls(filter);
Review Comment:
Can you add a comment here that the returned `Iterable` is consistent?
##########
metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizer.java:
##########
@@ -129,23 +167,37 @@ public synchronized void loadSnapshot(Map<Uuid, StandardAcl> acls) {
public List<AuthorizationResult> authorize(
AuthorizableRequestContext requestContext,
List<Action> actions) {
- StandardAuthorizerData curData = data;
List<AuthorizationResult> results = new ArrayList<>(actions.size());
- for (Action action: actions) {
- AuthorizationResult result = curData.authorize(requestContext, action);
- results.add(result);
+ lock.readLock().lock();
+ try {
+ for (Action action : actions) {
+ AuthorizationResult result = data.authorize(requestContext, action);
Review Comment:
Even in the lock, we should still load `data` into a local variable to avoid loading the volatile multiple times
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] mumrah merged pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
mumrah merged PR #12628:
URL: https://github.com/apache/kafka/pull/12628
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] akhileshchg commented on a diff in pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
akhileshchg commented on code in PR #12628:
URL: https://github.com/apache/kafka/pull/12628#discussion_r975614289
##########
metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java:
##########
@@ -193,15 +194,29 @@ StandardAuthorizerData copyWithNewAcls(Collection<Entry<Uuid, StandardAcl>> aclE
loadingComplete,
superUsers,
defaultRule.result,
- new ConcurrentSkipListSet<>(),
- new ConcurrentHashMap<>());
+ new TreeSet<>(),
+ new HashMap<>());
for (Entry<Uuid, StandardAcl> entry : aclEntries) {
newData.addAcl(entry.getKey(), entry.getValue());
}
log.info("Applied {} acl(s) from image.", aclEntries.size());
return newData;
}
+ StandardAuthorizerData copyWithNewAcls(TreeSet<StandardAcl> aclsByResource, HashMap<Uuid,
Review Comment:
Removed it.
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] akhileshchg commented on pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
akhileshchg commented on PR #12628:
URL: https://github.com/apache/kafka/pull/12628#issuecomment-1244722682
```Benchmark (aclCount) (authorizerType) (denyPercentage) (resourceCount) Mode Cnt Score Error Units
AclAuthorizerBenchmark.testAclsIterator 50 ACL 20 200000 avgt 730.070 ms/op
AclAuthorizerBenchmark.testAuthorizeByResourceType 50 ACL 20 200000 avgt 0.010 ms/op
AclAuthorizerBenchmark.testAuthorizer 50 ACL 20 200000 avgt 4.505 ms/op
AclAuthorizerBenchmark.testUpdateCache 50 ACL 20 200000 avgt 1936.356 ms/op
Benchmark (aclCount) (authorizerType) (denyPercentage) (resourceCount) Mode Cnt Score Error Units
AclAuthorizerBenchmark.testAclsIterator 50 KRAFT 20 200000 avgt 2084.634 ms/op
AclAuthorizerBenchmark.testAuthorizeByResourceType 50 KRAFT 20 200000 avgt 6180.318 ms/op
AclAuthorizerBenchmark.testAuthorizer 50 KRAFT 20 200000 avgt 2.768 ms/op
AclAuthorizerBenchmark.testUpdateCache 50 KRAFT 20 200000 avgt ≈ 10⁻⁶ ms/op
```
NOTE: `authorizeByResourceType` is not implemented in `StandardAuthorizer`, so it uses the default implementation in `Authorizer`, hence it is not in the same ballpark as AclAuthorizer. Similarly `updateCache` is not implemented for `StandardAuthorizer` (we use `AclMutator`, so we cannot compare the numbers).
With the new implementation, StandardAuthorizer seems to be doing worse on the `AclsIterator` benchmark than `AclAuthorizer` and doing better in `testAuthorizer` which calls `Authorizer#authorize`.
I updated the iterator method to only loop once through acls and the performance is in the same ballpark as AclAuthorizer.
```
Benchmark (aclCount) (authorizerType) (denyPercentage) (resourceCount) Mode Cnt Score Error Units
AclAuthorizerBenchmark.testAclsIterator 50 KRAFT 20 200000 avgt 833.482 ms/op
```
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] akhileshchg closed pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
akhileshchg closed pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
URL: https://github.com/apache/kafka/pull/12628
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] cmccabe commented on pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
cmccabe commented on PR #12628:
URL: https://github.com/apache/kafka/pull/12628#issuecomment-1251690504
Thanks, @akhileshchg . Can you remove the AclAuthorizerBenchmark change from here? I can post a separate PR for this (which we don't need in 3.3...)
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org
[GitHub] [kafka] akhileshchg commented on pull request #12628: KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.
Posted by GitBox <gi...@apache.org>.
akhileshchg commented on PR #12628:
URL: https://github.com/apache/kafka/pull/12628#issuecomment-1251845966
> Reopened the PR. Moved the ReadWrite lock from StandardAuthorizerData scope to StandardAuthorizer. I'm okay with most methods after moving the lock, but loadSnapshot can be pretty heavy with this implementation. Let me know your thoughts @mumrah @cmccabe @hachikuji
Reduced the critical section for `loadSnapshot`. The change is ready for review. @mumrah @cmccabe @hachikuji
--
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: jira-unsubscribe@kafka.apache.org
For queries about this service, please contact Infrastructure at:
users@infra.apache.org