Skip to content

KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.#12628

Merged
mumrah merged 9 commits intoapache:trunkfrom
akhileshchg:KAFKA-14214
Sep 20, 2022
Merged

KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads.#12628
mumrah merged 9 commits intoapache:trunkfrom
akhileshchg:KAFKA-14214

Conversation

@akhileshchg
Copy link
Copy Markdown
Contributor

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.

…istent 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.
Copy link
Copy Markdown
Member

@mumrah mumrah left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the patch @akhileshchg! Left a question inline.

As mentioned offline, it would be good to see the performance difference between the concurrent data structures and this PR. I expect this change will be slower in the case where we take a copy of the bindings in the iterator.


AclIterable(AclBindingFilter filter) {
this.filter = filter;
this.aclBindingList = aclsByResource
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

@akhileshchg
Copy link
Copy Markdown
Contributor Author

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

Copy link
Copy Markdown
Contributor

@hachikuji hachikuji left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I posted some comments on the old PR: #12627 (review). Could you take a look since it looks like they are still relevant. Also, I think we should come to some agreement regarding this comment:

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 every 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.

@akhileshchg akhileshchg reopened this Sep 19, 2022
@akhileshchg
Copy link
Copy Markdown
Contributor Author

akhileshchg commented Sep 19, 2022

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

@cmccabe
Copy link
Copy Markdown
Contributor

cmccabe commented Sep 20, 2022

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...)

@akhileshchg
Copy link
Copy Markdown
Contributor Author

I reverted the benchmark changes.

@showuon
Copy link
Copy Markdown
Member

showuon commented Sep 20, 2022

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? Or maybe in that case, we don't even worry about adding/removing the lock in loadSnapshot since there won't be other threads waiting for the lock

@akhileshchg
Copy link
Copy Markdown
Contributor Author

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

Copy link
Copy Markdown
Member

@mumrah mumrah left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@showuon I think we need the lock in loadSnapshot since it will get called any time the broker/controller makes a snapshot. With @akhileshchg latest changes, there should be a lot less contention in this method (basically just when the new StandardAuthorizerData is create and the volatile is written).

@akhileshchg thanks for picking this back up :) I left a few minor comments inline

Copy link
Copy Markdown
Member

@mumrah mumrah left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the updates, LGTM!

@mumrah mumrah merged commit 6c6b8e2 into apache:trunk Sep 20, 2022
mumrah pushed a commit that referenced this pull request Sep 20, 2022
…istent ACL reads. (#12628)

Fixes an issue with StandardAuthorizer#authorize that allowed inconsistent results. The underlying 
concurrent data structure (ConcurrentSkipListMap) had weak consistency guarantees. This meant
that a concurrent update to the authorizer data could result in the authorize function processing 
ACL updates out of order.

This patch replaces the concurrent data structures with regular non-thread safe equivalents and uses
a read/write lock for thread safety and strong consistency.

Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>, Colin P. McCabe <cmccabe@apache.org>, Luke Chen <showuon@gmail.com>
ijuma added a commit to confluentinc/kafka that referenced this pull request Sep 21, 2022
…eptember 2022)

`Jenkinsfile` was the only conflict and we ignore the changes since
they are not relevant to the Confluent build.

* apache-github/3.3: (61 commits)
  KAFKA-14214: Introduce read-write lock to StandardAuthorizer for consistent ACL reads. (apache#12628)
  KAFKA-14243: Temporarily disable unsafe downgrade (apache#12664)
  KAFKA-14240; Validate kraft snapshot state on startup (apache#12653)
  KAFKA-14233: disable testReloadUpdatedFilesWithoutConfigChange first to fix the build (apache#12658)
  KAFKA-14238;  KRaft metadata log should not delete segment past the latest snapshot (apache#12655)
  KAFKA-14156: Built-in partitioner may create suboptimal batches (apache#12570)
  MINOR: Adds KRaft versions of most streams system tests (apache#12458)
  MINOR; Add missing li end tag (apache#12640)
  MINOR: Mention that kraft is production ready in upgrade notes (apache#12635)
  MINOR: Add upgrade note regarding the Strictly Uniform Sticky Partitioner (KIP-794)  (apache#12630)
  KAFKA-14222; KRaft's memory pool should always allocate a buffer (apache#12625)
  KAFKA-14208; Do not raise wakeup in consumer during asynchronous offset commits (apache#12626)
  KAFKA-14196; Do not continue fetching partitions awaiting auto-commit prior to revocation (apache#12603)
  KAFKA-14215; Ensure forwarded requests are applied to broker request quota (apache#12624)
  MINOR; Remove end html tag from upgrade (apache#12605)
  Remove the html end tag from upgrade.html
  KAFKA-14205; Document how to replace the disk for the KRaft Controller (apache#12597)
  KAFKA-14203 Disable snapshot generation on broker after metadata errors (apache#12596)
  KAFKA-14216: Remove ZK reference from org.apache.kafka.server.quota.ClientQuotaCallback javadoc (apache#12617)
  KAFKA-14217: app-reset-tool.html should not show --zookeeper flag that no longer exists (apache#12618)
  ...
guozhangwang pushed a commit to guozhangwang/kafka that referenced this pull request Jan 25, 2023
…istent ACL reads. (apache#12628)

Fixes an issue with StandardAuthorizer#authorize that allowed inconsistent results. The underlying 
concurrent data structure (ConcurrentSkipListMap) had weak consistency guarantees. This meant
that a concurrent update to the authorizer data could result in the authorize function processing 
ACL updates out of order.

This patch replaces the concurrent data structures with regular non-thread safe equivalents and uses
a read/write lock for thread safety and strong consistency.

Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>, Colin P. McCabe <cmccabe@apache.org>, Luke Chen <showuon@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Kafka Broker

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants