This commit is contained in:
Alexander Kuzmenkov 2020-06-20 02:03:13 +03:00
parent fbecf42dfc
commit 96368b7d0c

View File

@ -15,6 +15,7 @@
#include <AggregateFunctions/ReservoirSampler.h> #include <AggregateFunctions/ReservoirSampler.h>
#include <AggregateFunctions/registerAggregateFunctions.h> #include <AggregateFunctions/registerAggregateFunctions.h>
#include <boost/program_options.hpp> #include <boost/program_options.hpp>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/randomSeed.h> #include <Common/randomSeed.h>
#include <Common/clearPasswordFromCommandLine.h> #include <Common/clearPasswordFromCommandLine.h>
@ -52,71 +53,6 @@ namespace ErrorCodes
extern const int EMPTY_DATA_PASSED; extern const int EMPTY_DATA_PASSED;
} }
template <typename Event>
class EventQueue
{
std::mutex mutex;
std::condition_variable condvar;
std::deque<Event> queue;
bool is_closed = false;
const size_t max_queue_size;
public:
EventQueue(size_t max_queue_size_) : max_queue_size(max_queue_size_) {}
template <typename... Args>
bool push(Args && ... args)
{
std::unique_lock lock(mutex);
condvar.wait(lock,
[this]() { return queue.size() < max_queue_size || is_closed; });
if (is_closed)
{
return false;
}
queue.push_back(std::forward<Args>(args)...);
condvar.notify_all();
return true;
}
bool pop(Event & event)
{
std::unique_lock lock(mutex);
condvar.wait(lock, [this]() { return queue.size() > 0 || is_closed; });
if (queue.size() > 0)
{
event = queue.front();
queue.pop_front();
condvar.notify_all();
return true;
}
assert(is_closed);
return false;
}
void close()
{
std::unique_lock lock(mutex);
is_closed = true;
condvar.notify_all();
}
void clearAndClose()
{
std::unique_lock lock(mutex);
is_closed = true;
queue.clear();
condvar.notify_all();
}
};
class Benchmark : public Poco::Util::Application class Benchmark : public Poco::Util::Application
{ {
public: public:
@ -128,7 +64,7 @@ public:
const String & query_id_, bool continue_on_errors_, const String & query_id_, bool continue_on_errors_,
bool print_stacktrace_, const Settings & settings_) bool print_stacktrace_, const Settings & settings_)
: :
concurrency(concurrency_), delay(delay_), queue(concurrency_ * 2), randomize(randomize_), concurrency(concurrency_), delay(delay_), queue(concurrency), randomize(randomize_),
cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_), cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_),
json_path(json_path_), confidence(confidence_), query_id(query_id_), json_path(json_path_), confidence(confidence_), query_id(query_id_),
continue_on_errors(continue_on_errors_), continue_on_errors(continue_on_errors_),
@ -204,7 +140,8 @@ private:
using Queries = std::vector<Query>; using Queries = std::vector<Query>;
Queries queries; Queries queries;
EventQueue<Query> queue; using Queue = ConcurrentBoundedQueue<Query>;
Queue queue;
using ConnectionPoolUniq = std::unique_ptr<ConnectionPool>; using ConnectionPoolUniq = std::unique_ptr<ConnectionPool>;
using ConnectionPoolUniqs = std::vector<ConnectionPoolUniq>; using ConnectionPoolUniqs = std::vector<ConnectionPoolUniq>;
@ -224,6 +161,9 @@ private:
Context global_context; Context global_context;
QueryProcessingStage::Enum query_processing_stage; QueryProcessingStage::Enum query_processing_stage;
/// Don't execute new queries after timelimit or SIGINT or exception
std::atomic<bool> shutdown{false};
std::atomic<size_t> queries_executed{0}; std::atomic<size_t> queries_executed{0};
struct Stats struct Stats
@ -306,7 +246,13 @@ private:
/// Try push new query and check cancellation conditions /// Try push new query and check cancellation conditions
bool tryPushQueryInteractively(const String & query, InterruptListener & interrupt_listener) bool tryPushQueryInteractively(const String & query, InterruptListener & interrupt_listener)
{ {
if (!queue.push(query)) bool inserted = false;
while (!inserted)
{
inserted = queue.tryPush(query, 100);
if (shutdown)
{ {
/// An exception occurred in a worker /// An exception occurred in a worker
return false; return false;
@ -330,6 +276,7 @@ private:
cumulative ? report(comparison_info_total) : report(comparison_info_per_interval); cumulative ? report(comparison_info_total) : report(comparison_info_per_interval);
delay_watch.restart(); delay_watch.restart();
} }
}
return true; return true;
} }
@ -369,13 +316,10 @@ private:
if (!tryPushQueryInteractively(queries[query_index], interrupt_listener)) if (!tryPushQueryInteractively(queries[query_index], interrupt_listener))
{ {
// A stop condition ocurred, so clear all the queries that are shutdown = true;
// in queue.
queue.clearAndClose();
break; break;
} }
} }
queue.close();
pool.wait(); pool.wait();
total_watch.stop(); total_watch.stop();
@ -407,10 +351,18 @@ private:
while (true) while (true)
{ {
if (!queue.pop(query)) bool extracted = false;
while (!extracted)
{
extracted = queue.tryPop(query, 100);
if (shutdown
|| (max_iterations && queries_executed == max_iterations))
{ {
return; return;
} }
}
const auto connection_index = distribution(generator); const auto connection_index = distribution(generator);
try try
@ -423,7 +375,7 @@ private:
<< query << "'.\n"; << query << "'.\n";
if (!continue_on_errors) if (!continue_on_errors)
{ {
queue.clearAndClose(); shutdown = true;
throw; throw;
} }
else else
@ -586,8 +538,7 @@ public:
~Benchmark() override ~Benchmark() override
{ {
queue.clearAndClose(); shutdown = true;
pool.wait();
} }
}; };