ClickHouse/dbms/programs/benchmark/Benchmark.cpp

558 lines
19 KiB
C++
Raw Normal View History

#include <port/unistd.h>
#include <stdlib.h>
#include <fcntl.h>
#include <signal.h>
#include <time.h>
#include <iostream>
#include <fstream>
#include <iomanip>
#include <random>
#include <pcg_random.hpp>
#include <Poco/File.h>
#include <Poco/Util/Application.h>
#include <Common/Stopwatch.h>
#include <Common/ThreadPool.h>
#include <AggregateFunctions/ReservoirSampler.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <boost/program_options.hpp>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/Exception.h>
#include <Common/randomSeed.h>
#include <Core/Types.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/UseSSL.h>
#include <DataStreams/RemoteBlockInputStream.h>
#include <Interpreters/Context.h>
#include <Client/Connection.h>
2018-06-05 20:09:51 +00:00
#include <Common/InterruptListener.h>
#include <Common/Config/configReadClient.h>
/** A tool for evaluating ClickHouse performance.
* The tool emulates a case with fixed amount of simultaneously executing queries.
*/
namespace DB
{
namespace ErrorCodes
{
2017-04-08 01:32:05 +00:00
extern const int BAD_ARGUMENTS;
extern const int EMPTY_DATA_PASSED;
}
class Benchmark : public Poco::Util::Application
{
public:
Benchmark(unsigned concurrency_, double delay_,
2019-08-05 18:38:08 +00:00
const std::vector<std::string> & hosts_, const std::vector<UInt16> & ports_,
bool cumulative_, bool secure_, const String & default_database_,
const String & user_, const String & password_, const String & stage,
bool randomize_, size_t max_iterations_, double max_time_,
2019-03-02 21:40:40 +00:00
const String & json_path_, const Settings & settings_)
:
concurrency(concurrency_), delay(delay_), queue(concurrency), randomize(randomize_),
cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_),
json_path(json_path_), settings(settings_), global_context(Context::createGlobal()), pool(concurrency)
{
2019-08-05 13:00:27 +00:00
const auto secure = secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable;
2019-08-05 18:38:08 +00:00
size_t connections_cnt = std::max(ports_.size(), hosts_.size());
2019-08-05 13:00:27 +00:00
2019-08-05 18:38:08 +00:00
connections.reserve(connections_cnt);
comparison_info_total.reserve(connections_cnt);
comparison_info_per_interval.reserve(connections_cnt);
2019-08-05 13:00:27 +00:00
2019-08-05 18:38:08 +00:00
for (size_t i = 0; i < connections_cnt; ++i)
2019-08-05 13:00:27 +00:00
{
2019-08-05 18:38:08 +00:00
UInt16 cur_port = i >= ports_.size() ? 9000 : ports_[i];
std::string cur_host = i >= hosts_.size() ? "localhost" : hosts_[i];
connections.emplace_back(std::make_shared<ConnectionPool>(concurrency, cur_host, cur_port, default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure));
2019-08-05 13:00:27 +00:00
comparison_info_per_interval.emplace_back(std::make_shared<Stats>());
comparison_info_total.emplace_back(std::make_shared<Stats>());
}
2019-07-08 02:14:32 +00:00
global_context.makeGlobalContext();
std::cerr << std::fixed << std::setprecision(3);
/// This is needed to receive blocks with columns of AggregateFunction data type
/// (example: when using stage = 'with_mergeable_state')
registerAggregateFunctions();
if (stage == "complete")
query_processing_stage = QueryProcessingStage::Complete;
else if (stage == "fetch_columns")
query_processing_stage = QueryProcessingStage::FetchColumns;
else if (stage == "with_mergeable_state")
query_processing_stage = QueryProcessingStage::WithMergeableState;
else
throw Exception("Unknown query processing stage: " + stage, ErrorCodes::BAD_ARGUMENTS);
}
2018-11-29 07:58:59 +00:00
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(config(), home_path);
}
2018-11-28 14:50:02 +00:00
int main(const std::vector<std::string> &)
{
if (!json_path.empty() && Poco::File(json_path).exists()) /// Clear file with previous results
Poco::File(json_path).remove();
readQueries();
runBenchmark();
2018-11-28 14:50:02 +00:00
return 0;
}
private:
2019-08-05 13:00:27 +00:00
using Entry = ConnectionPool::Entry;
using EntryPtr = std::shared_ptr<Entry>;
using EntryPtrs = std::vector<EntryPtr>;
unsigned concurrency;
double delay;
2019-08-05 13:00:27 +00:00
using Query = std::string;
using Queries = std::vector<Query>;
Queries queries;
using Queue = ConcurrentBoundedQueue<Query>;
Queue queue;
2019-08-05 13:00:27 +00:00
ConnectionPoolPtrs connections;
bool randomize;
bool cumulative;
size_t max_iterations;
double max_time;
String json_path;
Settings settings;
Context global_context;
QueryProcessingStage::Enum query_processing_stage;
/// Don't execute new queries after timelimit or SIGINT or exception
std::atomic<bool> shutdown{false};
2018-10-11 18:12:28 +00:00
std::atomic<size_t> queries_executed{0};
struct Stats
{
std::atomic<size_t> queries{0};
size_t read_rows = 0;
size_t read_bytes = 0;
size_t result_rows = 0;
size_t result_bytes = 0;
double work_time = 0;
using Sampler = ReservoirSampler<double>;
Sampler sampler {1 << 16};
void add(double seconds, size_t read_rows_inc, size_t read_bytes_inc, size_t result_rows_inc, size_t result_bytes_inc)
{
++queries;
work_time += seconds;
read_rows += read_rows_inc;
read_bytes += read_bytes_inc;
result_rows += result_rows_inc;
result_bytes += result_bytes_inc;
sampler.insert(seconds);
}
void clear()
{
queries = 0;
work_time = 0;
read_rows = 0;
read_bytes = 0;
result_rows = 0;
result_bytes = 0;
sampler.clear();
}
};
2019-08-05 13:00:27 +00:00
using MultiStats = std::vector<std::shared_ptr<Stats>>;
MultiStats comparison_info_per_interval;
MultiStats comparison_info_total;
Stopwatch total_watch;
Stopwatch delay_watch;
std::mutex mutex;
ThreadPool pool;
void readQueries()
{
ReadBufferFromFileDescriptor in(STDIN_FILENO);
while (!in.eof())
{
std::string query;
readText(query, in);
assertChar('\n', in);
if (!query.empty())
queries.emplace_back(query);
}
if (queries.empty())
throw Exception("Empty list of queries.", ErrorCodes::EMPTY_DATA_PASSED);
std::cerr << "Loaded " << queries.size() << " queries.\n";
}
void printNumberOfQueriesExecuted(size_t num)
{
std::cerr << "\nQueries executed: " << num;
if (queries.size() > 1)
std::cerr << " (" << (num * 100.0 / queries.size()) << "%)";
std::cerr << ".\n";
}
/// Try push new query and check cancellation conditions
bool tryPushQueryInteractively(const String & query, InterruptListener & interrupt_listener)
{
bool inserted = false;
while (!inserted)
{
inserted = queue.tryPush(query, 100);
if (shutdown)
{
/// An exception occurred in a worker
return false;
}
2019-08-05 13:00:27 +00:00
if (max_time > 0 && total_watch.elapsedSeconds() >= max_time)
{
std::cout << "Stopping launch of queries. Requested time limit is exhausted.\n";
return false;
}
if (interrupt_listener.check())
{
std::cout << "Stopping launch of queries. SIGINT recieved.\n";
return false;
}
if (delay > 0 && delay_watch.elapsedSeconds() > delay)
{
2019-08-05 13:00:27 +00:00
printNumberOfQueriesExecuted(queries_executed);
cumulative ? report(comparison_info_total) : report(comparison_info_per_interval);
delay_watch.restart();
}
}
return true;
}
void runBenchmark()
{
pcg64 generator(randomSeed());
std::uniform_int_distribution<size_t> distribution(0, queries.size() - 1);
for (size_t i = 0; i < concurrency; ++i)
2019-08-05 13:00:27 +00:00
{
EntryPtrs connection_entries;
connection_entries.reserve(connections.size());
for (const auto & connection : connections)
connection_entries.emplace_back(std::make_shared<Entry>(connection->get(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings))));
pool.schedule(std::bind(&Benchmark::thread, this, connection_entries));
}
InterruptListener interrupt_listener;
delay_watch.restart();
/// Push queries into queue
for (size_t i = 0; !max_iterations || i < max_iterations; ++i)
{
size_t query_index = randomize ? distribution(generator) : i % queries.size();
if (!tryPushQueryInteractively(queries[query_index], interrupt_listener))
2018-10-11 18:25:05 +00:00
{
shutdown = true;
break;
2018-10-11 18:25:05 +00:00
}
}
pool.wait();
2019-08-05 13:00:27 +00:00
total_watch.stop();
if (!json_path.empty())
2019-08-05 13:00:27 +00:00
reportJSON(comparison_info_total, json_path);
2019-08-05 13:00:27 +00:00
printNumberOfQueriesExecuted(queries_executed);
report(comparison_info_total);
}
2019-08-05 13:00:27 +00:00
void thread(EntryPtrs & connection_entries)
{
Query query;
2019-08-06 13:19:06 +00:00
/// Randomly choosing connection index
pcg64 generator(randomSeed());
std::uniform_int_distribution<size_t> distribution(0, connection_entries.size() - 1);
try
{
/// In these threads we do not accept INT signal.
sigset_t sig_set;
if (sigemptyset(&sig_set)
|| sigaddset(&sig_set, SIGINT)
|| pthread_sigmask(SIG_BLOCK, &sig_set, nullptr))
throwFromErrno("Cannot block signal.", ErrorCodes::CANNOT_BLOCK_SIGNAL);
while (true)
{
bool extracted = false;
while (!extracted)
{
extracted = queue.tryPop(query, 100);
2018-10-11 18:12:28 +00:00
if (shutdown || (max_iterations && queries_executed == max_iterations))
return;
}
2019-08-06 13:19:06 +00:00
execute(connection_entries, query, distribution(generator));
2018-10-11 18:12:28 +00:00
++queries_executed;
}
}
catch (...)
{
shutdown = true;
std::cerr << "An error occurred while processing query:\n" << query << "\n";
throw;
}
}
2019-08-06 13:19:06 +00:00
void execute(EntryPtrs & connection_entries, Query & query, size_t connection_index)
{
Stopwatch watch;
2019-03-02 21:40:40 +00:00
RemoteBlockInputStream stream(
2019-08-05 13:00:27 +00:00
*(*connection_entries[connection_index]),
2019-03-02 21:40:40 +00:00
query, {}, global_context, &settings, nullptr, Tables(), query_processing_stage);
Progress progress;
stream.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); });
stream.readPrefix();
2019-08-05 13:00:27 +00:00
while (Block block = stream.read());
stream.readSuffix();
const BlockStreamProfileInfo & info = stream.getProfileInfo();
double seconds = watch.elapsedSeconds();
std::lock_guard lock(mutex);
2014-05-06 17:08:51 +00:00
2019-08-05 13:00:27 +00:00
comparison_info_per_interval[connection_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes);
comparison_info_total[connection_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes);
}
2014-05-06 17:08:51 +00:00
2019-08-05 13:00:27 +00:00
void report(MultiStats & infos)
{
std::lock_guard lock(mutex);
2019-08-05 13:00:27 +00:00
std::cerr << "\n";
size_t info_counter = 1;
for (auto & info : infos)
{
/// Avoid zeros, nans or exceptions
if (0 == info->queries)
return;
std::cerr
<< "connection " << info_counter++ << ", "
<< "queries " << info->queries << ", "
<< "QPS: " << (info->queries / info->work_time) << ", "
<< "RPS: " << (info->read_rows / info->work_time) << ", "
<< "MiB/s: " << (info->read_bytes / info->work_time / 1048576) << ", "
<< "result RPS: " << (info->result_rows / info->work_time) << ", "
<< "result MiB/s: " << (info->result_bytes / info->work_time / 1048576) << "."
2019-08-05 13:00:27 +00:00
<< "\n";
}
std::cerr << "\n\t\t";
2019-08-05 13:00:27 +00:00
for (size_t i = 1; i <= infos.size(); ++i)
std::cerr << "connection " << i << "\t";
2019-08-05 13:00:27 +00:00
std::cerr << "\n";
auto print_percentile = [&](double percent)
{
2019-08-05 13:00:27 +00:00
std::cerr << percent << "%\t\t";
for (auto & info : infos)
{
std::cerr << info->sampler.quantileInterpolated(percent / 100.0) << " sec." << "\t";
}
std::cerr << "\n";
};
2014-04-06 23:18:07 +00:00
for (int percent = 0; percent <= 90; percent += 10)
print_percentile(percent);
2016-09-30 12:39:18 +00:00
print_percentile(95);
print_percentile(99);
print_percentile(99.9);
print_percentile(99.99);
if (!cumulative)
for (auto & info : infos)
info->clear();
}
2019-08-05 13:00:27 +00:00
void reportJSON(MultiStats & infos, const std::string & filename)
{
WriteBufferFromFile json_out(filename);
std::lock_guard lock(mutex);
auto print_key_value = [&](auto key, auto value, bool with_comma = true)
{
json_out << double_quote << key << ": " << value << (with_comma ? ",\n" : "\n");
};
2019-08-05 13:00:27 +00:00
auto print_percentile = [&json_out](Stats & info, auto percent, bool with_comma = true)
{
json_out << "\"" << percent << "\"" << ": " << info.sampler.quantileInterpolated(percent / 100.0) << (with_comma ? ",\n" : "\n");
};
json_out << "{\n";
2016-09-30 12:39:18 +00:00
2019-08-05 13:00:27 +00:00
for (size_t i = 1; i <= infos.size(); ++i)
{
auto info = infos[i - 1];
2016-09-30 12:39:18 +00:00
2019-08-05 13:00:27 +00:00
json_out << double_quote << "connection_" + toString(i) << ": {\n";
json_out << double_quote << "statistics" << ": {\n";
2016-09-30 12:39:18 +00:00
print_key_value("QPS", info->queries / info->work_time);
print_key_value("RPS", info->read_rows / info->work_time);
print_key_value("MiBPS", info->read_bytes / info->work_time);
print_key_value("RPS_result", info->result_rows / info->work_time);
print_key_value("MiBPS_result", info->result_bytes / info->work_time);
2019-08-05 13:00:27 +00:00
print_key_value("num_queries", info->queries.load(), false);
2016-09-30 12:39:18 +00:00
2019-08-05 13:00:27 +00:00
json_out << "},\n";
json_out << double_quote << "query_time_percentiles" << ": {\n";
2016-09-30 12:39:18 +00:00
2019-08-05 13:00:27 +00:00
for (int percent = 0; percent <= 90; percent += 10)
print_percentile(*info, percent);
2019-08-05 13:00:27 +00:00
print_percentile(*info, 95);
print_percentile(*info, 99);
print_percentile(*info, 99.9);
print_percentile(*info, 99.99, false);
json_out << "}\n";
json_out << (i == infos.size() ? "}\n" : "},\n");
2019-08-05 13:00:27 +00:00
}
json_out << "}\n";
}
public:
~Benchmark()
{
shutdown = true;
}
};
}
#ifndef __clang__
#pragma GCC optimize("-fno-var-tracking-assignments")
#endif
int mainEntryClickHouseBenchmark(int argc, char ** argv)
{
using namespace DB;
bool print_stacktrace = true;
try
{
using boost::program_options::value;
boost::program_options::options_description desc("Allowed options");
desc.add_options()
2018-10-11 21:12:14 +00:00
("help", "produce help message")
("concurrency,c", value<unsigned>()->default_value(1), "number of parallel queries")
("delay,d", value<double>()->default_value(1), "delay between intermediate reports in seconds (set 0 to disable reports)")
("stage", value<std::string>()->default_value("complete"), "request query processing up to specified stage: complete,fetch_columns,with_mergeable_state")
2018-10-11 21:12:14 +00:00
("iterations,i", value<size_t>()->default_value(0), "amount of queries to be executed")
("timelimit,t", value<double>()->default_value(0.), "stop launch of queries after specified time limit")
("randomize,r", value<bool>()->default_value(false), "randomize order of execution")
("json", value<std::string>()->default_value(""), "write final report to specified file in JSON format")
("host,h", value<std::vector<std::string>>()->default_value(std::vector<std::string>{"localhost"}, "localhost"), "note that more than one host can be described")
("port,p", value<std::vector<UInt16>>()->default_value(std::vector<UInt16>{9000}, "9000"), "note that more than one port can be described")
("cumulative", "prints cumulative data instead of data per interval")
("secure,s", "Use TLS connection")
2018-10-11 21:12:14 +00:00
("user", value<std::string>()->default_value("default"), "")
("password", value<std::string>()->default_value(""), "")
("database", value<std::string>()->default_value("default"), "")
("stacktrace", "print stack traces of exceptions")
;
Settings settings;
settings.addProgramOptions(desc);
boost::program_options::variables_map options;
boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options);
boost::program_options::notify(options);
if (options.count("help"))
{
std::cout << "Usage: " << argv[0] << " [options] < queries.txt\n";
std::cout << desc << "\n";
return 1;
}
print_stacktrace = options.count("stacktrace");
UseSSL use_ssl;
Benchmark benchmark(
options["concurrency"].as<unsigned>(),
options["delay"].as<double>(),
2019-08-05 18:38:08 +00:00
options["host"].as<std::vector<std::string>>(),
options["port"].as<std::vector<UInt16>>(),
options.count("cumulative"),
2018-11-28 15:31:09 +00:00
options.count("secure"),
options["database"].as<std::string>(),
options["user"].as<std::string>(),
options["password"].as<std::string>(),
options["stage"].as<std::string>(),
options["randomize"].as<bool>(),
options["iterations"].as<size_t>(),
options["timelimit"].as<double>(),
options["json"].as<std::string>(),
settings);
return benchmark.run();
}
catch (...)
{
std::cerr << getCurrentExceptionMessage(print_stacktrace, true) << std::endl;
return getCurrentExceptionCode();
}
}