Merge pull request #4171 from yandex/refactor_performance_test

Refactor performance test
This commit is contained in:
alesapin 2019-01-30 16:54:50 +03:00 committed by GitHub
commit 3de94798d5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 2040 additions and 1465 deletions

View File

@ -1,4 +1,16 @@
add_library (clickhouse-performance-test-lib ${LINK_MODE} PerformanceTest.cpp)
add_library (clickhouse-performance-test-lib ${LINK_MODE}
JSONString.cpp
StopConditionsSet.cpp
TestStopConditions.cpp
TestStats.cpp
ConfigPreprocessor.cpp
PerformanceTest.cpp
PerformanceTestInfo.cpp
executeQuery.cpp
applySubstitutions.cpp
ReportBuilder.cpp
PerformanceTestSuite.cpp
)
target_link_libraries (clickhouse-performance-test-lib PRIVATE dbms clickhouse_common_io clickhouse_common_config ${Boost_PROGRAM_OPTIONS_LIBRARY})
target_include_directories (clickhouse-performance-test-lib SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR})

View File

@ -0,0 +1,85 @@
#include "ConfigPreprocessor.h"
#include <Core/Types.h>
#include <regex>
namespace DB
{
std::vector<XMLConfigurationPtr> ConfigPreprocessor::processConfig(
const Strings & tests_tags,
const Strings & tests_names,
const Strings & tests_names_regexp,
const Strings & skip_tags,
const Strings & skip_names,
const Strings & skip_names_regexp) const
{
std::vector<XMLConfigurationPtr> result;
for (const auto & path : paths)
result.emplace_back(new XMLConfiguration(path));
/// Leave tests:
removeConfigurationsIf(result, FilterType::Tag, tests_tags, true);
removeConfigurationsIf(result, FilterType::Name, tests_names, true);
removeConfigurationsIf(result, FilterType::Name_regexp, tests_names_regexp, true);
/// Skip tests
removeConfigurationsIf(result, FilterType::Tag, skip_tags, false);
removeConfigurationsIf(result, FilterType::Name, skip_names, false);
removeConfigurationsIf(result, FilterType::Name_regexp, skip_names_regexp, false);
return result;
}
void ConfigPreprocessor::removeConfigurationsIf(
std::vector<XMLConfigurationPtr> & configs,
ConfigPreprocessor::FilterType filter_type,
const Strings & values,
bool leave) const
{
auto checker = [&filter_type, &values, &leave] (XMLConfigurationPtr & config)
{
if (values.size() == 0)
return false;
bool remove_or_not = false;
if (filter_type == FilterType::Tag)
{
Strings tags_keys;
config->keys("tags", tags_keys);
Strings tags(tags_keys.size());
for (size_t i = 0; i != tags_keys.size(); ++i)
tags[i] = config->getString("tags.tag[" + std::to_string(i) + "]");
for (const std::string & config_tag : tags)
{
if (std::find(values.begin(), values.end(), config_tag) != values.end())
remove_or_not = true;
}
}
if (filter_type == FilterType::Name)
{
remove_or_not = (std::find(values.begin(), values.end(), config->getString("name", "")) != values.end());
}
if (filter_type == FilterType::Name_regexp)
{
std::string config_name = config->getString("name", "");
auto regex_checker = [&config_name](const std::string & name_regexp)
{
std::regex pattern(name_regexp);
return std::regex_search(config_name, pattern);
};
remove_or_not = config->has("name") ? (std::find_if(values.begin(), values.end(), regex_checker) != values.end()) : false;
}
if (leave)
remove_or_not = !remove_or_not;
return remove_or_not;
};
auto new_end = std::remove_if(configs.begin(), configs.end(), checker);
configs.erase(new_end, configs.end());
}
}

View File

@ -0,0 +1,50 @@
#pragma once
#include <Poco/DOM/Document.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Core/Types.h>
#include <vector>
#include <string>
namespace DB
{
using XMLConfiguration = Poco::Util::XMLConfiguration;
using XMLConfigurationPtr = Poco::AutoPtr<XMLConfiguration>;
using XMLDocumentPtr = Poco::AutoPtr<Poco::XML::Document>;
class ConfigPreprocessor
{
public:
ConfigPreprocessor(const Strings & paths_)
: paths(paths_)
{}
std::vector<XMLConfigurationPtr> processConfig(
const Strings & tests_tags,
const Strings & tests_names,
const Strings & tests_names_regexp,
const Strings & skip_tags,
const Strings & skip_names,
const Strings & skip_names_regexp) const;
private:
enum class FilterType
{
Tag,
Name,
Name_regexp
};
/// Removes configurations that has a given value.
/// If leave is true, the logic is reversed.
void removeConfigurationsIf(
std::vector<XMLConfigurationPtr> & configs,
FilterType filter_type,
const Strings & values,
bool leave = false) const;
const Strings paths;
};
}

View File

@ -0,0 +1,66 @@
#include "JSONString.h"
#include <regex>
#include <sstream>
namespace DB
{
namespace
{
std::string pad(size_t padding)
{
return std::string(padding * 4, ' ');
}
const std::regex NEW_LINE{"\n"};
}
void JSONString::set(const std::string & key, std::string value, bool wrap)
{
if (value.empty())
value = "null";
bool reserved = (value[0] == '[' || value[0] == '{' || value == "null");
if (!reserved && wrap)
value = '"' + std::regex_replace(value, NEW_LINE, "\\n") + '"';
content[key] = value;
}
void JSONString::set(const std::string & key, const std::vector<JSONString> & run_infos)
{
std::ostringstream value;
value << "[\n";
for (size_t i = 0; i < run_infos.size(); ++i)
{
value << pad(padding + 1) + run_infos[i].asString(padding + 2);
if (i != run_infos.size() - 1)
value << ',';
value << "\n";
}
value << pad(padding) << ']';
content[key] = value.str();
}
std::string JSONString::asString(size_t cur_padding) const
{
std::ostringstream repr;
repr << "{";
for (auto it = content.begin(); it != content.end(); ++it)
{
if (it != content.begin())
repr << ',';
/// construct "key": "value" string with padding
repr << "\n" << pad(cur_padding) << '"' << it->first << '"' << ": " << it->second;
}
repr << "\n" << pad(cur_padding - 1) << '}';
return repr.str();
}
}

View File

