mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
fixup
This commit is contained in:
parent
fbecf42dfc
commit
96368b7d0c
@ -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();
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user