Skip to content

Commit

Permalink
tests: add rpk profile tests
Browse files Browse the repository at this point in the history
Introduces new rpk/rpk_remote handlers for newly
introduced command 'profile'.

Also add the deletion of the rpk.yaml as part of
the node cleanup process.
  • Loading branch information
r-vasquez committed Jun 5, 2023
1 parent b253ab0 commit 95f1cfe
Show file tree
Hide file tree
Showing 5 changed files with 184 additions and 14 deletions.
22 changes: 22 additions & 0 deletions tests/rptest/clients/rpk.py
Original file line number Diff line number Diff line change
Expand Up @@ -1181,3 +1181,25 @@ def describe_log_dirs(self):
int(size)))

return result

def cloud_login_cc(self, id, secret):

cmd = [
self._rpk_binary(), "cloud", "login", "--client-id", id,
"--client-secret", secret
]

self._redpanda.logger.debug(
"Executing command: %s cloud login --client-id %s --client-secret [redacted]",
self._rpk_binary(), id)

return self._execute(cmd, log_cmd=False)

def cloud_logout(self, clear_credentials=True):

cmd = [self._rpk_binary(), "cloud", "logout"]

if clear_credentials:
cmd += ["--clear-credentials"]

return self._execute(cmd)
47 changes: 47 additions & 0 deletions tests/rptest/clients/rpk_remote.py
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,53 @@ def _run_config(self, cmd, path=None, timeout=30):

return self._execute(cmd, timeout=timeout)

def _run_profile(self, cmd):
cmd = [self._rpk_binary(), "profile"] + cmd
return self._execute(cmd)

def create_profile(self, name):
cmd = ["create", name]
return self._run_profile(cmd)

def create_profile_simple(self, name, cfg_location):
return self._execute(['create', name, "--from-simple", cfg_location])

def use_profile(self, name):
cmd = ["use", name]
return self._run_profile(cmd)

def delete_profile(self, name):
cmd = ["delete", name]
return self._run_profile(cmd)

def rename_profile(self, new_name):
cmd = ["rename-to", new_name]
return self._run_profile(cmd)

def set_profile(self, kv):
cmd = ["set", kv]
return self._run_profile(cmd)

def list_profiles(self):
cmd = ["list"]
out = self._run_profile(cmd)
lines = out.splitlines()
if len(lines) == 1:
return []

def profile_line(line):
parts = line.split()
# We remove the asterisk that denotes that is the selected profile. Not needed here.
return parts[0].strip("*")

for i, line in enumerate(lines):
if line.split() == ["NAME", "DESCRIPTION"]:
return list(map(profile_line, lines[i + 1:]))

def create_topic_no_flags(self, name):
cmd = [self._rpk_binary(), "topic", "create", name]
return self._execute(cmd)

def _execute(self, cmd, timeout=30):
self._redpanda.logger.debug("Executing command: %s", cmd)

Expand Down
3 changes: 3 additions & 0 deletions tests/rptest/services/redpanda.py
Original file line number Diff line number Diff line change
Expand Up @@ -693,6 +693,7 @@ class RedpandaServiceBase(Service):
TRIM_LOGS_KEY = "trim_logs"
DATA_DIR = os.path.join(PERSISTENT_ROOT, "data")
NODE_CONFIG_FILE = "/etc/redpanda/redpanda.yaml"
RPK_CONFIG_FILE = "/root/.config/rpk/rpk.yaml"
CLUSTER_BOOTSTRAP_CONFIG_FILE = "/etc/redpanda/.bootstrap.yaml"
TLS_SERVER_KEY_FILE = "/etc/redpanda/server.key"
TLS_SERVER_CRT_FILE = "/etc/redpanda/server.crt"
Expand Down Expand Up @@ -2840,6 +2841,8 @@ def clean_node(self,
f"{RedpandaService.PERSISTENT_ROOT}/data/*")
if node.account.exists(RedpandaService.NODE_CONFIG_FILE):
node.account.remove(f"{RedpandaService.NODE_CONFIG_FILE}")
if node.account.exists(RedpandaService.RPK_CONFIG_FILE):
node.account.remove(f"{RedpandaService.RPK_CONFIG_FILE}")
if node.account.exists(RedpandaService.CLUSTER_BOOTSTRAP_CONFIG_FILE):
node.account.remove(
f"{RedpandaService.CLUSTER_BOOTSTRAP_CONFIG_FILE}")
Expand Down
111 changes: 111 additions & 0 deletions tests/rptest/tests/rpk_profile_test.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,111 @@
# Copyright 2023 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.utils.rpk_config import read_rpk_cfg, read_redpanda_cfg
from rptest.util import expect_exception
from rptest.services.cluster import cluster