@ -0,0 +1,40 @@
#pragma once
#include <Core/Types.h>
#include <sys/stat.h>
#include <type_traits>
#include <vector>
#include <map>
namespace DB
{
/// NOTE The code is totally wrong.
class JSONString
{
private:
std::map<std::string, std::string> content;
size_t padding;
public:
explicit JSONString(size_t padding_ = 1) : padding(padding_) {}
void set(const std::string & key, std::string value, bool wrap = true);
template <typename T>
std::enable_if_t<std::is_arithmetic_v<T>> set(const std::string key, T value)
{
set(key, std::to_string(value), /*wrap= */ false);
}
void set(const std::string & key, const std::vector<JSONString> & run_infos);
std::string asString() const
{
return asString(padding);
}
std::string asString(size_t cur_padding) const;
};
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,59 @@
#pragma once
#include <Client/Connection.h>
#include <Common/InterruptListener.h>
#include <common/logger_useful.h>
#include <Poco/Util/XMLConfiguration.h>
#include "PerformanceTestInfo.h"
namespace DB
{
using XMLConfiguration = Poco::Util::XMLConfiguration;
using XMLConfigurationPtr = Poco::AutoPtr<XMLConfiguration>;
using QueriesWithIndexes = std::vector<std::pair<std::string, size_t>>;
class PerformanceTest
{
public:
PerformanceTest(
const XMLConfigurationPtr & config_,
Connection & connection_,
InterruptListener & interrupt_listener_,
const PerformanceTestInfo & test_info_,
Context & context_);
bool checkPreconditions() const;
std::vector<TestStats> execute();
const PerformanceTestInfo & getTestInfo() const
{
return test_info;
}
bool checkSIGINT() const
{
return got_SIGINT;
}
private:
void runQueries(
const QueriesWithIndexes & queries_with_indexes,
std::vector<TestStats> & statistics_by_run);
UInt64 calculateMaxExecTime() const;
private:
XMLConfigurationPtr config;
Connection & connection;
InterruptListener & interrupt_listener;
PerformanceTestInfo test_info;
Context & context;
Poco::Logger * log;
bool got_SIGINT = false;
};
}

View File

@ -0,0 +1,271 @@
#include "PerformanceTestInfo.h"
#include <Common/getMultipleKeysFromConfig.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFile.h>
#include <boost/filesystem.hpp>
#include "applySubstitutions.h"
#include <iostream>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
namespace
{
void extractSettings(
const XMLConfigurationPtr & config,
const std::string & key,
const Strings & settings_list,
std::map<std::string, std::string> & settings_to_apply)
{
for (const std::string & setup : settings_list)
{
if (setup == "profile")
continue;
std::string value = config->getString(key + "." + setup);
if (value.empty())
value = "true";
settings_to_apply[setup] = value;
}
}
void checkMetricsInput(const Strings & metrics, ExecutionType exec_type)
{
Strings loop_metrics = {
"min_time", "quantiles", "total_time",
"queries_per_second", "rows_per_second",
"bytes_per_second"};
Strings non_loop_metrics = {
"max_rows_per_second", "max_bytes_per_second",
"avg_rows_per_second", "avg_bytes_per_second"};
if (exec_type == ExecutionType::Loop)
{
for (const std::string & metric : metrics)
{
auto non_loop_pos =
std::find(non_loop_metrics.begin(), non_loop_metrics.end(), metric);
if (non_loop_pos != non_loop_metrics.end())
throw Exception("Wrong type of metric for loop execution type (" + metric + ")",
ErrorCodes::BAD_ARGUMENTS);
}
}
else
{
for (const std::string & metric : metrics)
{
auto loop_pos = std::find(loop_metrics.begin(), loop_metrics.end(), metric);
if (loop_pos != loop_metrics.end())
throw Exception(
"Wrong type of metric for non-loop execution type (" + metric + ")",
ErrorCodes::BAD_ARGUMENTS);
}
}
}
}
namespace fs = boost::filesystem;
PerformanceTestInfo::PerformanceTestInfo(
XMLConfigurationPtr config,
const std::string & profiles_file_)
: profiles_file(profiles_file_)
{
test_name = config->getString("name");
applySettings(config);
extractQueries(config);
processSubstitutions(config);
getExecutionType(config);
getStopConditions(config);
getMetrics(config);
}
void PerformanceTestInfo::applySettings(XMLConfigurationPtr config)
{
if (config->has("settings"))
{
std::map<std::string, std::string> settings_to_apply;
Strings config_settings;
config->keys("settings", config_settings);
auto settings_contain = [&config_settings] (const std::string & setting)
{
auto position = std::find(config_settings.begin(), config_settings.end(), setting);
return position != config_settings.end();
};
/// Preprocess configuration file
if (settings_contain("profile"))
{
if (!profiles_file.empty())
{
std::string profile_name = config->getString("settings.profile");
XMLConfigurationPtr profiles_config(new XMLConfiguration(profiles_file));
Strings profile_settings;
profiles_config->keys("profiles." + profile_name, profile_settings);
extractSettings(profiles_config, "profiles." + profile_name, profile_settings, settings_to_apply);
}
}
extractSettings(config, "settings", config_settings, settings_to_apply);
/// This macro goes through all settings in the Settings.h
/// and, if found any settings in test's xml configuration
/// with the same name, sets its value to settings
std::map<std::string, std::string>::iterator it;
#define EXTRACT_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) \
it = settings_to_apply.find(#NAME); \
if (it != settings_to_apply.end()) \
settings.set(#NAME, settings_to_apply[#NAME]);
APPLY_FOR_SETTINGS(EXTRACT_SETTING)
#undef EXTRACT_SETTING
if (settings_contain("average_rows_speed_precision"))
TestStats::avg_rows_speed_precision =
config->getDouble("settings.average_rows_speed_precision");
if (settings_contain("average_bytes_speed_precision"))
TestStats::avg_bytes_speed_precision =
config->getDouble("settings.average_bytes_speed_precision");
}
}
void PerformanceTestInfo::extractQueries(XMLConfigurationPtr config)
{
if (config->has("query"))
queries = getMultipleValuesFromConfig(*config, "", "query");
if (config->has("query_file"))
{
const std::string filename = config->getString("query_file");
if (filename.empty())
throw Exception("Empty file name", ErrorCodes::BAD_ARGUMENTS);
bool tsv = fs::path(filename).extension().string() == ".tsv";
ReadBufferFromFile query_file(filename);
std::string query;
if (tsv)
{
while (!query_file.eof())
{
readEscapedString(query, query_file);
assertChar('\n', query_file);
queries.push_back(query);
}
}
else
{
readStringUntilEOF(query, query_file);
queries.push_back(query);
}
}
if (queries.empty())
throw Exception("Did not find any query to execute: " + test_name,
ErrorCodes::BAD_ARGUMENTS);
}
void PerformanceTestInfo::processSubstitutions(XMLConfigurationPtr config)
{
if (config->has("substitutions"))
{
/// Make "subconfig" of inner xml block
ConfigurationPtr substitutions_view(config->createView("substitutions"));
constructSubstitutions(substitutions_view, substitutions);
auto queries_pre_format = queries;
queries.clear();
for (const auto & query : queries_pre_format)
{
auto formatted = formatQueries(query, substitutions);
queries.insert(queries.end(), formatted.begin(), formatted.end());
}
}
}
void PerformanceTestInfo::getExecutionType(XMLConfigurationPtr config)
{
if (!config->has("type"))
throw Exception("Missing type property in config: " + test_name,
ErrorCodes::BAD_ARGUMENTS);
std::string config_exec_type = config->getString("type");
if (config_exec_type == "loop")
exec_type = ExecutionType::Loop;
else if (config_exec_type == "once")
exec_type = ExecutionType::Once;
else
throw Exception("Unknown type " + config_exec_type + " in :" + test_name,
ErrorCodes::BAD_ARGUMENTS);
}
void PerformanceTestInfo::getStopConditions(XMLConfigurationPtr config)
{
TestStopConditions stop_conditions_template;
if (config->has("stop_conditions"))
{
ConfigurationPtr stop_conditions_config(config->createView("stop_conditions"));
stop_conditions_template.loadFromConfig(stop_conditions_config);
}
if (stop_conditions_template.empty())
throw Exception("No termination conditions were found in config",
ErrorCodes::BAD_ARGUMENTS);
times_to_run = config->getUInt("times_to_run", 1);
for (size_t i = 0; i < times_to_run * queries.size(); ++i)
stop_conditions_by_run.push_back(stop_conditions_template);
}
void PerformanceTestInfo::getMetrics(XMLConfigurationPtr config)
{
ConfigurationPtr metrics_view(config->createView("metrics"));
metrics_view->keys(metrics);
if (config->has("main_metric"))
{
Strings main_metrics;
config->keys("main_metric", main_metrics);
if (main_metrics.size())
main_metric = main_metrics[0];
}
if (!main_metric.empty())
{
if (std::find(metrics.begin(), metrics.end(), main_metric) == metrics.end())
metrics.push_back(main_metric);
}
else
{
if (metrics.empty())
throw Exception("You shoud specify at least one metric",
ErrorCodes::BAD_ARGUMENTS);
main_metric = metrics[0];
}
if (metrics.size() > 0)
checkMetricsInput(metrics, exec_type);
}
}

View File

@ -0,0 +1,54 @@
#pragma once
#include <string>
#include <vector>
#include <map>
#include <Interpreters/Settings.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/AutoPtr.h>
#include "StopConditionsSet.h"
#include "TestStopConditions.h"
#include "TestStats.h"
namespace DB
{
enum class ExecutionType
{
Loop,
Once
};
using XMLConfiguration = Poco::Util::XMLConfiguration;
using XMLConfigurationPtr = Poco::AutoPtr<XMLConfiguration>;
using StringToVector = std::map<std::string, Strings>;
/// Class containing all info to run performance test
class PerformanceTestInfo
{
public:
PerformanceTestInfo(XMLConfigurationPtr config, const std::string & profiles_file_);
std::string test_name;
std::string main_metric;
Strings queries;
Strings metrics;
Settings settings;
ExecutionType exec_type;
StringToVector substitutions;
size_t times_to_run;
std::string profiles_file;
std::vector<TestStopConditions> stop_conditions_by_run;
private:
void applySettings(XMLConfigurationPtr config);
void extractQueries(XMLConfigurationPtr config);
void processSubstitutions(XMLConfigurationPtr config);
void getExecutionType(XMLConfigurationPtr config);
void getStopConditions(XMLConfigurationPtr config);
void getMetrics(XMLConfigurationPtr config);
};
}

View File

@ -0,0 +1,382 @@
#include <algorithm>
#include <iostream>
#include <limits>
#include <regex>
#include <thread>
#include <memory>
#include <port/unistd.h>
#include <sys/stat.h>
#include <boost/filesystem.hpp>
#include <boost/program_options.hpp>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/Logger.h>
#include <Poco/ConsoleChannel.h>
#include <Poco/FormattingChannel.h>
#include <Poco/PatternFormatter.h>
#include <common/logger_useful.h>
#include <Client/Connection.h>
#include <Core/Types.h>
#include <Interpreters/Context.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/UseSSL.h>
#include <Interpreters/Settings.h>
#include <Poco/AutoPtr.h>
#include <Common/Exception.h>
#include <Common/InterruptListener.h>
#include "TestStopConditions.h"
#include "TestStats.h"
#include "ConfigPreprocessor.h"
#include "PerformanceTest.h"
#include "ReportBuilder.h"
namespace fs = boost::filesystem;
namespace po = boost::program_options;
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int FILE_DOESNT_EXIST;
}
/** Tests launcher for ClickHouse.
* The tool walks through given or default folder in order to find files with
* tests' descriptions and launches it.
*/
class PerformanceTestSuite
{
public:
PerformanceTestSuite(const std::string & host_,
const UInt16 port_,
const bool secure_,
const std::string & default_database_,
const std::string & user_,
const std::string & password_,
const bool lite_output_,
const std::string & profiles_file_,
Strings && input_files_,
Strings && tests_tags_,
Strings && skip_tags_,
Strings && tests_names_,
Strings && skip_names_,
Strings && tests_names_regexp_,
Strings && skip_names_regexp_,
const ConnectionTimeouts & timeouts)
: connection(host_, port_, default_database_, user_,
password_, timeouts, "performance-test", Protocol::Compression::Enable,
secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable)
, tests_tags(std::move(tests_tags_))
, tests_names(std::move(tests_names_))
, tests_names_regexp(std::move(tests_names_regexp_))
, skip_tags(std::move(skip_tags_))
, skip_names(std::move(skip_names_))
, skip_names_regexp(std::move(skip_names_regexp_))
, lite_output(lite_output_)
, profiles_file(profiles_file_)
, input_files(input_files_)
, log(&Poco::Logger::get("PerformanceTestSuite"))
{
if (input_files.size() < 1)
throw Exception("No tests were specified", ErrorCodes::BAD_ARGUMENTS);
}
/// This functionality seems strange.
//void initialize(Poco::Util::Application & self [[maybe_unused]])
//{
// std::string home_path;
// const char * home_path_cstr = getenv("HOME");
// if (home_path_cstr)
// home_path = home_path_cstr;
// configReadClient(Poco::Util::Application::instance().config(), home_path);
//}
int run()
{
std::string name;
UInt64 version_major;
UInt64 version_minor;
UInt64 version_patch;
UInt64 version_revision;
connection.getServerVersion(name, version_major, version_minor, version_patch, version_revision);
std::stringstream ss;
ss << version_major << "." << version_minor << "." << version_patch;
server_version = ss.str();
report_builder = std::make_shared<ReportBuilder>(server_version);
processTestsConfigurations(input_files);
return 0;
}
private:
Connection connection;
const Strings & tests_tags;
const Strings & tests_names;
const Strings & tests_names_regexp;
const Strings & skip_tags;
const Strings & skip_names;
const Strings & skip_names_regexp;
Context global_context = Context::createGlobal();
std::shared_ptr<ReportBuilder> report_builder;
std::string server_version;
InterruptListener interrupt_listener;
using XMLConfiguration = Poco::Util::XMLConfiguration;
using XMLConfigurationPtr = Poco::AutoPtr<XMLConfiguration>;
bool lite_output;
std::string profiles_file;
Strings input_files;
std::vector<XMLConfigurationPtr> tests_configurations;
Poco::Logger * log;
void processTestsConfigurations(const Strings & paths)
{
LOG_INFO(log, "Preparing test configurations");
ConfigPreprocessor config_prep(paths);
tests_configurations = config_prep.processConfig(
tests_tags,
tests_names,
tests_names_regexp,
skip_tags,
skip_names,
skip_names_regexp);
LOG_INFO(log, "Test configurations prepared");
if (tests_configurations.size())
{
Strings outputs;
for (auto & test_config : tests_configurations)
{
auto [output, signal] = runTest(test_config);
if (lite_output)
std::cout << output;
else
outputs.push_back(output);
if (signal)
break;
}
if (!lite_output && outputs.size())
{
std::cout << "[" << std::endl;
for (size_t i = 0; i != outputs.size(); ++i)
{
std::cout << outputs[i];
if (i != outputs.size() - 1)
std::cout << ",";
std::cout << std::endl;
}
std::cout << "]" << std::endl;
}
}
}
std::pair<std::string, bool> runTest(XMLConfigurationPtr & test_config)
{
PerformanceTestInfo info(test_config, profiles_file);
LOG_INFO(log, "Config for test '" << info.test_name << "' parsed");
PerformanceTest current(test_config, connection, interrupt_listener, info, global_context);
current.checkPreconditions();
LOG_INFO(log, "Preconditions for test '" << info.test_name << "' are fullfilled");
LOG_INFO(log, "Running test '" << info.test_name << "'");
auto result = current.execute();
LOG_INFO(log, "Test '" << info.test_name << "' finished");
if (lite_output)
return {report_builder->buildCompactReport(info, result), current.checkSIGINT()};
else
return {report_builder->buildFullReport(info, result), current.checkSIGINT()};
}
};
}
static void getFilesFromDir(const fs::path & dir, std::vector<std::string> & input_files, const bool recursive = false)
{
Poco::Logger * log = &Poco::Logger::get("PerformanceTestSuite");
if (dir.extension().string() == ".xml")
LOG_WARNING(log, dir.string() + "' is a directory, but has .xml extension");
fs::directory_iterator end;
for (fs::directory_iterator it(dir); it != end; ++it)
{
const fs::path file = (*it);
if (recursive && fs::is_directory(file))
getFilesFromDir(file, input_files, recursive);
else if (!fs::is_directory(file) && file.extension().string() == ".xml")
input_files.push_back(file.string());
}
}
static std::vector<std::string> getInputFiles(const po::variables_map & options, Poco::Logger * log)
{
std::vector<std::string> input_files;
bool recursive = options.count("recursive");
if (!options.count("input-files"))
{
LOG_INFO(log, "Trying to find test scenario files in the current folder...");
fs::path curr_dir(".");
getFilesFromDir(curr_dir, input_files, recursive);
if (input_files.empty())
throw DB::Exception("Did not find any xml files", DB::ErrorCodes::BAD_ARGUMENTS);
else
LOG_INFO(log, "Found " << input_files.size() << " files");
}
else
{
input_files = options["input-files"].as<std::vector<std::string>>();
LOG_INFO(log, "Found " + std::to_string(input_files.size()) + " input files");
std::vector<std::string> collected_files;
for (const std::string & filename : input_files)
{
fs::path file(filename);
if (!fs::exists(file))
throw DB::Exception("File '" + filename + "' does not exist", DB::ErrorCodes::FILE_DOESNT_EXIST);
if (fs::is_directory(file))
{
getFilesFromDir(file, collected_files, recursive);
}
else
{
if (file.extension().string() != ".xml")
throw DB::Exception("File '" + filename + "' does not have .xml extension", DB::ErrorCodes::BAD_ARGUMENTS);
collected_files.push_back(filename);
}
}
input_files = std::move(collected_files);
}
std::sort(input_files.begin(), input_files.end());
return input_files;
}
int mainEntryClickHousePerformanceTest(int argc, char ** argv)
try
{
using po::value;
using Strings = DB::Strings;
po::options_description desc("Allowed options");
desc.add_options()
("help", "produce help message")
("lite", "use lite version of output")
("profiles-file", value<std::string>()->default_value(""), "Specify a file with global profiles")
("host,h", value<std::string>()->default_value("localhost"), "")
("port", value<UInt16>()->default_value(9000), "")
("secure,s", "Use TLS connection")
("database", value<std::string>()->default_value("default"), "")
("user", value<std::string>()->default_value("default"), "")
("password", value<std::string>()->default_value(""), "")
("log-level", value<std::string>()->default_value("information"), "Set log level")
("tags", value<Strings>()->multitoken(), "Run only tests with tag")
("skip-tags", value<Strings>()->multitoken(), "Do not run tests with tag")
("names", value<Strings>()->multitoken(), "Run tests with specific name")
("skip-names", value<Strings>()->multitoken(), "Do not run tests with name")
("names-regexp", value<Strings>()->multitoken(), "Run tests with names matching regexp")
("skip-names-regexp", value<Strings>()->multitoken(), "Do not run tests with names matching regexp")
("recursive,r", "Recurse in directories to find all xml's");
/// These options will not be displayed in --help
po::options_description hidden("Hidden options");
hidden.add_options()
("input-files", value<std::vector<std::string>>(), "");
/// But they will be legit, though. And they must be given without name
po::positional_options_description positional;
positional.add("input-files", -1);
po::options_description cmdline_options;
cmdline_options.add(desc).add(hidden);
po::variables_map options;
po::store(
po::command_line_parser(argc, argv).
options(cmdline_options).positional(positional).run(), options);
po::notify(options);
Poco::AutoPtr<Poco::PatternFormatter> formatter(new Poco::PatternFormatter("%Y.%m.%d %H:%M:%S.%F <%p> %s: %t"));
Poco::AutoPtr<Poco::ConsoleChannel> console_chanel(new Poco::ConsoleChannel);
Poco::AutoPtr<Poco::FormattingChannel> channel(new Poco::FormattingChannel(formatter, console_chanel));
Poco::Logger::root().setLevel(options["log-level"].as<std::string>());
Poco::Logger::root().setChannel(channel);
Poco::Logger * log = &Poco::Logger::get("PerformanceTestSuite");
if (options.count("help"))
{
std::cout << "Usage: " << argv[0] << " [options] [test_file ...] [tests_folder]\n";
std::cout << desc << "\n";
return 0;
}
Strings input_files = getInputFiles(options, log);
Strings tests_tags = options.count("tags") ? options["tags"].as<Strings>() : Strings({});
Strings skip_tags = options.count("skip-tags") ? options["skip-tags"].as<Strings>() : Strings({});
Strings tests_names = options.count("names") ? options["names"].as<Strings>() : Strings({});
Strings skip_names = options.count("skip-names") ? options["skip-names"].as<Strings>() : Strings({});
Strings tests_names_regexp = options.count("names-regexp") ? options["names-regexp"].as<Strings>() : Strings({});
Strings skip_names_regexp = options.count("skip-names-regexp") ? options["skip-names-regexp"].as<Strings>() : Strings({});
auto timeouts = DB::ConnectionTimeouts::getTCPTimeoutsWithoutFailover(DB::Settings());
DB::UseSSL use_ssl;
DB::PerformanceTestSuite performance_test_suite(
options["host"].as<std::string>(),
options["port"].as<UInt16>(),
options.count("secure"),
options["database"].as<std::string>(),
options["user"].as<std::string>(),
options["password"].as<std::string>(),
options.count("lite") > 0,
options["profiles-file"].as<std::string>(),
std::move(input_files),
std::move(tests_tags),
std::move(skip_tags),
std::move(tests_names),
std::move(skip_names),
std::move(tests_names_regexp),
std::move(skip_names_regexp),
timeouts);
return performance_test_suite.run();
}
catch (...)
{
std::cout << DB::getCurrentExceptionMessage(/*with stacktrace = */ true) << std::endl;
int code = DB::getCurrentExceptionCode();
return code ? code : 1;
}

