Merge branch 'master' into split-odbc-bridge

This commit is contained in:
Alexey Milovidov 2019-01-30 17:01:06 +03:00
commit 73a6209eba
46 changed files with 2467 additions and 1489 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);
}

View File

@ -222,9 +222,7 @@ void ExternalLoader::reloadAndUpdate(bool throw_on_error)
}
else
{
tryLogCurrentException(log, "Cannot update " + object_name + " '" + name + "', leaving old version");
if (throw_on_error)
throw;
tryLogException(exception, log, "Cannot update " + object_name + " '" + name + "', leaving old version");
}
}
}

View File

@ -125,6 +125,7 @@ if [ -n "$*" ]; then
else
TEST_RUN=${TEST_RUN=1}
TEST_PERF=${TEST_PERF=1}
TEST_DICT=${TEST_DICT=1}
CLICKHOUSE_CLIENT_QUERY="${CLICKHOUSE_CLIENT} --config ${CLICKHOUSE_CONFIG_CLIENT} --port $CLICKHOUSE_PORT_TCP -m -n -q"
$CLICKHOUSE_CLIENT_QUERY 'SELECT * from system.build_options; SELECT * FROM system.clusters;'
CLICKHOUSE_TEST="env PATH=$PATH:$BIN_DIR ${TEST_DIR}clickhouse-test --binary ${BIN_DIR}${CLICKHOUSE_BINARY} --configclient $CLICKHOUSE_CONFIG_CLIENT --configserver $CLICKHOUSE_CONFIG --tmp $DATA_DIR/tmp --queries $QUERIES_DIR $TEST_OPT0 $TEST_OPT"
@ -139,6 +140,7 @@ else
fi
( [ "$TEST_RUN" ] && $CLICKHOUSE_TEST ) || ${TEST_TRUE:=false}
( [ "$TEST_PERF" ] && $CLICKHOUSE_PERFORMANCE_TEST $* ) || true
( [ "$TEST_DICT" ] && mkdir -p $DATA_DIR/etc/dictionaries/ && cd $CUR_DIR/external_dictionaries && python generate_and_test.py --port=$CLICKHOUSE_PORT_TCP --client=$CLICKHOUSE_CLIENT --source=$CUR_DIR/external_dictionaries/source.tsv --reference=$CUR_DIR/external_dictionaries/reference --generated=$DATA_DIR/etc/dictionaries/ --no_mysql --no_mongo ) || true
$CLICKHOUSE_CLIENT_QUERY "SELECT event, value FROM system.events; SELECT metric, value FROM system.metrics; SELECT metric, value FROM system.asynchronous_metrics;"
$CLICKHOUSE_CLIENT_QUERY "SELECT 'Still alive'"
fi

View File

@ -394,8 +394,8 @@ def generate_dictionaries(args):
</source>
<lifetime>
<min>0</min>
<max>0</max>
<min>5</min>
<max>15</max>
</lifetime>
<layout>

View File

