ClickHouse/utils/keeper-bench/Runner.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

140 lines
4.1 KiB
C++
Raw Normal View History

2021-04-12 08:10:23 +00:00
#pragma once
2024-04-10 10:56:29 +00:00
#include "Common/ZooKeeper/ZooKeeperArgs.h"
2021-04-12 08:10:23 +00:00
#include <Common/ZooKeeper/ZooKeeperImpl.h>
#include "Generator.h"
#include <Common/ZooKeeper/IKeeper.h>
2023-04-06 15:10:58 +00:00
#include <Common/Config/ConfigProcessor.h>
2021-04-12 08:10:23 +00:00
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Common/Stopwatch.h>
#include <Common/ThreadPool.h>
2021-04-12 15:40:42 +00:00
#include <Common/InterruptListener.h>
#include <Common/CurrentMetrics.h>
2021-04-12 15:40:42 +00:00
#include <Core/Types.h>
2023-04-07 13:02:42 +00:00
#include <Poco/Util/AbstractConfiguration.h>
2024-04-10 10:56:29 +00:00
#include "Interpreters/Context.h"
2021-04-12 15:40:42 +00:00
#include "Stats.h"
2021-04-12 08:10:23 +00:00
2023-04-17 11:25:46 +00:00
#include <filesystem>
2021-04-12 08:10:23 +00:00
using Ports = std::vector<UInt16>;
2021-04-12 15:40:42 +00:00
using Strings = std::vector<std::string>;
2021-04-12 08:10:23 +00:00
2024-04-10 10:56:29 +00:00
struct BenchmarkContext
{
public:
void initializeFromConfig(const Poco::Util::AbstractConfiguration & config);
void startup(Coordination::ZooKeeper & zookeeper);
void cleanup(Coordination::ZooKeeper & zookeeper);
2024-05-10 12:15:01 +00:00
2024-04-10 10:56:29 +00:00
private:
struct Node
{
StringGetter name;
std::optional<StringGetter> data;
std::vector<std::shared_ptr<Node>> children;
size_t repeat_count = 0;
std::shared_ptr<Node> clone() const;
void createNode(Coordination::ZooKeeper & zookeeper, const std::string & parent_path, const Coordination::ACLs & acls) const;
void dumpTree(int level = 0) const;
};
static std::shared_ptr<Node> parseNode(const std::string & key, const Poco::Util::AbstractConfiguration & config);
std::vector<std::shared_ptr<Node>> root_nodes;
Coordination::ACLs default_acls;
};
2021-04-12 08:10:23 +00:00
class Runner
{
public:
2021-04-12 15:40:42 +00:00
Runner(
2023-04-07 14:14:39 +00:00
std::optional<size_t> concurrency_,
2023-04-06 10:25:37 +00:00
const std::string & config_path,
2024-04-10 10:56:29 +00:00
const std::string & input_request_log_,
2024-05-10 12:15:01 +00:00
const std::string & setup_nodes_snapshot_path_,
2021-04-12 15:40:42 +00:00
const Strings & hosts_strings_,
2023-04-07 14:14:39 +00:00
std::optional<double> max_time_,
std::optional<double> delay_,
std::optional<bool> continue_on_error_,
std::optional<size_t> max_iterations_);
2021-04-12 08:10:23 +00:00
void thread(std::vector<std::shared_ptr<Coordination::ZooKeeper>> zookeepers);
2021-04-12 08:10:23 +00:00
2021-04-12 15:40:42 +00:00
void printNumberOfRequestsExecuted(size_t num)
{
std::cerr << "Requests executed: " << num << ".\n";
}
2023-04-07 14:14:39 +00:00
bool tryPushRequestInteractively(Coordination::ZooKeeperRequestPtr && request, DB::InterruptListener & interrupt_listener);
2021-04-12 15:40:42 +00:00
2021-04-13 11:55:08 +00:00
void runBenchmark();
2021-04-12 08:10:23 +00:00
2023-04-07 14:14:39 +00:00
~Runner();
2021-04-12 08:10:23 +00:00
private:
2024-04-10 10:56:29 +00:00
struct ConnectionInfo
{
std::string host;
bool secure = false;
int32_t session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS;
int32_t connection_timeout_ms = Coordination::DEFAULT_CONNECTION_TIMEOUT_MS;
int32_t operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS;
bool use_compression = false;
size_t sessions = 1;
};
2023-04-07 13:02:42 +00:00
void parseHostsFromConfig(const Poco::Util::AbstractConfiguration & config);
2021-04-12 08:10:23 +00:00
2024-04-10 10:56:29 +00:00
void runBenchmarkWithGenerator();
void runBenchmarkFromLog();
void createConnections();
std::vector<std::shared_ptr<Coordination::ZooKeeper>> refreshConnections();
std::shared_ptr<Coordination::ZooKeeper> getConnection(const ConnectionInfo & connection_info, size_t connection_info_idx);
std::string input_request_log;
2024-05-10 12:15:01 +00:00
std::string setup_nodes_snapshot_path;
2024-04-10 10:56:29 +00:00
2021-04-13 07:33:18 +00:00
size_t concurrency = 1;
2021-04-12 08:10:23 +00:00
2023-04-07 14:14:39 +00:00
std::optional<ThreadPool> pool;
2023-04-07 13:02:42 +00:00
2023-04-07 14:14:39 +00:00
std::optional<Generator> generator;
2021-04-13 07:33:18 +00:00
double max_time = 0;
double delay = 1;
bool continue_on_error = false;
2024-04-10 10:56:29 +00:00
size_t max_iterations = 0;
2021-04-13 07:33:18 +00:00
std::atomic<size_t> requests_executed = 0;
std::atomic<bool> shutdown = false;
2021-04-12 08:10:23 +00:00
2021-04-12 15:40:42 +00:00
std::shared_ptr<Stats> info;
2023-04-17 11:25:46 +00:00
bool print_to_stdout;
std::optional<std::filesystem::path> file_output;
bool output_file_with_timestamp;
2021-04-12 15:40:42 +00:00
Stopwatch total_watch;
Stopwatch delay_watch;
std::mutex mutex;
using Queue = ConcurrentBoundedQueue<Coordination::ZooKeeperRequestPtr>;
2023-04-07 14:14:39 +00:00
std::optional<Queue> queue;
2021-04-12 08:10:23 +00:00
2023-04-07 13:02:42 +00:00
std::mutex connection_mutex;
2024-04-10 10:56:29 +00:00
ConnectionInfo default_connection_info;
2023-04-07 13:02:42 +00:00
std::vector<ConnectionInfo> connection_infos;
std::vector<std::shared_ptr<Coordination::ZooKeeper>> connections;
std::unordered_map<size_t, size_t> connections_to_info_map;
2024-04-10 10:56:29 +00:00
DB::SharedContextHolder shared_context;
DB::ContextMutablePtr global_context;
BenchmarkContext benchmark_context;
2021-04-12 08:10:23 +00:00
};