View File

@ -0,0 +1,195 @@
#include "ReportBuilder.h"
#include <algorithm>
#include <regex>
#include <sstream>
#include <thread>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/getFQDNOrHostName.h>
#include <common/getMemoryAmount.h>
#include "JSONString.h"
namespace DB
{
namespace
{
const std::regex QUOTE_REGEX{"\""};
}
ReportBuilder::ReportBuilder(const std::string & server_version_)
: server_version(server_version_)
, hostname(getFQDNOrHostName())
, num_cores(getNumberOfPhysicalCPUCores())
, num_threads(std::thread::hardware_concurrency())
, ram(getMemoryAmount())
{
}
std::string ReportBuilder::getCurrentTime() const
{
return DateLUT::instance().timeToString(time(nullptr));
}
std::string ReportBuilder::buildFullReport(
const PerformanceTestInfo & test_info,
std::vector<TestStats> & stats) const
{
JSONString json_output;
json_output.set("hostname", hostname);
json_output.set("num_cores", num_cores);
json_output.set("num_threads", num_threads);
json_output.set("ram", ram);
json_output.set("server_version", server_version);
json_output.set("time", getCurrentTime());
json_output.set("test_name", test_info.test_name);
json_output.set("main_metric", test_info.main_metric);
auto has_metric = [&test_info] (const std::string & metric_name)
{
return std::find(test_info.metrics.begin(),
test_info.metrics.end(), metric_name) != test_info.metrics.end();
};
if (test_info.substitutions.size())
{
JSONString json_parameters(2); /// here, 2 is the size of \t padding
for (auto it = test_info.substitutions.begin(); it != test_info.substitutions.end(); ++it)
{
std::string parameter = it->first;
Strings values = it->second;
std::ostringstream array_string;
array_string << "[";
for (size_t i = 0; i != values.size(); ++i)
{
array_string << '"' << std::regex_replace(values[i], QUOTE_REGEX, "\\\"") << '"';
if (i != values.size() - 1)
{
array_string << ", ";
}
}
array_string << ']';
json_parameters.set(parameter, array_string.str());
}
json_output.set("parameters", json_parameters.asString());
}
std::vector<JSONString> run_infos;
for (size_t query_index = 0; query_index < test_info.queries.size(); ++query_index)
{
for (size_t number_of_launch = 0; number_of_launch < test_info.times_to_run; ++number_of_launch)
{
size_t stat_index = number_of_launch * test_info.queries.size() + query_index;
TestStats & statistics = stats[stat_index];
if (!statistics.ready)
continue;
JSONString runJSON;
auto query = std::regex_replace(test_info.queries[query_index], QUOTE_REGEX, "\\\"");
runJSON.set("query", query);
if (!statistics.exception.empty())
runJSON.set("exception", statistics.exception);
if (test_info.exec_type == ExecutionType::Loop)
{
/// in seconds
if (has_metric("min_time"))
runJSON.set("min_time", statistics.min_time / double(1000));
if (has_metric("quantiles"))
{
JSONString quantiles(4); /// here, 4 is the size of \t padding
for (double percent = 10; percent <= 90; percent += 10)
{
std::string quantile_key = std::to_string(percent / 100.0);
while (quantile_key.back() == '0')
quantile_key.pop_back();
quantiles.set(quantile_key,
statistics.sampler.quantileInterpolated(percent / 100.0));
}
quantiles.set("0.95",
statistics.sampler.quantileInterpolated(95 / 100.0));
quantiles.set("0.99",
statistics.sampler.quantileInterpolated(99 / 100.0));
quantiles.set("0.999",
statistics.sampler.quantileInterpolated(99.9 / 100.0));
quantiles.set("0.9999",
statistics.sampler.quantileInterpolated(99.99 / 100.0));
runJSON.set("quantiles", quantiles.asString());
}
if (has_metric("total_time"))
runJSON.set("total_time", statistics.total_time);
if (has_metric("queries_per_second"))
runJSON.set("queries_per_second",
double(statistics.queries) / statistics.total_time);
if (has_metric("rows_per_second"))
runJSON.set("rows_per_second",
double(statistics.total_rows_read) / statistics.total_time);
if (has_metric("bytes_per_second"))
runJSON.set("bytes_per_second",
double(statistics.total_bytes_read) / statistics.total_time);
}
else
{
if (has_metric("max_rows_per_second"))
runJSON.set("max_rows_per_second", statistics.max_rows_speed);
if (has_metric("max_bytes_per_second"))
runJSON.set("max_bytes_per_second", statistics.max_bytes_speed);
if (has_metric("avg_rows_per_second"))
runJSON.set("avg_rows_per_second", statistics.avg_rows_speed_value);
if (has_metric("avg_bytes_per_second"))
runJSON.set("avg_bytes_per_second", statistics.avg_bytes_speed_value);
}
run_infos.push_back(runJSON);
}
}
json_output.set("runs", run_infos);
return json_output.asString();
}
std::string ReportBuilder::buildCompactReport(
const PerformanceTestInfo & test_info,
std::vector<TestStats> & stats) const
{
std::ostringstream output;
for (size_t query_index = 0; query_index < test_info.queries.size(); ++query_index)
{
for (size_t number_of_launch = 0; number_of_launch < test_info.times_to_run; ++number_of_launch)
{
if (test_info.queries.size() > 1)
output << "query \"" << test_info.queries[query_index] << "\", ";
output << "run " << std::to_string(number_of_launch + 1) << ": ";
output << test_info.main_metric << " = ";
size_t index = number_of_launch * test_info.queries.size() + query_index;
output << stats[index].getStatisticByName(test_info.main_metric);
output << "\n";
}
}
return output.str();
}
}