@ -22,7 +22,7 @@ ORDER BY (dt, id) SETTINGS index_granularity = 8192;
insert into test.table1 (dt, id, arr) values ('2019-01-14', 1, ['aaa']);
insert into test.table2 (dt, id, arr) values ('2019-01-14', 1, ['aaa','bbb','ccc']);
select dt, id, groupArrayArray(arr)
select dt, id, arraySort(groupArrayArray(arr))
from (
select dt, id, arr from test.table1
where dt = '2019-01-14' and id = 1

View File

@ -110,7 +110,7 @@
<table>query_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</query_log>
<dictionaries_config>*_dictionary.xml</dictionaries_config>
<dictionaries_config>dictionaries/dictionary_*.xml</dictionaries_config>
<compression incl="clickhouse_compression">
</compression>
<distributed_ddl>

View File

@ -323,7 +323,7 @@ Outputs data as separate JSON objects for each row (newline delimited JSON).
Unlike the JSON format, there is no substitution of invalid UTF-8 sequences. Any set of bytes can be output in the rows. This is necessary so that data can be formatted without losing any information. Values are escaped in the same way as for JSON.
For parsing, any order is supported for the values of different columns. It is acceptable for some values to be omitted they are treated as equal to their default values. In this case, zeros and blank rows are used as default values. Complex values that could be specified in the table are not supported as defaults. Whitespace between elements is ignored. If a comma is placed after the objects, it is ignored. Objects don't necessarily have to be separated by new lines.
For parsing, any order is supported for the values of different columns. It is acceptable for some values to be omitted they are treated as equal to their default values. In this case, zeros and blank rows are used as default values. Complex values that could be specified in the table are not supported as defaults, but it can be turned on by option `insert_sample_with_metadata=1`. Whitespace between elements is ignored. If a comma is placed after the objects, it is ignored. Objects don't necessarily have to be separated by new lines.
## Native {#native}

View File

@ -81,6 +81,9 @@ If an error occurred while reading rows but the error counter is still less than
If `input_format_allow_errors_ratio` is exceeded, ClickHouse throws an exception.
## insert_sample_with_metadata
For INSERT queries, specifies that the server need to send metadata about column defaults to the client. This will be used to calculate default expressions. Disabled by default.
## join_default_strictness

View File

@ -469,4 +469,64 @@ If you want to get a list of unique items in an array, you can use arrayReduce('
A special function. See the section ["ArrayJoin function"](array_join.md#functions_arrayjoin).
## arrayDifference(arr)
Takes an array, returns an array with the difference between all pairs of neighboring elements. For example:
```sql
SELECT arrayDifference([1, 2, 3, 4])
```
```
┌─arrayDifference([1, 2, 3, 4])─┐
│ [0,1,1,1] │
└───────────────────────────────┘
```
## arrayDistinct(arr)
Takes an array, returns an array containing the different elements in all the arrays. For example:
```sql
SELECT arrayDifference([1, 2, 3, 4])
```
```
┌─arrayDifference([1, 2, 3, 4])─┐
│ [0,1,1,1] │
└───────────────────────────────┘
```
## arrayEnumerateDense(arr)
Returns an array of the same size as the source array, indicating where each element first appears in the source array. For example: arrayEnumerateDense([10,20,10,30]) = [1,2,1,4].
## arrayIntersect(arr)
Takes an array, returns the intersection of all array elements. For example:
```sql
SELECT
arrayIntersect([1, 2], [1, 3], [2, 3]) AS no_intersect,
arrayIntersect([1, 2], [1, 3], [1, 4]) AS intersect
```
```
┌─no_intersect─┬─intersect─┐
│ [] │ [1] │
└──────────────┴───────────┘
```
## arrayReduce(agg_func, arr1, ...)
Applies an aggregate function to array and returns its result.If aggregate function has multiple arguments, then this function can be applied to multiple arrays of the same size.
arrayReduce('agg_func', arr1, ...) - apply the aggregate function `agg_func` to arrays `arr1...`. If multiple arrays passed, then elements on corresponding positions are passed as multiple arguments to the aggregate function. For example: SELECT arrayReduce('max', [1,2,3]) = 3
## arrayReverse(arr)
Returns an array of the same size as the source array, containing the result of inverting all elements of the source array.
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/array_functions/) <!--hide-->

View File

@ -16,5 +16,16 @@ The result type is an integer with bits equal to the maximum bits of its argumen
## bitShiftRight(a, b)
## bitRotateLeft(a, b)
## bitRotateRight(a, b)
## bitTest(a, b)
## bitTestAll(a, b)
## bitTestAny(a, b)
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/bit_functions/) <!--hide-->

View File

@ -20,17 +20,29 @@ SELECT
Only time zones that differ from UTC by a whole number of hours are supported.
## toTimeZone
Convert time or date and time to the specified time zone.
## toYear
Converts a date or date with time to a UInt16 number containing the year number (AD).
## toQuarter
Converts a date or date with time to a UInt8 number containing the quarter number.
## toMonth
Converts a date or date with time to a UInt8 number containing the month number (1-12).
## toDayOfYear
Converts a date or date with time to a UInt8 number containing the number of the day of the year (1-366).
## toDayOfMonth
-Converts a date or date with time to a UInt8 number containing the number of the day of the month (1-31).
Converts a date or date with time to a UInt8 number containing the number of the day of the month (1-31).
## toDayOfWeek
@ -50,11 +62,20 @@ Converts a date with time to a UInt8 number containing the number of the minute
Converts a date with time to a UInt8 number containing the number of the second in the minute (0-59).
Leap seconds are not accounted for.
## toUnixTimestamp
Converts a date with time to a unix timestamp.
## toMonday
Rounds down a date or date with time to the nearest Monday.
Returns the date.
## toStartOfISOYear
Rounds down a date or date with time to the first day of ISO year.
Returns the date.
## toStartOfMonth
Rounds down a date or date with time to the first day of the month.
@ -104,6 +125,10 @@ Converts a date with time to a certain fixed date, while preserving the time.
Converts a date with time or date to the number of the year, starting from a certain fixed point in the past.
## toRelativeQuarterNum
Converts a date with time or date to the number of the quarter, starting from a certain fixed point in the past.
## toRelativeMonthNum
Converts a date with time or date to the number of the month, starting from a certain fixed point in the past.
@ -128,6 +153,14 @@ Converts a date with time or date to the number of the minute, starting from a c
Converts a date with time or date to the number of the second, starting from a certain fixed point in the past.
## toISOYear
Converts a date or date with time to a UInt16 number containing the ISO Year number.
## toISOWeek
Converts a date or date with time to a UInt8 number containing the ISO Week number.
## now
Accepts zero arguments and returns the current time at one of the moments of request execution.
@ -148,6 +181,60 @@ The same as 'today() - 1'.
Rounds the time to the half hour.
This function is specific to Yandex.Metrica, since half an hour is the minimum amount of time for breaking a session into two sessions if a tracking tag shows a single user's consecutive pageviews that differ in time by strictly more than this amount. This means that tuples (the tag ID, user ID, and time slot) can be used to search for pageviews that are included in the corresponding session.
## toYYYYMM
Converts a date or date with time to a UInt32 number containing the year and month number (YYYY * 100 + MM).
## toYYYYMMDD
Converts a date or date with time to a UInt32 number containing the year and month number (YYYY * 10000 + MM * 100 + DD).
## toYYYYMMDDhhmmss
Converts a date or date with time to a UInt64 number containing the year and month number (YYYY * 10000000000 + MM * 100000000 + DD * 1000000 + hh * 10000 + mm * 100 + ss).
## addYears, addMonths, addWeeks, addDays, addHours, addMinutes, addSeconds, addQuarters
Function adds a Date/DateTime interval to a Date/DateTime and then return the Date/DateTime. For example:
```sql
WITH
toDate('2018-01-01') AS date,
toDateTime('2018-01-01 00:00:00') AS date_time
SELECT
addYears(date, 1) AS add_years_with_date,
addYears(date_time, 1) AS add_years_with_date_time
```
```
┌─add_years_with_date─┬─add_years_with_date_time─┐
│ 2019-01-01 │ 2019-01-01 00:00:00 │
└─────────────────────┴──────────────────────────┘
```
## subtractYears, subtractMonths, subtractWeeks, subtractDays, subtractHours, subtractMinutes, subtractSeconds, subtractQuarters
Function subtract a Date/DateTime interval to a Date/DateTime and then return the Date/DateTime. For example:
```sql
WITH
toDate('2019-01-01') AS date,
toDateTime('2019-01-01 00:00:00') AS date_time
SELECT
subtractYears(date, 1) AS subtract_years_with_date,
subtractYears(date_time, 1) AS subtract_years_with_date_time
```
```
┌─subtract_years_with_date─┬─subtract_years_with_date_time─┐
│ 2018-01-01 │ 2018-01-01 00:00:00 │
└──────────────────────────┴───────────────────────────────┘
```
## dateDiff('unit', t1, t2, \[timezone\])
Return the difference between two times, t1 and t2 can be Date or DateTime, If timezone is specified, it applied to both arguments. If not, timezones from datatypes t1 and t2 are used. If that timezones are not the same, the result is unspecified.
## timeSlots(StartTime, Duration,\[, Size\])
For a time interval starting at 'StartTime' and continuing for 'Duration' seconds, it returns an array of moments in time, consisting of points from this interval rounded down to the 'Size' in seconds. 'Size' is an optional parameter: a constant UInt32, set to 1800 by default.

View File

@ -21,7 +21,7 @@ If there is no `id` key in the dictionary, it returns the default value specifie
## dictGetTOrDefault {#ext_dict_functions_dictGetTOrDefault}
`dictGetT('dict_name', 'attr_name', id, default)`
`dictGetTOrDefault('dict_name', 'attr_name', id, default)`
The same as the `dictGetT` functions, but the default value is taken from the function's last argument.

View File

@ -64,5 +64,52 @@ A fast, decent-quality non-cryptographic hash function for a string obtained fro
`URLHash(s, N)` Calculates a hash from a string up to the N level in the URL hierarchy, without one of the trailing symbols `/`,`?` or `#` at the end, if present.
Levels are the same as in URLHierarchy. This function is specific to Yandex.Metrica.
## farmHash64
Calculates FarmHash64 from a string.
Accepts a String-type argument. Returns UInt64.
For more information, see the link: [FarmHash64](https://github.com/google/farmhash)
## javaHash
Calculates JavaHash from a string.
Accepts a String-type argument. Returns Int32.
For more information, see the link: [JavaHash](http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452)
## hiveHash
Calculates HiveHash from a string.
Accepts a String-type argument. Returns Int32.
Same as for [JavaHash](./hash_functions.md#javaHash), except that the return value never has a negative number.
## metroHash64
Calculates MetroHash from a string.
Accepts a String-type argument. Returns UInt64.
For more information, see the link: [MetroHash64](http://www.jandrewrogers.com/2015/05/27/metrohash/)
## jumpConsistentHash
Calculates JumpConsistentHash form a UInt64.
Accepts a UInt64-type argument. Returns Int32.
For more information, see the link: [JumpConsistentHash](https://arxiv.org/pdf/1406.2294.pdf)
## murmurHash2_32, murmurHash2_64
Calculates MurmurHash2 from a string.
Accepts a String-type argument. Returns UInt64 Or UInt32.
For more information, see the link: [MurmurHash2](https://github.com/aappleby/smhasher)
## murmurHash3_32, murmurHash3_64, murmurHash3_128
Calculates MurmurHash3 from a string.
Accepts a String-type argument. Returns UInt64 Or UInt32 Or FixedString(16).
For more information, see the link: [MurmurHash3](https://github.com/aappleby/smhasher)
## xxHash32, xxHash64
Calculates xxHash from a string.
ccepts a String-type argument. Returns UInt64 Or UInt32.
For more information, see the link: [xxHash](http://cyan4973.github.io/xxHash/)
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/hash_functions/) <!--hide-->

View File

@ -87,6 +87,20 @@ SELECT arrayCumSum([1, 1, 1, 1]) AS res
└──────────────┘
```
### arrayCumSumNonNegative(arr)
Same as arrayCumSum, returns an array of partial sums of elements in the source array (a running sum). Different arrayCumSum, when then returned value contains a value less than zero, the value is replace with zero and the subsequent calculation is performed with zero parameters. For example:
``` sql
SELECT arrayCumSumNonNegative([1, 1, -4, 1]) AS res
```
```
┌─res───────┐
│ [1,2,0,1] │
└───────────┘
```
### arraySort(\[func,\] arr1, ...)
Returns an array as result of sorting the elements of `arr1` in ascending order. If the `func` function is specified, sorting order is determined by the result of the function `func` applied to the elements of array (arrays)
@ -112,6 +126,6 @@ Returns an array as result of sorting the elements of `arr1` in descending order
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/higher_order_functions/) <!--hide-->

View File

@ -113,5 +113,38 @@ LIMIT 10
The reverse function of IPv6NumToString. If the IPv6 address has an invalid format, it returns a string of null bytes.
HEX can be uppercase or lowercase.
## IPv4ToIPv6(x)
Takes a UInt32 number. Interprets it as an IPv4 address in big endian. Returns a FixedString(16) value containing the IPv6 address in binary format. Examples:
``` sql
SELECT IPv6NumToString(IPv4ToIPv6(IPv4StringToNum('192.168.0.1'))) AS addr
```
```
┌─addr───────────────┐
│ ::ffff:192.168.0.1 │
└────────────────────┘
```
## cutIPv6(x, bitsToCutForIPv6, bitsToCutForIPv4)
Accepts a FixedString(16) value containing the IPv6 address in binary format. Returns a string containing the address of the specified number of bits removed in text format. For example:
```sql
WITH
IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D') AS ipv6,
IPv4ToIPv6(IPv4StringToNum('192.168.0.1')) AS ipv4
SELECT
cutIPv6(ipv6, 2, 0),
cutIPv6(ipv4, 0, 2)
```
```
┌─cutIPv6(ipv6, 2, 0)─────────────────┬─cutIPv6(ipv4, 0, 2)─┐
│ 2001:db8:ac10:fe01:feed:babe:cafe:0 │ ::ffff:192.168.0.0 │
└─────────────────────────────────────┴─────────────────────┘
```
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/ip_address_functions/) <!--hide-->

View File

@ -14,7 +14,7 @@ Returns a Float64 number that is close to the number π.
Accepts a numeric argument and returns a Float64 number close to the exponent of the argument.
## log(x)
## log(x), ln(x)
Accepts a numeric argument and returns a Float64 number close to the natural logarithm of the argument.
@ -94,8 +94,16 @@ The arc cosine.
The arc tangent.
## pow(x, y)
## pow(x, y), power(x, y)
Takes two numeric arguments x and y. Returns a Float64 number close to x to the power of y.
## intExp2
Accepts a numeric argument and returns a UInt64 number close to 2 to the power of x.
## intExp10
Accepts a numeric argument and returns a UInt64 number close to 10 to the power of x.
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/math_functions/) <!--hide-->

View File

@ -44,6 +44,10 @@ However, the argument is still evaluated. This can be used for benchmarks.
Sleeps 'seconds' seconds on each data block. You can specify an integer or a floating-point number.
## sleepEachRow(seconds)
Sleeps 'seconds' seconds on each row. You can specify an integer or a floating-point number.
## currentDatabase()
Returns the name of the current database.
@ -242,6 +246,18 @@ Returns the server's uptime in seconds.
Returns the version of the server as a string.
## timezone()
Returns the timezone of the server.
## blockNumber
Returns the sequence number of the data block where the row is located.
## rowNumberInBlock
Returns the ordinal number of the row in the data block. Different data blocks are always recalculated.
## rowNumberInAllBlocks()
Returns the ordinal number of the row in the data block. This function only considers the affected data blocks.
@ -283,6 +299,10 @@ FROM
└─────────┴─────────────────────┴───────┘
```
## runningDifferenceStartingWithFirstValue
Same as for [runningDifference](./other_functions.md#runningDifference), the difference is the value of the first row, returned the value of the first row, and each subsequent row returns the difference from the previous row.
## MACNumToString(num)
Accepts a UInt64 number. Interprets it as a MAC address in big endian. Returns a string containing the corresponding MAC address in the format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form).
@ -440,7 +460,7 @@ The expression passed to the function is not calculated, but ClickHouse applies
**Returned value**
- 1.
- 1.
**Example**
@ -558,5 +578,34 @@ SELECT replicate(1, ['a', 'b', 'c'])
└───────────────────────────────┘
```
## filesystemAvailable
Returns the remaining space information of the disk, in bytes. This information is evaluated using the configured by path.
## filesystemCapacity
Returns the capacity information of the disk, in bytes. This information is evaluated using the configured by path.
## finalizeAggregation
Takes state of aggregate function. Returns result of aggregation (finalized state).
## runningAccumulate
Takes the states of the aggregate function and returns a column with values, are the result of the accumulation of these states for a set of block lines, from the first to the current line.
For example, takes state of aggregate function (example runningAccumulate(uniqState(UserID))), and for each row of block, return result of aggregate function on merge of states of all previous rows and current row.
So, result of function depends on partition of data to blocks and on order of data in block.
## joinGet('join_storage_table_name', 'get_column', join_key)
Get data from a table of type Join using the specified join key.
## modelEvaluate(model_name, ...)
Evaluate external model.
Accepts a model name and model arguments. Returns Float64.
## throwIf(x)
Throw an exception if the argument is non zero.
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/other_functions/) <!--hide-->

View File

@ -16,5 +16,8 @@ Uses a linear congruential generator.
Returns a pseudo-random UInt64 number, evenly distributed among all UInt64-type numbers.
Uses a linear congruential generator.
## randConstant
Returns a pseudo-random UInt32 number, The value is one for different blocks.
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/random_functions/) <!--hide-->

View File

@ -12,7 +12,7 @@ Examples: `floor(123.45, 1) = 123.4, floor(123.45, -1) = 120.`
For integer arguments, it makes sense to round with a negative 'N' value (for non-negative 'N', the function doesn't do anything).
If rounding causes overflow (for example, floor(-128, -1)), an implementation-specific result is returned.
## ceil(x\[, N\])
## ceil(x\[, N\]), ceiling(x\[, N\])
Returns the smallest round number that is greater than or equal to 'x'. In every other way, it is the same as the 'floor' function (see above).
@ -66,5 +66,8 @@ Accepts a number. If the number is less than one, it returns 0. Otherwise, it ro
Accepts a number. If the number is less than 18, it returns 0. Otherwise, it rounds the number down to a number from the set: 18, 25, 35, 45, 55. This function is specific to Yandex.Metrica and used for implementing the report on user age.
## roundDown(num, arr)
Accept a number, round it down to an element in the specified array. If the value is less than the lowest bound, the lowest bound is returned.
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/rounding_functions/) <!--hide-->

View File

@ -24,11 +24,21 @@ The function also works for arrays.
Returns the length of a string in Unicode code points (not in characters), assuming that the string contains a set of bytes that make up UTF-8 encoded text. If this assumption is not met, it returns some result (it doesn't throw an exception).
The result type is UInt64.
## lower
## char_length, CHAR_LENGTH
Returns the length of a string in Unicode code points (not in characters), assuming that the string contains a set of bytes that make up UTF-8 encoded text. If this assumption is not met, it returns some result (it doesn't throw an exception).
The result type is UInt64.
## character_length, CHARACTER_LENGTH
Returns the length of a string in Unicode code points (not in characters), assuming that the string contains a set of bytes that make up UTF-8 encoded text. If this assumption is not met, it returns some result (it doesn't throw an exception).
The result type is UInt64.
## lower, lcase
Converts ASCII Latin symbols in a string to lowercase.
## upper
## upper, ucase
Converts ASCII Latin symbols in a string to uppercase.
@ -58,7 +68,11 @@ Reverses a sequence of Unicode code points, assuming that the string contains a
Concatenates the strings listed in the arguments, without a separator.
## substring(s, offset, length)
## concatAssumeInjective(s1, s2, ...)
Same as [concat](./string_functions.md#concat-s1-s2), the difference is that you need to ensure that concat(s1, s2, s3) -> s4 is injective, it will be used for optimization of GROUP BY
## substring(s, offset, length), mid(s, offset, length), substr(s, offset, length)
Returns a substring starting with the byte from the 'offset' index that is 'length' bytes long. Character indexing starts from one (as in standard SQL). The 'offset' and 'length' arguments must be constants.
@ -83,4 +97,24 @@ Decode base64-encoded string 's' into original string. In case of failure raises
## tryBase64Decode(s)
Similar to base64Decode, but in case of error an empty string would be returned.
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/string_functions/) <!--hide-->
## endsWith(s, suffix)
Returns whether to end with the specified suffix. Returns 1 if the string ends with the specified suffix, otherwise it returns 0.
## startsWith(s, prefix)
Returns whether to end with the specified prefix. Returns 1 if the string ends with the specified prefix, otherwise it returns 0.
## trimLeft(s)
Returns a string that removes the whitespace characters on left side.
## trimRight(s)
Returns a string that removes the whitespace characters on right side.
## trimBoth(s)
Returns a string that removes the whitespace characters on either side.
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/string_functions/) <!--hide-->

View File

@ -5,7 +5,7 @@
Replaces the first occurrence, if it exists, of the 'pattern' substring in 'haystack' with the 'replacement' substring.
Hereafter, 'pattern' and 'replacement' must be constants.
## replaceAll(haystack, pattern, replacement)
## replaceAll(haystack, pattern, replacement), replace(haystack, pattern, replacement)
Replaces all occurrences of the 'pattern' substring in 'haystack' with the 'replacement' substring.
@ -78,4 +78,12 @@ SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res
```
## regexpQuoteMeta(s)
The function adds a backslash before some predefined characters in the string.
Predefined characters: '0', '\\', '|', '(', ')', '^', '$', '.', '[', ']', '?', '*', '+', '{', ':', '-'.
This implementation slightly differs from re2::RE2::QuoteMeta. It escapes zero byte as \0 instead of \x00 and it escapes only required characters.
For more information, see the link: [RE2](https://github.com/google/re2/blob/master/re2/re2.cc#L473)
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/string_replace_functions/) <!--hide-->

View File

@ -3,7 +3,7 @@
The search is case-sensitive in all these functions.
The search substring or regular expression must be a constant in all these functions.
## position(haystack, needle)
## position(haystack, needle), locate(haystack, needle)
Search for the substring `needle` in the string `haystack`.
Returns the position (in bytes) of the found substring, starting from 1, or returns 0 if the substring was not found.

View File

@ -7,10 +7,12 @@
## toFloat32, toFloat64
## toUInt8OrZero, toUInt16OrZero, toUInt32OrZero, toUInt64OrZero, toInt8OrZero, toInt16OrZero, toInt32OrZero, toInt64OrZero, toFloat32OrZero, toFloat64OrZero
## toDate, toDateTime
## toUInt8OrZero, toUInt16OrZero, toUInt32OrZero, toUInt64OrZero, toInt8OrZero, toInt16OrZero, toInt32OrZero, toInt64OrZero, toFloat32OrZero, toFloat64OrZero, toDateOrZero, toDateTimeOrZero
## toUInt8OrNull, toUInt16OrNull, toUInt32OrNull, toUInt64OrNull, toInt8OrNull, toInt16OrNull, toInt32OrNull, toInt64OrNull, toFloat32OrNull, toFloat64OrNull, toDateOrNull, toDateTimeOrNull
## toDecimal32(value, S), toDecimal64(value, S), toDecimal128(value, S)
Converts `value` to [Decimal](../../data_types/decimal.md) of precision `S`. The `value` can be a number or a string. The `S` (scale) parameter specifies the number of decimal places.
@ -99,6 +101,9 @@ These functions accept a string and interpret the bytes placed at the beginning
This function accepts a number or date or date with time, and returns a string containing bytes representing the corresponding value in host order (little endian). Null bytes are dropped from the end. For example, a UInt32 type value of 255 is a string that is one byte long.
## reinterpretAsFixedString
This function accepts a number or date or date with time, and returns a FixedString containing bytes representing the corresponding value in host order (little endian). Null bytes are dropped from the end. For example, a UInt32 type value of 255 is a FixedString that is one byte long.
## CAST(x, t)
@ -141,5 +146,39 @@ SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null
└─────────────────────────────────────────┘
```
## toIntervalYear, toIntervalQuarter, toIntervalMonth, toIntervalWeek, toIntervalDay, toIntervalHour, toIntervalMinute, toIntervalSecond
Converts a Number type argument to a Interval type (duration).
The interval type is actually very useful, you can use this type of data to perform arithmetic operations directly with Date or DateTime. At the same time, ClickHouse provides a more convenient syntax for declaring Interval type data. For example:
```sql
WITH
toDate('2019-01-01') AS date,
INTERVAL 1 WEEK AS interval_week,
toIntervalWeek(1) AS interval_to_week
SELECT
date + interval_week,
date + interval_to_week
```
```
┌─plus(date, interval_week)─┬─plus(date, interval_to_week)─┐
│ 2019-01-08 │ 2019-01-08 │
└───────────────────────────┴──────────────────────────────┘
```
## parseDateTimeBestEffort
Parse a number type argument to a Date or DateTime type.
different from toDate and toDateTime, parseDateTimeBestEffort can progress more complex date format.
For more information, see the link: [Complex Date Format](https://xkcd.com/1179/)
## parseDateTimeBestEffortOrNull
Same as for [parseDateTimeBestEffort](./type_conversion_functions.md#parseDateTimeBestEffort) except that it returns null when it encounters a date format that cannot be processed.
## parseDateTimeBestEffortOrZero
Same as for [parseDateTimeBestEffort](./type_conversion_functions.md#parseDateTimeBestEffort) except that it returns zero date or zero date time when it encounters a date format that cannot be processed.
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/type_conversion_functions/) <!--hide-->

View File

@ -323,7 +323,7 @@ ClickHouse поддерживает [NULL](../query_language/syntax.md), кот
В отличие от формата JSON, нет замены невалидных UTF-8 последовательностей. В строках может выводиться произвольный набор байт. Это сделано для того, чтобы данные форматировались без потери информации. Экранирование значений осуществляется аналогично формату JSON.
При парсинге, поддерживается расположение значений разных столбцов в произвольном порядке. Допустимо отсутствие некоторых значений - тогда они воспринимаются как равные значениям по умолчанию. При этом, в качестве значений по умолчанию используются нули, пустые строки и не поддерживаются сложные значения по умолчанию, которые могут быть заданы в таблице. Пропускаются пробельные символы между элементами. После объектов может быть расположена запятая, которая игнорируется. Объекты не обязательно должны быть разделены переводами строк.
При парсинге, поддерживается расположение значений разных столбцов в произвольном порядке. Допустимо отсутствие некоторых значений - тогда они воспринимаются как равные значениям по умолчанию. При этом, в качестве значений по умолчанию используются нули, и пустые строки. Сложные значения которые могут быть заданы в таблице, не поддерживаются по умолчанию, но их можно включить с помощью опции `insert_sample_with_metadata = 1`. Пропускаются пробельные символы между элементами. После объектов может быть расположена запятая, которая игнорируется. Объекты не обязательно должны быть разделены переводами строк.
## Native {#native}

View File

@ -322,6 +322,10 @@ ClickHouse применяет настройку в тех случаях, ко
Если значение истинно, то при выполнении INSERT из входных данных пропускаются (не рассматриваются) колонки с неизвестными именами, иначе в данной ситуации будет сгенерировано исключение.
Работает для форматов JSONEachRow и TSKV.
## insert_sample_with_metadata
Для запросов INSERT. Указывает, что серверу необходимо отправлять клиенту метаданные о значениях столбцов по умолчанию, которые будут использоваться для вычисления выражений по умолчанию. По умолчанию отключено.
## output_format_json_quote_64bit_integers
Если значение истинно, то при использовании JSON\* форматов UInt64 и Int64 числа выводятся в кавычках (из соображений совместимости с большинством реализаций JavaScript), иначе - без кавычек.