Skip to content

Commit

Permalink
Merge pull request #5407 from r-vasquez/tuner-ducktape
Browse files Browse the repository at this point in the history
rpk tune ducktape tests
  • Loading branch information
twmb committed Jul 13, 2022
2 parents 4a5ab68 + 4224dc8 commit fe0db14
Show file tree
Hide file tree
Showing 5 changed files with 145 additions and 0 deletions.
6 changes: 6 additions & 0 deletions tests/rptest/clients/rpk_remote.py
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,12 @@ def cluster_config_force_reset(self, property_name):
def cluster_config_lint(self):
return self._execute([self._rpk_binary(), 'cluster', 'config', 'lint'])

def tune(self, tuner):
return self._execute([self._rpk_binary(), 'redpanda', 'tune', tuner])

def mode_set(self, mode):
return self._execute([self._rpk_binary(), 'redpanda', 'mode', mode])

def _run_config(self, cmd, path=None, timeout=30):
cmd = [self._rpk_binary(), 'redpanda', 'config'] + cmd

Expand Down
1 change: 1 addition & 0 deletions tests/rptest/test_suite_quick.yml
Original file line number Diff line number Diff line change
Expand Up @@ -17,3 +17,4 @@ quick:
- tests/wasm_identity_test.py
- tests/wasm_partition_movement_test.py
- tests/wasm_redpanda_failure_recovery_test.py
- tests/rpk_tuner_test.py
1 change: 1 addition & 0 deletions tests/rptest/test_suite_rpk.yml
Original file line number Diff line number Diff line change
Expand Up @@ -12,3 +12,4 @@ quick:
- tests/rpk_topic_test.py
- tests/rpk_cluster_test.py
- tests/rpk_config_test.py
- tests/rpk_tuner_test.py
49 changes: 49 additions & 0 deletions tests/rptest/tests/rpk_config_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -234,3 +234,52 @@ def test_config_change_then_restart_node(self):
rpk.config_set(key, value)

self.redpanda.restart_nodes(node)

@cluster(num_nodes=1)
def test_config_change_mode_prod(self):
"""
Verify that after running rpk redpanda mode prod, the
configuration values of the tuners change accordingly.
"""
node = self.redpanda.nodes[0]
rpk = RpkRemoteTool(self.redpanda, node)
rpk.mode_set("prod")
expected_config = yaml.full_load('''
enable_usage_stats: false
tune_network: true
tune_disk_scheduler: true
tune_disk_nomerges: true
tune_disk_write_cache: true
tune_disk_irq: true
tune_fstrim: false
tune_cpu: true
tune_aio_events: true
tune_clocksource: true
tune_swappiness: true
tune_transparent_hugepages: false
enable_memory_locking: false
tune_coredump: false
coredump_dir: /var/lib/redpanda/coredump
tune_ballast_file: true
overprovisioned: false
''')
with tempfile.TemporaryDirectory() as d:
node.account.copy_from(RedpandaService.NODE_CONFIG_FILE, d)

with open(os.path.join(d, 'redpanda.yaml')) as f:
actual_config = yaml.full_load(f.read())

# Delete 'admin_api' and 'kafka_api' since they are not
# needed for this test and the brokers change depending
# on the container it's running.
del actual_config['rpk']['kafka_api']
del actual_config['rpk']['admin_api']

if actual_config['rpk'] != expected_config:
self.logger.error("Configs differ")
self.logger.error(
f"Expected: {yaml.dump(expected_config)}")
self.logger.error(
f"Actual: {yaml.dump(actual_config['rpk'])}")
assert actual_config['rpk'] == expected_config
assert actual_config['redpanda']['developer_mode'] == False
88 changes: 88 additions & 0 deletions tests/rptest/tests/rpk_tuner_test.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
# Copyright 2022 Redpanda Data, Inc.
#
# Use of this software is governed by the Business Source License
# included in the file licenses/BSL.md
#
# As of the Change Date specified in that file, in accordance with
# the Business Source License, use of this software will be governed
# by the Apache License, Version 2.0

from rptest.services.cluster import cluster
from rptest.tests.redpanda_test import RedpandaTest
from rptest.clients.rpk_remote import RpkRemoteTool


class RpkTunerTest(RedpandaTest):
def __init__(self, ctx):
super(RpkTunerTest, self).__init__(test_context=ctx)
self._ctx = ctx

@cluster(num_nodes=1)
def test_tune_prod_all(self):
"""
Test will set production mode and execute rpk redpanda tune all,
we expect the command to exit with 1 if an error happens.
"""
node = self.redpanda.nodes[0]
rpk = RpkRemoteTool(self.redpanda, node)
rpk.mode_set("prod")

rpk.tune("all")

@cluster(num_nodes=1)
def test_tune_fstrim(self):
"""
Validate fstrim tuner execution,
fstrim was disabled in production mode https://github.com/redpanda-data/redpanda/issues/3068
"""
node = self.redpanda.nodes[0]
rpk = RpkRemoteTool(self.redpanda, node)
rpk.config_set('rpk.tune_fstrim', 'true')

rpk.tune("fstrim")

@cluster(num_nodes=1)
def test_tune_transparent_hugepages(self):
"""
Validate transparent hugepage tuner execution.
THP tuner is disabled in production mode
"""
node = self.redpanda.nodes[0]
rpk = RpkRemoteTool(self.redpanda, node)
rpk.config_set('rpk.tune_transparent_hugepages', 'true')

rpk.tune("transparent_hugepages")

@cluster(num_nodes=1)
def test_tune_list(self):
"""
Forward compatible test, the purpose is to check if available
tuners match our current setup, if a new tuner gets added we
will catch it here.
"""
node = self.redpanda.nodes[0]
rpk = RpkRemoteTool(self.redpanda, node)
# Set all tuners:
rpk.mode_set("prod")
rpk.config_set('rpk.tune_fstrim', 'true')
rpk.config_set('rpk.tune_transparent_hugepages', 'true')
rpk.config_set('rpk.tune_coredump', 'true')

expected = '''TUNER ENABLED SUPPORTED UNSUPPORTED-REASON
aio_events true true
ballast_file true true
clocksource true true
coredump true true
cpu true true
disk_irq true true
disk_nomerges true true
disk_scheduler true true
disk_write_cache true false Disk write cache tuner is only supported in GCP
fstrim true true
net true true
swappiness true true
transparent_hugepages true true
'''
output = rpk.tune("list")

assert output == expected

0 comments on commit fe0db14

Please sign in to comment.