View File

@ -0,0 +1,32 @@
#pragma once
#include "PerformanceTestInfo.h"
#include <vector>
#include <string>
namespace DB
{
class ReportBuilder
{
public:
explicit ReportBuilder(const std::string & server_version_);
std::string buildFullReport(
const PerformanceTestInfo & test_info,
std::vector<TestStats> & stats) const;
std::string buildCompactReport(
const PerformanceTestInfo & test_info,
std::vector<TestStats> & stats) const;
private:
std::string server_version;
std::string hostname;
size_t num_cores;
size_t num_threads;
size_t ram;
private:
std::string getCurrentTime() const;
};
}

View File

@ -0,0 +1,63 @@
#include "StopConditionsSet.h"
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
void StopConditionsSet::loadFromConfig(const ConfigurationPtr & stop_conditions_view)
{
Strings keys;
stop_conditions_view->keys(keys);
for (const std::string & key : keys)
{
if (key == "total_time_ms")
total_time_ms.value = stop_conditions_view->getUInt64(key);
else if (key == "rows_read")
rows_read.value = stop_conditions_view->getUInt64(key);
else if (key == "bytes_read_uncompressed")
bytes_read_uncompressed.value = stop_conditions_view->getUInt64(key);
else if (key == "iterations")
iterations.value = stop_conditions_view->getUInt64(key);
else if (key == "min_time_not_changing_for_ms")
min_time_not_changing_for_ms.value = stop_conditions_view->getUInt64(key);
else if (key == "max_speed_not_changing_for_ms")
max_speed_not_changing_for_ms.value = stop_conditions_view->getUInt64(key);
else if (key == "average_speed_not_changing_for_ms")
average_speed_not_changing_for_ms.value = stop_conditions_view->getUInt64(key);
else
throw Exception("Met unkown stop condition: " + key, ErrorCodes::LOGICAL_ERROR);
}
++initialized_count;
}
void StopConditionsSet::reset()
{
total_time_ms.fulfilled = false;
rows_read.fulfilled = false;
bytes_read_uncompressed.fulfilled = false;
iterations.fulfilled = false;
min_time_not_changing_for_ms.fulfilled = false;
max_speed_not_changing_for_ms.fulfilled = false;
average_speed_not_changing_for_ms.fulfilled = false;
fulfilled_count = 0;
}
void StopConditionsSet::report(UInt64 value, StopConditionsSet::StopCondition & condition)
{
if (condition.value && !condition.fulfilled && value >= condition.value)
{
condition.fulfilled = true;
++fulfilled_count;
}
}
}

