Skip to content

Commit

Permalink
Merge pull request redpanda-data#17352 from dotnwat/test-utils-headers
Browse files Browse the repository at this point in the history
Remove global dependencies from test_utils/random
  • Loading branch information
dotnwat authored Mar 23, 2024
2 parents 0d37b5a + 623cdc5 commit 3e06805
Show file tree
Hide file tree
Showing 17 changed files with 256 additions and 196 deletions.
49 changes: 49 additions & 0 deletions src/v/cluster/tests/randoms.h
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,8 @@
#pragma once

#include "cluster/health_monitor_types.h"
#include "cluster/partition_balancer_state.h"
#include "cluster/partition_balancer_types.h"
#include "cluster/tx_snapshot_utils.h"
#include "model/tests/randoms.h"
#include "random/generators.h"
Expand Down Expand Up @@ -193,3 +195,50 @@ inline tx_snapshot_v3::tx_seqs_snapshot random_tx_seqs_snapshot() {
}

} // namespace cluster

namespace tests {

inline cluster::producer_ptr
random_producer_state(cluster::producer_state_manager& psm) {
return ss::make_lw_shared<cluster::producer_state>(
psm,
model::producer_identity{
random_generators::get_int<int64_t>(),
random_generators::get_int<int16_t>()},
tests::random_named_int<raft::group_id>(),
ss::noncopyable_function<void()>{});
}

inline cluster::partition_balancer_status random_balancer_status() {
return random_generators::random_choice({
cluster::partition_balancer_status::off,
cluster::partition_balancer_status::starting,
cluster::partition_balancer_status::ready,
cluster::partition_balancer_status::in_progress,
cluster::partition_balancer_status::stalled,
});
}

inline cluster::partition_balancer_violations::unavailable_node
random_unavailable_node() {
return {
tests::random_named_int<model::node_id>(),
model::timestamp(random_generators::get_int<int64_t>())};
}

inline cluster::partition_balancer_violations::full_node random_full_node() {
return {
tests::random_named_int<model::node_id>(),
random_generators::get_int<uint32_t>()};
}

inline cluster::partition_balancer_violations
random_partition_balancer_violations() {
auto random_un_gen = tests::random_vector(
[]() { return random_unavailable_node(); });
auto random_fn_gen = tests::random_vector(
[]() { return random_full_node(); });
return {std::move(random_un_gen), std::move(random_fn_gen)};
}

} // namespace tests
1 change: 1 addition & 0 deletions src/v/cluster/tests/serialization_rt_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
#include "random/generators.h"
#include "reflection/adl.h"
#include "reflection/async_adl.h"
#include "security/tests/randoms.h"
#include "storage/types.h"
#include "test_utils/randoms.h"
#include "test_utils/rpc.h"
Expand Down
1 change: 1 addition & 0 deletions src/v/compat/acls_generator.h
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
#include "cluster/types.h"
#include "compat/cluster_generator.h"
#include "compat/generator.h"
#include "security/tests/randoms.h"
#include "test_utils/randoms.h"

namespace compat {
Expand Down
32 changes: 32 additions & 0 deletions src/v/model/tests/randoms.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
#include "model/metadata.h"
#include "model/record.h"
#include "model/timestamp.h"
#include "net/tests/randoms.h"
#include "pandaproxy/schema_registry/subject_name_strategy.h"
#include "random/generators.h"
#include "test_utils/randoms.h"
Expand Down Expand Up @@ -191,3 +192,34 @@ random_subject_name_strategy() {
}

} // namespace model

namespace tests {

inline model::record_batch_type random_batch_type() {
return random_generators::random_choice(
std::vector<model::record_batch_type>{
model::record_batch_type::raft_data,
model::record_batch_type::raft_configuration,
model::record_batch_type::controller,
model::record_batch_type::kvstore,
model::record_batch_type::checkpoint,
model::record_batch_type::topic_management_cmd,
model::record_batch_type::ghost_batch,
model::record_batch_type::id_allocator,
model::record_batch_type::tx_prepare,
model::record_batch_type::tx_fence,
model::record_batch_type::tm_update,
model::record_batch_type::user_management_cmd,
model::record_batch_type::acl_management_cmd,
model::record_batch_type::group_prepare_tx,
model::record_batch_type::group_commit_tx,
model::record_batch_type::group_abort_tx,
model::record_batch_type::node_management_cmd,
model::record_batch_type::data_policy_management_cmd,
model::record_batch_type::archival_metadata,
model::record_batch_type::cluster_config_cmd,
model::record_batch_type::feature_update,
});
}

} // namespace tests
25 changes: 25 additions & 0 deletions src/v/net/tests/randoms.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* Copyright 2020 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
*/
#pragma once

#include "net/unresolved_address.h"
#include "random/generators.h"

namespace tests {

inline net::unresolved_address random_net_address() {
return net::unresolved_address(
random_generators::gen_alphanum_string(
random_generators::get_int(1, 100)),
random_generators::get_int(1025, 65535));
}

} // namespace tests
1 change: 1 addition & 0 deletions src/v/raft/persisted_stm.cc
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@
#include "raft/persisted_stm.h"

#include "bytes/iostream.h"
#include "cluster/types.h"
#include "raft/consensus.h"
#include "raft/errc.h"
#include "raft/offset_monitor.h"
Expand Down
1 change: 1 addition & 0 deletions src/v/raft/tests/configuration_serialization_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
// by the Apache License, Version 2.0

