Skip to content

Commit

Permalink
k/quotas: shortcut quota methods if no quotas
Browse files Browse the repository at this point in the history
Minimize the impact of client quota management in the default (and
expectly most common case) of having no quotas configured.
  • Loading branch information
pgellert committed Jun 27, 2024
1 parent 6c3a86b commit a22894d
Show file tree
Hide file tree
Showing 4 changed files with 26 additions and 0 deletions.
9 changes: 9 additions & 0 deletions src/v/kafka/server/client_quota_translator.cc
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@

#include <seastar/util/variant_utils.hh>

#include <absl/algorithm/container.h>

#include <optional>

namespace kafka {
Expand Down Expand Up @@ -288,4 +290,11 @@ client_quota_translator::get_default_config(client_quota_type qt) const {
}
}

bool client_quota_translator::is_empty() const {
return _quota_store.local().size() == 0
&& absl::c_all_of(all_client_quota_types, [this](auto qt) {
return !get_default_config(qt);
});
}

} // namespace kafka
3 changes: 3 additions & 0 deletions src/v/kafka/server/client_quota_translator.h
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,9 @@ class client_quota_translator {
/// `watch` can be used to register for quota changes
void watch(on_change_fn&& fn);

/// Returns true if there are no quotas configured
bool is_empty() const;

private:
using quota_config
= std::unordered_map<ss::sstring, config::client_group_quota>;
Expand Down
12 changes: 12 additions & 0 deletions src/v/kafka/server/quota_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -320,6 +320,9 @@ ss::future<std::chrono::milliseconds> quota_manager::record_partition_mutations(
/// KIP-599 throttles create_topics / delete_topics / create_partitions
/// request. This delay should only be applied to these requests if the
/// quota has been exceeded
if (_translator.is_empty()) {
co_return 0ms;
}
auto ctx = client_quota_request_ctx{
.q_type = client_quota_type::partition_mutation_quota,
.client_id = client_id,
Expand Down Expand Up @@ -391,6 +394,9 @@ ss::future<clock::duration> quota_manager::record_produce_tp_and_throttle(
std::optional<std::string_view> client_id,
uint64_t bytes,
clock::time_point now) {
if (_translator.is_empty()) {
co_return 0ms;
}
auto ctx = client_quota_request_ctx{
.q_type = client_quota_type::produce_quota,
.client_id = client_id,
Expand Down Expand Up @@ -440,6 +446,9 @@ ss::future<> quota_manager::record_fetch_tp(
std::optional<std::string_view> client_id,
uint64_t bytes,
clock::time_point now) {
if (_translator.is_empty()) {
co_return;
}
auto ctx = client_quota_request_ctx{
.q_type = client_quota_type::fetch_quota,
.client_id = client_id,
Expand Down Expand Up @@ -469,6 +478,9 @@ ss::future<> quota_manager::record_fetch_tp(

ss::future<clock::duration> quota_manager::throttle_fetch_tp(
std::optional<std::string_view> client_id, clock::time_point now) {
if (_translator.is_empty()) {
co_return 0ms;
}
auto ctx = client_quota_request_ctx{
.q_type = client_quota_type::fetch_quota,
.client_id = client_id,
Expand Down
2 changes: 2 additions & 0 deletions src/v/kafka/server/tests/client_quota_translator_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,7 @@ SEASTAR_THREAD_TEST_CASE(quota_translator_default_test) {
auto limits = f.tr.find_quota_value(key);
BOOST_CHECK_EQUAL(test_client_id_key, key);
BOOST_CHECK_EQUAL(default_limits, limits);
BOOST_CHECK(f.tr.is_empty());
}

SEASTAR_THREAD_TEST_CASE(quota_translator_modified_default_test) {
Expand All @@ -116,6 +117,7 @@ SEASTAR_THREAD_TEST_CASE(quota_translator_modified_default_test) {
auto limits = f.tr.find_quota_value(key);
BOOST_CHECK_EQUAL(test_client_id_key, key);
BOOST_CHECK_EQUAL(expected_limits, limits);
BOOST_CHECK(!f.tr.is_empty());
}

void run_quota_translator_client_group_test(fixture& f) {
Expand Down

0 comments on commit a22894d

Please sign in to comment.