View File

@ -0,0 +1,39 @@
#pragma once
#include <Core/Types.h>
#include <Poco/Util/XMLConfiguration.h>
namespace DB
{
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
/// A set of supported stop conditions.
struct StopConditionsSet
{
void loadFromConfig(const ConfigurationPtr & stop_conditions_view);
void reset();
/// Note: only conditions with UInt64 minimal thresholds are supported.
/// I.e. condition is fulfilled when value is exceeded.
struct StopCondition
{
UInt64 value = 0;
bool fulfilled = false;
};
void report(UInt64 value, StopCondition & condition);
StopCondition total_time_ms;
StopCondition rows_read;
StopCondition bytes_read_uncompressed;
StopCondition iterations;
StopCondition min_time_not_changing_for_ms;
StopCondition max_speed_not_changing_for_ms;
StopCondition average_speed_not_changing_for_ms;
size_t initialized_count = 0;
size_t fulfilled_count = 0;
};
}

View File

@ -0,0 +1,165 @@
#include "TestStats.h"
namespace DB
{
namespace
{
const std::string FOUR_SPACES = " ";
}
std::string TestStats::getStatisticByName(const std::string & statistic_name)
{
if (statistic_name == "min_time")
return std::to_string(min_time) + "ms";
if (statistic_name == "quantiles")
{
std::string result = "\n";
for (double percent = 10; percent <= 90; percent += 10)
{
result += FOUR_SPACES + std::to_string((percent / 100));
result += ": " + std::to_string(sampler.quantileInterpolated(percent / 100.0));
result += "\n";
}
result += FOUR_SPACES + "0.95: " + std::to_string(sampler.quantileInterpolated(95 / 100.0)) + "\n";
result += FOUR_SPACES + "0.99: " + std::to_string(sampler.quantileInterpolated(99 / 100.0)) + "\n";
result += FOUR_SPACES + "0.999: " + std::to_string(sampler.quantileInterpolated(99.9 / 100.)) + "\n";
result += FOUR_SPACES + "0.9999: " + std::to_string(sampler.quantileInterpolated(99.99 / 100.));
return result;
}
if (statistic_name == "total_time")
return std::to_string(total_time) + "s";
if (statistic_name == "queries_per_second")
return std::to_string(queries / total_time);
if (statistic_name == "rows_per_second")
return std::to_string(total_rows_read / total_time);
if (statistic_name == "bytes_per_second")
return std::to_string(total_bytes_read / total_time);
if (statistic_name == "max_rows_per_second")
return std::to_string(max_rows_speed);
if (statistic_name == "max_bytes_per_second")
return std::to_string(max_bytes_speed);
if (statistic_name == "avg_rows_per_second")
return std::to_string(avg_rows_speed_value);
if (statistic_name == "avg_bytes_per_second")
return std::to_string(avg_bytes_speed_value);
return "";
}
void TestStats::update_min_time(UInt64 min_time_candidate)
{
if (min_time_candidate < min_time)
{
min_time = min_time_candidate;
min_time_watch.restart();
}
}
void TestStats::update_max_speed(
size_t max_speed_candidate,
Stopwatch & max_speed_watch,
UInt64 & max_speed)
{
if (max_speed_candidate > max_speed)
{
max_speed = max_speed_candidate;
max_speed_watch.restart();
}
}
void TestStats::update_average_speed(
double new_speed_info,
Stopwatch & avg_speed_watch,
size_t & number_of_info_batches,
double precision,
double & avg_speed_first,
double & avg_speed_value)
{
avg_speed_value = ((avg_speed_value * number_of_info_batches) + new_speed_info);
++number_of_info_batches;
avg_speed_value /= number_of_info_batches;
if (avg_speed_first == 0)
{
avg_speed_first = avg_speed_value;
}
if (std::abs(avg_speed_value - avg_speed_first) >= precision)
{
avg_speed_first = avg_speed_value;
avg_speed_watch.restart();
}
}
void TestStats::add(size_t rows_read_inc, size_t bytes_read_inc)
{
total_rows_read += rows_read_inc;
total_bytes_read += bytes_read_inc;
last_query_rows_read += rows_read_inc;
last_query_bytes_read += bytes_read_inc;
double new_rows_speed = last_query_rows_read / watch_per_query.elapsedSeconds();
double new_bytes_speed = last_query_bytes_read / watch_per_query.elapsedSeconds();
/// Update rows speed
update_max_speed(new_rows_speed, max_rows_speed_watch, max_rows_speed);
update_average_speed(new_rows_speed,
avg_rows_speed_watch,
number_of_rows_speed_info_batches,
avg_rows_speed_precision,
avg_rows_speed_first,
avg_rows_speed_value);
/// Update bytes speed
update_max_speed(new_bytes_speed, max_bytes_speed_watch, max_bytes_speed);
update_average_speed(new_bytes_speed,
avg_bytes_speed_watch,
number_of_bytes_speed_info_batches,
avg_bytes_speed_precision,
avg_bytes_speed_first,
avg_bytes_speed_value);
}
void TestStats::updateQueryInfo()
{
++queries;
sampler.insert(watch_per_query.elapsedSeconds());
update_min_time(watch_per_query.elapsed() / (1000 * 1000)); /// ns to ms
}
TestStats::TestStats()
{
watch.reset();
watch_per_query.reset();
min_time_watch.reset();
max_rows_speed_watch.reset();
max_bytes_speed_watch.reset();
avg_rows_speed_watch.reset();
avg_bytes_speed_watch.reset();
}
void TestStats::startWatches()
{
watch.start();
watch_per_query.start();
min_time_watch.start();
max_rows_speed_watch.start();
max_bytes_speed_watch.start();
avg_rows_speed_watch.start();
avg_bytes_speed_watch.start();
}
}