#include "bytes/iobuf_parser.h"
#include "model/adl_serde.h"
#include "model/metadata.h"
#include "model/tests/random_batch.h"
#include "model/tests/randoms.h"
Expand Down
1 change: 1 addition & 0 deletions src/v/security/tests/ephemeral_credential_store_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
#include "security/ephemeral_credential_store.h"
#include "security/scram_authenticator.h"
#include "security/scram_credential.h"
#include "security/tests/randoms.h"
#include "security/types.h"
#include "test_utils/randoms.h"

Expand Down
130 changes: 130 additions & 0 deletions src/v/security/tests/randoms.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,130 @@
/*
* Copyright 2020 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
*/
#pragma once

#include "bytes/random.h"
#include "random/generators.h"
#include "security/acl.h"
#include "security/scram_credential.h"
#include "test_utils/randoms.h"

namespace tests {

inline security::scram_credential random_credential() {
return security::scram_credential(
random_generators::get_bytes(256),
random_generators::get_bytes(256),
random_generators::get_bytes(256),
random_generators::get_int(1, 10));
}

inline security::resource_type random_resource_type() {
return random_generators::random_choice<security::resource_type>(
{security::resource_type::cluster,
security::resource_type::group,
security::resource_type::topic,
security::resource_type::transactional_id});
}

inline security::pattern_type random_pattern_type() {
return random_generators::random_choice<security::pattern_type>(
{security::pattern_type::literal, security::pattern_type::prefixed});
}

inline security::resource_pattern random_resource_pattern() {
return {
random_resource_type(),
random_generators::gen_alphanum_string(10),
random_pattern_type()};
}

inline security::acl_principal random_acl_principal() {
return {
security::principal_type::user,
random_generators::gen_alphanum_string(12)};
}

inline security::acl_host create_acl_host() {
return security::acl_host(ss::net::inet_address("127.0.0.1"));
}

inline security::acl_operation random_acl_operation() {
return random_generators::random_choice<security::acl_operation>(
{security::acl_operation::all,
security::acl_operation::alter,
security::acl_operation::alter_configs,
security::acl_operation::describe_configs,
security::acl_operation::cluster_action,
security::acl_operation::create,
security::acl_operation::remove,
security::acl_operation::read,
security::acl_operation::idempotent_write,
security::acl_operation::describe});
}

inline security::acl_permission random_acl_permission() {
return random_generators::random_choice<security::acl_permission>(
{security::acl_permission::allow, security::acl_permission::deny});
}

inline security::acl_entry random_acl_entry() {
return {
random_acl_principal(),
create_acl_host(),
random_acl_operation(),
random_acl_permission()};
}

inline security::acl_binding random_acl_binding() {
return {random_resource_pattern(), random_acl_entry()};
}

inline security::resource_pattern_filter random_resource_pattern_filter() {
auto resource = tests::random_optional(
[] { return random_resource_type(); });

auto name = tests::random_optional(
[] { return random_generators::gen_alphanum_string(14); });

auto pattern = tests::random_optional([] {
using ret_t = std::variant<
security::pattern_type,
security::resource_pattern_filter::pattern_match>;
if (tests::random_bool()) {
return ret_t(random_pattern_type());
} else {
return ret_t(security::resource_pattern_filter::pattern_match{});
}
});

return {resource, std::move(name), pattern};
}

inline security::acl_entry_filter random_acl_entry_filter() {
auto principal = tests::random_optional(
[] { return random_acl_principal(); });

auto host = tests::random_optional([] { return create_acl_host(); });

auto operation = tests::random_optional(
[] { return random_acl_operation(); });

auto permission = tests::random_optional(
[] { return random_acl_permission(); });

return {std::move(principal), host, operation, permission};
}

inline security::acl_binding_filter random_acl_binding_filter() {
return {random_resource_pattern_filter(), random_acl_entry_filter()};
}

} // namespace tests
4 changes: 2 additions & 2 deletions src/v/storage/log_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,6 @@

#include "base/likely.h"
#include "base/vlog.h"
#include "cluster/cluster_utils.h"
#include "cluster/topic_table.h"
#include "config/configuration.h"
#include "model/fundamental.h"
#include "model/metadata.h"
Expand Down Expand Up @@ -66,6 +64,8 @@
#include <filesystem>
#include <optional>

using namespace std::chrono_literals;

namespace storage {
using logs_type = absl::flat_hash_map<model::ntp, log_housekeeping_meta>;

Expand Down
1 change: 0 additions & 1 deletion src/v/storage/log_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@

#include "base/seastarx.h"
#include "base/units.h"
#include "cluster/topic_table.h"
#include "config/property.h"
#include "container/intrusive_list_helpers.h"
#include "features/feature_table.h"
Expand Down
1 change: 1 addition & 0 deletions src/v/storage/offset_translator.cc
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
#include "storage/offset_translator.h"

#include "base/vlog.h"
#include "reflection/adl.h"
#include "storage/api.h"
#include "storage/kvstore.h"
#include "storage/logger.h"
Expand Down
1 change: 1 addition & 0 deletions src/v/storage/tests/compaction_index_format_tests.cc
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
#include "bytes/bytes.h"
#include "bytes/iobuf_parser.h"
#include "bytes/random.h"
#include "model/tests/randoms.h"
#include "random/generators.h"
#include "reflection/adl.h"
#include "storage/compacted_index.h"
Expand Down
Loading

0 comments on commit 3e06805

Please sign in to comment.