Skip to content

Commit

Permalink
k/quotas: test_throughput_groups_exemptions
Browse files Browse the repository at this point in the history
A new test to verify that when a user matches a tput ctrl group,
the connections authenticated by the user are not throttled while
the rest still are.
  • Loading branch information
dlex committed Jun 28, 2023
1 parent cc62a5e commit b0c0cc7
Showing 1 changed file with 91 additions and 1 deletion.
92 changes: 91 additions & 1 deletion tests/rptest/tests/throughput_limits_snc_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
# by the Apache License, Version 2.0

import random, time, math, json, string
import socket
from enum import Enum
from typing import Tuple

Expand All @@ -16,11 +17,13 @@
from rptest.clients.rpk import RpkTool
from rptest.clients.types import TopicSpec
from rptest.services.cluster import cluster
from rptest.services.redpanda import MetricsEndpoint
from rptest.services.redpanda import MetricsEndpoint, SaslCredentials, SecurityConfig
from rptest.services.rpk_producer import RpkProducer
from rptest.tests.redpanda_test import RedpandaTest
from rptest.services.kcat_consumer import KcatConsumer
from rptest.clients.kafka_cat import KafkaCat
from rptest.services import tls
from rptest.services.admin import Admin

# This file is about throughput limiting that works at shard/node/cluster (SNC)
# levels, like cluster-wide and node-wide throughput limits
Expand All @@ -39,6 +42,7 @@ def __init__(self, test_ctx: TestContext, *args, **kwargs):
num_brokers=3,
*args,
**kwargs)
self.superuser: SaslCredentials = self.redpanda.SUPERUSER_CREDENTIALS
rnd_seed_override = test_ctx.globals.get("random_seed")
if rnd_seed_override is None:
# default seed value is composed from
Expand Down Expand Up @@ -340,3 +344,89 @@ def on_message(consumer: KcatConsumer, message: dict):
producer.stop()
consumer.stop()
consumer2.stop()

@cluster(num_nodes=5)
def test_throughput_groups_exemptions(self):
"""
Clients with configured tput exemptions are not limited.
This includes configuration by
- client_id (TBD)
- auth user
"""

security = SecurityConfig()
security.enable_sasl = True
tp_limit = 16 * kiB
self.redpanda.set_security_settings(security)
self.redpanda.set_extra_rp_conf({
self.ConfigProp.THROUGHPUT_CONTROL.value: [{
'principals': [{
'user': self.superuser.username
}]
}],
self.ConfigProp.QUOTA_NODE_MAX_IN.value:
tp_limit,
self.ConfigProp.QUOTA_NODE_MAX_EG.value:
tp_limit,
})
self.topics = [TopicSpec(partition_count=1)]
super(ThroughputLimitsSnc, self).setUp()

admin = Admin(self.redpanda)
user = SaslCredentials("user2", "password2", "SCRAM-SHA-256")
admin.create_user(user.username, user.password, user.algorithm)

rpk = RpkTool(self.redpanda,
username=self.superuser.username,
password=self.superuser.password,
sasl_mechanism=self.superuser.algorithm)
rpk.acl_create_allow_cluster(user.username, 'ALL')
rpk.acl_create_allow_topic(user.username, self.topic, 'ALL')

# run 2 producers, for admin and for user
# measure tput, check that user tput is under the threshold and
# admin tput is over the threshold
msg_size = 8 * kiB
msg_count = 30
producer0 = RpkProducer(self.test_context,
self.redpanda,
self.topic,
msg_size=msg_size,
msg_count=msg_count,
max_message_bytes=3 * msg_size,
printable=True,
user=self.superuser.username,
password=self.superuser.password)
producer1 = RpkProducer(self.test_context,
self.redpanda,
self.topic,
msg_size=msg_size,
msg_count=msg_count,
max_message_bytes=3 * msg_size,
printable=True,
user=user.username,
password=user.password)
producer0.start()
producer1.start()
start = time.time()

producer0.wait()
time0 = time.time() - start
assert producer1.is_running(
), "producer1 is throttled and must have finished last"

producer1.wait()
time1 = time.time() - start

producer0.stop()
producer1.stop()
producer0.free()
producer1.free()

rate0 = msg_count * msg_size / time0
rate1 = msg_count * msg_size / time1
self.logger.info(f"Non-throttled producer: {time0} s, {rate0} B/s")
self.logger.info(
f"Producer throttled at {tp_limit} B/s: {time1} s, {rate1} B/s")
assert rate0 / tp_limit > 5, "Exempt clients's rate must be much larger than the limit"
assert rate1 / tp_limit < 1, "Non-exempt clients's rate must be lower than the limit"

0 comments on commit b0c0cc7

Please sign in to comment.