View File

@ -0,0 +1,87 @@
#pragma once
#include <Core/Types.h>
#include <limits>
#include <Common/Stopwatch.h>
#include <AggregateFunctions/ReservoirSampler.h>
namespace DB
{
struct TestStats
{
TestStats();
Stopwatch watch;
Stopwatch watch_per_query;
Stopwatch min_time_watch;
Stopwatch max_rows_speed_watch;
Stopwatch max_bytes_speed_watch;
Stopwatch avg_rows_speed_watch;
Stopwatch avg_bytes_speed_watch;
bool last_query_was_cancelled = false;
size_t queries = 0;
size_t total_rows_read = 0;
size_t total_bytes_read = 0;
size_t last_query_rows_read = 0;
size_t last_query_bytes_read = 0;
using Sampler = ReservoirSampler<double>;
Sampler sampler{1 << 16};
/// min_time in ms
UInt64 min_time = std::numeric_limits<UInt64>::max();
double total_time = 0;
UInt64 max_rows_speed = 0;
UInt64 max_bytes_speed = 0;
double avg_rows_speed_value = 0;
double avg_rows_speed_first = 0;
static inline double avg_rows_speed_precision = 0.001;
double avg_bytes_speed_value = 0;
double avg_bytes_speed_first = 0;
static inline double avg_bytes_speed_precision = 0.001;
size_t number_of_rows_speed_info_batches = 0;
size_t number_of_bytes_speed_info_batches = 0;
bool ready = false; // check if a query wasn't interrupted by SIGINT
std::string exception;
/// Hack, actually this field doesn't required for statistics
bool got_SIGINT = false;
std::string getStatisticByName(const std::string & statistic_name);
void update_min_time(UInt64 min_time_candidate);
void update_average_speed(
double new_speed_info,
Stopwatch & avg_speed_watch,
size_t & number_of_info_batches,
double precision,
double & avg_speed_first,
double & avg_speed_value);
void update_max_speed(
size_t max_speed_candidate,
Stopwatch & max_speed_watch,
UInt64 & max_speed);
void add(size_t rows_read_inc, size_t bytes_read_inc);
void updateQueryInfo();
void setTotalTime()
{
total_time = watch.elapsedSeconds();
}
void startWatches();
};
}