from rptest.tests.redpanda_test import RedpandaTest
from rptest.clients.rpk_remote import RpkRemoteTool
from rptest.clients.rpk import RpkTool
from ducktape.cluster.remoteaccount import RemoteCommandError
from rptest.services.redpanda import RedpandaService


class RpkProfileTest(RedpandaTest):
def __init__(self, ctx):
super(RpkProfileTest, self).__init__(test_context=ctx)
self._ctx = ctx
self._rpk = RpkTool(self.redpanda)

@cluster(num_nodes=1)
def test_e2e_profile(self):
"""
Test an e2e flow of different operations using rpk profile:
Create 2 -> List -> Use -> Delete -> Use deleted -> Rename
"""
pr1 = "profile_1"
pr2 = "profile_2"
node = self.redpanda.get_node(0)
rpk = RpkRemoteTool(self.redpanda, node)

# Create profiles
rpk.create_profile(pr1)
rpk.create_profile(pr2)

rpk_cfg = read_rpk_cfg(node)

assert rpk_cfg["current_profile"] == pr2
# rpk pushes to the top the last profile used
assert rpk_cfg["profiles"][0]["name"] == pr2
assert rpk_cfg["profiles"][1]["name"] == pr1

# List profiles
profile_list = rpk.list_profiles()
assert len(profile_list) == 2
assert pr1 in profile_list and pr2 in profile_list

# Change selected profile
rpk.use_profile(pr1)
rpk_cfg = read_rpk_cfg(node)
assert rpk_cfg["current_profile"] == pr1

rpk.delete_profile(pr2)
profile_list = rpk.list_profiles()
assert len(profile_list) == 1

# Now we try to use an already deleted profile
with expect_exception(RemoteCommandError,
lambda e: "returned non-zero exit" in str(e)):
rpk.use_profile(pr2)

# Finally, we rename it
rpk.rename_profile("new_name")
rpk_cfg = read_rpk_cfg(node)
assert rpk_cfg["current_profile"] == "new_name"

@cluster(num_nodes=3)
def test_use_profile(self):
"""
Test that creates a profile, assign the brokers and create a
topic without using the --brokers flag that is used in every
ducktape test so far.
"""
node = self.redpanda.get_node(0)
rpk = RpkRemoteTool(self.redpanda, node)
rpk.create_profile("noflag")

rpk.set_profile("brokers=" + self.redpanda.brokers())
rpk.create_topic_no_flags("no-flag-test")

topic_list = self._rpk.list_topics()
assert "no-flag-test" in topic_list

@cluster(num_nodes=3)
def test_create_profile_from_simple(self):
"""
Create redpanda.yaml, use create rpk profile --from simple
"""
node = self.redpanda.get_node(0)
rpk = RpkRemoteTool(self.redpanda, node)

# We set the broker list in the redpanda.yaml
rpk.config_set("rpk.kafka_api.brokers", self.redpanda.brokers_list())

# Then we create the profile based on the redpanda.yaml
rpk.create_profile_simple("simple_test",
RedpandaService.NODE_CONFIG_FILE)

rpk_cfg = read_rpk_cfg(node)
redpanda_cfg = read_redpanda_cfg(node)

rpk_brokers = rpk_cfg["profiles"][0]["kafka_api"]["brokers"]
redpanda_brokers = redpanda_cfg["rpk"]["kafka_api"]["brokers"]

assert rpk_brokers == redpanda_brokers
15 changes: 1 addition & 14 deletions tests/rptest/utils/rpk_config.py
Original file line number Diff line number Diff line change
Expand Up @@ -13,21 +13,8 @@

from rptest.services.redpanda import RedpandaService

# This lives in the runner
RUNNER_RPK_CONFIG_FILE = "/root/.config/rpk/rpk.yaml"


def clean_runner_rpk_cfg():
if os.path.exists(RUNNER_RPK_CONFIG_FILE):
os.remove(RUNNER_RPK_CONFIG_FILE)


def read_rpk_cfg():
with open(RUNNER_RPK_CONFIG_FILE) as f:
return yaml.full_load(f.read())


def read_node_rpk_cfg(node):
def read_rpk_cfg(node):
with tempfile.TemporaryDirectory() as d:
node.account.copy_from(RedpandaService.RPK_CONFIG_FILE, d)
with open(os.path.join(d, 'rpk.yaml')) as f:
Expand Down

0 comments on commit 95f1cfe

Please sign in to comment.