View File

@ -0,0 +1,38 @@
#include "TestStopConditions.h"
namespace DB
{
void TestStopConditions::loadFromConfig(ConfigurationPtr & stop_conditions_config)
{
if (stop_conditions_config->has("all_of"))
{
ConfigurationPtr config_all_of(stop_conditions_config->createView("all_of"));
conditions_all_of.loadFromConfig(config_all_of);
}
if (stop_conditions_config->has("any_of"))
{
ConfigurationPtr config_any_of(stop_conditions_config->createView("any_of"));
conditions_any_of.loadFromConfig(config_any_of);
}
}
bool TestStopConditions::areFulfilled() const
{
return (conditions_all_of.initialized_count && conditions_all_of.fulfilled_count >= conditions_all_of.initialized_count)
|| (conditions_any_of.initialized_count && conditions_any_of.fulfilled_count);
}
UInt64 TestStopConditions::getMaxExecTime() const
{
UInt64 all_of_time = conditions_all_of.total_time_ms.value;
if (all_of_time == 0 && conditions_all_of.initialized_count != 0) /// max time is not set in all conditions
return 0;
else if(all_of_time != 0 && conditions_all_of.initialized_count > 1) /// max time is set, but we have other conditions
return 0;
UInt64 any_of_time = conditions_any_of.total_time_ms.value;
return std::max(all_of_time, any_of_time);
}
}

View File

@ -0,0 +1,57 @@
#pragma once
#include "StopConditionsSet.h"
#include <Poco/Util/XMLConfiguration.h>
namespace DB
{
/// Stop conditions for a test run. The running test will be terminated in either of two conditions:
/// 1. All conditions marked 'all_of' are fulfilled
/// or
/// 2. Any condition marked 'any_of' is fulfilled
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
class TestStopConditions
{
public:
void loadFromConfig(ConfigurationPtr & stop_conditions_config);
inline bool empty() const
{
return !conditions_all_of.initialized_count && !conditions_any_of.initialized_count;
}
#define DEFINE_REPORT_FUNC(FUNC_NAME, CONDITION) \
void FUNC_NAME(UInt64 value) \
{ \
conditions_all_of.report(value, conditions_all_of.CONDITION); \
conditions_any_of.report(value, conditions_any_of.CONDITION); \
}
DEFINE_REPORT_FUNC(reportTotalTime, total_time_ms)
DEFINE_REPORT_FUNC(reportRowsRead, rows_read)
DEFINE_REPORT_FUNC(reportBytesReadUncompressed, bytes_read_uncompressed)
DEFINE_REPORT_FUNC(reportIterations, iterations)
DEFINE_REPORT_FUNC(reportMinTimeNotChangingFor, min_time_not_changing_for_ms)
DEFINE_REPORT_FUNC(reportMaxSpeedNotChangingFor, max_speed_not_changing_for_ms)
DEFINE_REPORT_FUNC(reportAverageSpeedNotChangingFor, average_speed_not_changing_for_ms)
#undef REPORT
bool areFulfilled() const;
void reset()
{
conditions_all_of.reset();
conditions_any_of.reset();
}
/// Return max exec time for these conditions
/// Return zero if max time cannot be determined
UInt64 getMaxExecTime() const;
private:
StopConditionsSet conditions_all_of;
StopConditionsSet conditions_any_of;
};
}

View File

@ -0,0 +1,82 @@
#include "applySubstitutions.h"
#include <algorithm>
#include <vector>
namespace DB
{
void constructSubstitutions(ConfigurationPtr & substitutions_view, StringToVector & out_substitutions)
{
Strings xml_substitutions;
substitutions_view->keys(xml_substitutions);
for (size_t i = 0; i != xml_substitutions.size(); ++i)
{
const ConfigurationPtr xml_substitution(substitutions_view->createView("substitution[" + std::to_string(i) + "]"));
/// Property values for substitution will be stored in a vector
/// accessible by property name
Strings xml_values;
xml_substitution->keys("values", xml_values);
std::string name = xml_substitution->getString("name");
for (size_t j = 0; j != xml_values.size(); ++j)
{
out_substitutions[name].push_back(xml_substitution->getString("values.value[" + std::to_string(j) + "]"));
}
}
}
/// Recursive method which goes through all substitution blocks in xml
/// and replaces property {names} by their values
void runThroughAllOptionsAndPush(StringToVector::iterator substitutions_left,
StringToVector::iterator substitutions_right,
const std::string & template_query,
Strings & out_queries)
{
if (substitutions_left == substitutions_right)
{
out_queries.push_back(template_query); /// completely substituted query
return;
}
std::string substitution_mask = "{" + substitutions_left->first + "}";
if (template_query.find(substitution_mask) == std::string::npos) /// nothing to substitute here
{
runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, template_query, out_queries);
return;
}
for (const std::string & value : substitutions_left->second)
{
/// Copy query string for each unique permutation
std::string query = template_query;
size_t substr_pos = 0;
while (substr_pos != std::string::npos)
{
substr_pos = query.find(substitution_mask);
if (substr_pos != std::string::npos)
query.replace(substr_pos, substitution_mask.length(), value);
}
runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, query, out_queries);
}
}
Strings formatQueries(const std::string & query, StringToVector substitutions_to_generate)
{
Strings queries_res;
runThroughAllOptionsAndPush(
substitutions_to_generate.begin(),
substitutions_to_generate.end(),
query,
queries_res);
return queries_res;
}
}

View File

@ -0,0 +1,19 @@
#pragma once
#include <Poco/Util/XMLConfiguration.h>
#include <Core/Types.h>
#include <vector>
#include <string>
#include <map>
namespace DB
{
using StringToVector = std::map<std::string, Strings>;
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
void constructSubstitutions(ConfigurationPtr & substitutions_view, StringToVector & out_substitutions);
Strings formatQueries(const std::string & query, StringToVector substitutions_to_generate);
}

View File

@ -0,0 +1,73 @@
#include "executeQuery.h"
#include <IO/Progress.h>
#include <DataStreams/RemoteBlockInputStream.h>
#include <Core/Block.h>
namespace DB
{
namespace
{
void checkFulfilledConditionsAndUpdate(
const Progress & progress, RemoteBlockInputStream & stream,
TestStats & statistics, TestStopConditions & stop_conditions,
InterruptListener & interrupt_listener)
{
statistics.add(progress.rows, progress.bytes);
stop_conditions.reportRowsRead(statistics.total_rows_read);
stop_conditions.reportBytesReadUncompressed(statistics.total_bytes_read);
stop_conditions.reportTotalTime(statistics.watch.elapsed() / (1000 * 1000));
stop_conditions.reportMinTimeNotChangingFor(statistics.min_time_watch.elapsed() / (1000 * 1000));
stop_conditions.reportMaxSpeedNotChangingFor(statistics.max_rows_speed_watch.elapsed() / (1000 * 1000));
stop_conditions.reportAverageSpeedNotChangingFor(statistics.avg_rows_speed_watch.elapsed() / (1000 * 1000));
if (stop_conditions.areFulfilled())
{
statistics.last_query_was_cancelled = true;
stream.cancel(false);
}
if (interrupt_listener.check())
{
statistics.got_SIGINT = true;
statistics.last_query_was_cancelled = true;
stream.cancel(false);
}
}
}
void executeQuery(
Connection & connection,
const std::string & query,
TestStats & statistics,
TestStopConditions & stop_conditions,
InterruptListener & interrupt_listener,
Context & context)
{
statistics.watch_per_query.restart();
statistics.last_query_was_cancelled = false;
statistics.last_query_rows_read = 0;
statistics.last_query_bytes_read = 0;
Settings settings;
RemoteBlockInputStream stream(connection, query, {}, context, &settings);
stream.setProgressCallback(
[&](const Progress & value)
{
checkFulfilledConditionsAndUpdate(
value, stream, statistics,
stop_conditions, interrupt_listener);
});
stream.readPrefix();
while (Block block = stream.read());
stream.readSuffix();
if (!statistics.last_query_was_cancelled)
statistics.updateQueryInfo();
statistics.setTotalTime();
}
}

View File

@ -0,0 +1,18 @@
#pragma once
#include <string>
#include "TestStats.h"
#include "TestStopConditions.h"
#include <Common/InterruptListener.h>
#include <Interpreters/Context.h>
#include <Client/Connection.h>
namespace DB
{
void executeQuery(
Connection & connection,
const std::string & query,
TestStats & statistics,
TestStopConditions & stop_conditions,
InterruptListener & interrupt_listener,
Context & context);
}