2020-02-17 14:27:09 +00:00
# include <unistd.h>
2022-05-08 17:01:47 +00:00
# include <cstdlib>
# include <csignal>
2014-03-10 09:33:18 +00:00
# include <iostream>
# include <iomanip>
2023-10-23 14:48:30 +00:00
# include <optional>
2016-01-12 02:55:39 +00:00
# include <random>
2023-08-22 03:52:57 +00:00
# include <string_view>
2017-09-09 23:17:38 +00:00
# include <pcg_random.hpp>
2024-05-03 16:20:03 +00:00
# include <Poco/UUID.h>
# include <Poco/UUIDGenerator.h>
2014-03-10 09:33:18 +00:00
# include <Poco/Util/Application.h>
2017-04-01 09:19:00 +00:00
# include <Common/Stopwatch.h>
2019-01-11 19:12:36 +00:00
# include <Common/ThreadPool.h>
2017-04-01 09:19:00 +00:00
# include <AggregateFunctions/ReservoirSampler.h>
2017-09-06 20:01:44 +00:00
# include <AggregateFunctions/registerAggregateFunctions.h>
2014-03-10 09:33:18 +00:00
# include <boost/program_options.hpp>
2020-06-19 23:03:13 +00:00
# include <Common/ConcurrentBoundedQueue.h>
2017-04-01 09:19:00 +00:00
# include <Common/Exception.h>
# include <Common/randomSeed.h>
2020-06-14 19:23:05 +00:00
# include <Common/clearPasswordFromCommandLine.h>
2017-04-01 09:19:00 +00:00
# include <IO/ReadBufferFromFileDescriptor.h>
# include <IO/WriteBufferFromFile.h>
# include <IO/ReadHelpers.h>
# include <IO/WriteHelpers.h>
# include <IO/Operators.h>
2017-12-27 17:58:52 +00:00
# include <IO/ConnectionTimeouts.h>
2018-09-06 18:05:33 +00:00
# include <IO/UseSSL.h>
2021-10-15 20:18:20 +00:00
# include <QueryPipeline/RemoteQueryExecutor.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/Context.h>
# include <Client/Connection.h>
2018-06-05 20:09:51 +00:00
# include <Common/InterruptListener.h>
2023-10-23 14:48:30 +00:00
# include <Common/Config/ConfigProcessor.h>
# include <Common/Config/getClientConfigPath.h>
2019-08-23 15:47:27 +00:00
# include <Common/TerminalSize.h>
2019-08-28 00:15:08 +00:00
# include <Common/StudentTTest.h>
2023-03-22 07:49:22 +00:00
# include <Common/CurrentMetrics.h>
2023-10-23 14:48:30 +00:00
# include <Common/ErrorCodes.h>
2024-03-19 16:04:29 +00:00
# include <Core/BaseSettingsProgramOptions.h>
2014-03-10 09:33:18 +00:00
2021-07-01 23:30:18 +00:00
2016-09-28 16:49:59 +00:00
/** A tool for evaluating ClickHouse performance.
* The tool emulates a case with fixed amount of simultaneously executing queries .
2014-03-10 09:33:18 +00:00
*/
2023-03-22 07:49:22 +00:00
namespace CurrentMetrics
{
extern const Metric LocalThread ;
extern const Metric LocalThreadActive ;
2023-11-18 18:07:59 +00:00
extern const Metric LocalThreadScheduled ;
2023-03-22 07:49:22 +00:00
}
2014-03-10 09:33:18 +00:00
namespace DB
{
2019-08-21 08:53:41 +00:00
using Ports = std : : vector < UInt16 > ;
2023-08-22 03:52:57 +00:00
static constexpr std : : string_view DEFAULT_CLIENT_NAME = " benchmark " ;
2019-08-21 08:53:41 +00:00
2016-01-12 02:55:39 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int CANNOT_BLOCK_SIGNAL ;
2018-11-22 21:19:58 +00:00
extern const int EMPTY_DATA_PASSED ;
2016-01-12 02:55:39 +00:00
}
2018-11-28 14:41:51 +00:00
class Benchmark : public Poco : : Util : : Application
2014-03-10 09:33:18 +00:00
{
public :
2022-12-07 13:51:16 +00:00
Benchmark ( unsigned concurrency_ ,
double delay_ ,
Strings & & hosts_ ,
Ports & & ports_ ,
bool round_robin_ ,
bool cumulative_ ,
bool secure_ ,
const String & default_database_ ,
const String & user_ ,
const String & password_ ,
const String & quota_key_ ,
const String & stage ,
bool randomize_ ,
size_t max_iterations_ ,
double max_time_ ,
size_t confidence_ ,
const String & query_id_ ,
2024-05-03 16:20:03 +00:00
const String & query_id_prefix_ ,
2022-12-07 13:51:16 +00:00
const String & query_to_execute_ ,
size_t max_consecutive_errors_ ,
bool continue_on_errors_ ,
bool reconnect_ ,
bool display_client_side_time_ ,
bool print_stacktrace_ ,
const Settings & settings_ )
2016-09-28 16:49:59 +00:00
:
2022-12-07 13:51:16 +00:00
round_robin ( round_robin_ ) ,
concurrency ( concurrency_ ) ,
delay ( delay_ ) ,
queue ( concurrency ) ,
randomize ( randomize_ ) ,
cumulative ( cumulative_ ) ,
max_iterations ( max_iterations_ ) ,
max_time ( max_time_ ) ,
confidence ( confidence_ ) ,
query_id ( query_id_ ) ,
2024-05-03 16:20:03 +00:00
query_id_prefix ( query_id_prefix_ ) ,
2022-12-07 13:51:16 +00:00
query_to_execute ( query_to_execute_ ) ,
continue_on_errors ( continue_on_errors_ ) ,
max_consecutive_errors ( max_consecutive_errors_ ) ,
reconnect ( reconnect_ ) ,
2022-08-14 01:33:42 +00:00
display_client_side_time ( display_client_side_time_ ) ,
2022-12-07 13:51:16 +00:00
print_stacktrace ( print_stacktrace_ ) ,
settings ( settings_ ) ,
shared_context ( Context : : createShared ( ) ) ,
global_context ( Context : : createGlobal ( shared_context . get ( ) ) ) ,
2023-11-18 18:07:59 +00:00
pool ( CurrentMetrics : : LocalThread , CurrentMetrics : : LocalThreadActive , CurrentMetrics : : LocalThreadScheduled , concurrency )
2014-03-10 09:33:18 +00:00
{
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 ) ;
2021-07-20 19:49:15 +00:00
comparison_info_total . reserve ( round_robin ? 1 : connections_cnt ) ;
comparison_info_per_interval . reserve ( round_robin ? 1 : 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 ] ;
2020-07-12 18:54:13 +00:00
connections . emplace_back ( std : : make_unique < ConnectionPool > (
2020-09-14 21:55:43 +00:00
concurrency ,
cur_host , cur_port ,
2022-08-03 19:44:08 +00:00
default_database_ , user_ , password_ , quota_key_ ,
2021-07-20 19:49:15 +00:00
/* cluster_= */ " " ,
/* cluster_secret_= */ " " ,
2023-08-22 03:52:57 +00:00
/* client_name_= */ std : : string ( DEFAULT_CLIENT_NAME ) ,
2021-07-20 19:49:15 +00:00
Protocol : : Compression : : Enable ,
secure ) ) ;
if ( ! round_robin | | comparison_info_per_interval . empty ( ) )
{
comparison_info_per_interval . emplace_back ( std : : make_shared < Stats > ( ) ) ;
comparison_info_total . emplace_back ( std : : make_shared < Stats > ( ) ) ;
}
2019-08-05 13:00:27 +00:00
}
2021-04-10 23:33:54 +00:00
global_context - > makeGlobalContext ( ) ;
global_context - > setSettings ( settings ) ;
2023-08-22 03:52:57 +00:00
global_context - > setClientName ( std : : string ( DEFAULT_CLIENT_NAME ) ) ;
global_context - > setQueryKindInitial ( ) ;
2019-07-08 02:14:32 +00:00
2014-03-10 09:33:18 +00:00
std : : cerr < < std : : fixed < < std : : setprecision ( 3 ) ;
2017-04-01 07:20:54 +00:00
2017-09-06 20:01:44 +00:00
/// This is needed to receive blocks with columns of AggregateFunction data type
/// (example: when using stage = 'with_mergeable_state')
registerAggregateFunctions ( ) ;
2020-11-10 19:37:44 +00:00
query_processing_stage = QueryProcessingStage : : fromString ( stage ) ;
2018-11-28 14:41:51 +00:00
}
2020-01-21 08:54:26 +00:00
void initialize ( Poco : : Util : : Application & self [[maybe_unused]] ) override
2018-11-28 14:41:51 +00:00
{
std : : string home_path ;
2022-08-21 18:24:17 +00:00
const char * home_path_cstr = getenv ( " HOME " ) ; // NOLINT(concurrency-mt-unsafe)
2018-11-28 14:41:51 +00:00
if ( home_path_cstr )
home_path = home_path_cstr ;
2023-10-23 14:48:30 +00:00
std : : optional < std : : string > config_path ;
if ( config ( ) . has ( " config-file " ) )
config_path . emplace ( config ( ) . getString ( " config-file " ) ) ;
else
config_path = getClientConfigPath ( home_path ) ;
if ( config_path . has_value ( ) )
{
ConfigProcessor config_processor ( * config_path ) ;
auto loaded_config = config_processor . loadConfig ( ) ;
config ( ) . add ( loaded_config . configuration ) ;
}
2018-11-28 14:41:51 +00:00
}
2020-01-21 08:54:26 +00:00
int main ( const std : : vector < std : : string > & ) override
2018-11-28 14:41:51 +00:00
{
2014-03-10 09:33:18 +00:00
readQueries ( ) ;
2018-11-28 14:41:51 +00:00
runBenchmark ( ) ;
2018-11-28 14:50:02 +00:00
return 0 ;
2014-03-10 09:33:18 +00:00
}
private :
2019-08-05 13:00:27 +00:00
using Entry = ConnectionPool : : Entry ;
using EntryPtr = std : : shared_ptr < Entry > ;
using EntryPtrs = std : : vector < EntryPtr > ;
2017-04-01 07:20:54 +00:00
2021-07-20 19:49:15 +00:00
bool round_robin ;
2014-03-10 09:33:18 +00:00
unsigned concurrency ;
2014-04-06 01:19:43 +00:00
double delay ;
2017-04-01 07:20:54 +00:00
2019-08-05 13:00:27 +00:00
using Query = std : : string ;
2016-05-28 10:35:44 +00:00
using Queries = std : : vector < Query > ;
2014-03-10 09:33:18 +00:00
Queries queries ;
2017-04-01 07:20:54 +00:00
2020-06-19 23:03:13 +00:00
using Queue = ConcurrentBoundedQueue < Query > ;
Queue queue ;
2017-04-01 07:20:54 +00:00
2019-08-21 08:53:41 +00:00
using ConnectionPoolUniq = std : : unique_ptr < ConnectionPool > ;
using ConnectionPoolUniqs = std : : vector < ConnectionPoolUniq > ;
ConnectionPoolUniqs connections ;
2019-08-05 13:00:27 +00:00
2016-01-12 02:55:39 +00:00
bool randomize ;
2019-08-06 10:54:35 +00:00
bool cumulative ;
2016-09-28 16:49:59 +00:00
size_t max_iterations ;
double max_time ;
2020-02-27 17:21:01 +00:00
size_t confidence ;
2020-12-06 08:53:02 +00:00
String query_id ;
2024-05-03 16:20:03 +00:00
String query_id_prefix ;
2020-12-06 08:53:02 +00:00
String query_to_execute ;
2020-06-12 18:28:07 +00:00
bool continue_on_errors ;
2022-12-07 13:51:16 +00:00
size_t max_consecutive_errors ;
2021-01-31 09:36:55 +00:00
bool reconnect ;
2022-08-14 01:33:42 +00:00
bool display_client_side_time ;
2020-06-15 16:39:00 +00:00
bool print_stacktrace ;
2020-07-22 05:59:38 +00:00
const Settings & settings ;
2020-04-17 09:47:40 +00:00
SharedContextHolder shared_context ;
2021-05-31 14:49:02 +00:00
ContextMutablePtr global_context ;
2015-12-26 01:04:12 +00:00
QueryProcessingStage : : Enum query_processing_stage ;
2017-04-01 07:20:54 +00:00
2022-12-07 13:51:16 +00:00
std : : atomic < size_t > consecutive_errors { 0 } ;
2020-06-19 23:03:13 +00:00
/// 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 } ;
2014-05-06 18:02:57 +00:00
struct Stats
{
2017-02-01 20:30:46 +00:00
std : : atomic < size_t > queries { 0 } ;
2020-06-19 22:41:15 +00:00
size_t errors = 0 ;
2014-05-06 18:02:57 +00:00
size_t read_rows = 0 ;
size_t read_bytes = 0 ;
size_t result_rows = 0 ;
size_t result_bytes = 0 ;
2017-04-01 07:20:54 +00:00
2016-05-28 10:35:44 +00:00
using Sampler = ReservoirSampler < double > ;
2014-05-06 18:02:57 +00:00
Sampler sampler { 1 < < 16 } ;
2017-04-01 07:20:54 +00:00
2023-08-10 21:39:06 +00:00
void add ( double duration , size_t read_rows_inc , size_t read_bytes_inc , size_t result_rows_inc , size_t result_bytes_inc )
2014-05-06 18:02:57 +00:00
{
+ + queries ;
read_rows + = read_rows_inc ;
read_bytes + = read_bytes_inc ;
result_rows + = result_rows_inc ;
result_bytes + = result_bytes_inc ;
2023-08-10 21:39:06 +00:00
sampler . insert ( duration ) ;
2014-05-06 18:02:57 +00:00
}
2017-04-01 07:20:54 +00:00
2014-05-06 18:02:57 +00:00
void clear ( )
{
queries = 0 ;
read_rows = 0 ;
read_bytes = 0 ;
result_rows = 0 ;
result_bytes = 0 ;
sampler . clear ( ) ;
}
} ;
2017-04-01 07:20:54 +00:00
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 ;
2019-08-28 00:15:08 +00:00
StudentTTest t_test ;
2019-08-05 13:00:27 +00:00
Stopwatch total_watch ;
2017-02-01 20:30:46 +00:00
Stopwatch delay_watch ;
2017-04-01 07:20:54 +00:00
2016-05-28 10:15:36 +00:00
std : : mutex mutex ;
2017-04-01 07:20:54 +00:00
2016-08-02 01:46:05 +00:00
ThreadPool pool ;
2017-04-01 07:20:54 +00:00
2014-03-10 09:33:18 +00:00
void readQueries ( )
{
2020-12-07 08:35:44 +00:00
if ( query_to_execute . empty ( ) )
{
2020-12-06 08:53:02 +00:00
ReadBufferFromFileDescriptor in ( STDIN_FILENO ) ;
2017-04-01 07:20:54 +00:00
2020-12-06 08:53:02 +00:00
while ( ! in . eof ( ) )
{
String query ;
readText ( query , in ) ;
assertChar ( ' \n ' , in ) ;
if ( ! query . empty ( ) )
queries . emplace_back ( std : : move ( query ) ) ;
}
2017-04-01 07:20:54 +00:00
2020-12-06 08:53:02 +00:00
if ( queries . empty ( ) )
2023-01-23 13:16:14 +00:00
throw Exception ( ErrorCodes : : EMPTY_DATA_PASSED , " Empty list of queries. " ) ;
2020-12-07 08:35:44 +00:00
}
else
{
2020-12-06 08:53:02 +00:00
queries . emplace_back ( query_to_execute ) ;
2014-03-10 09:33:18 +00:00
}
2017-04-01 07:20:54 +00:00
2015-12-26 03:28:03 +00:00
std : : cerr < < " Loaded " < < queries . size ( ) < < " queries. \n " ;
2014-03-10 09:33:18 +00:00
}
2017-04-01 07:20:54 +00:00
2015-12-26 03:42:16 +00:00
void printNumberOfQueriesExecuted ( size_t num )
{
std : : cerr < < " \n Queries executed: " < < num ;
if ( queries . size ( ) > 1 )
std : : cerr < < " ( " < < ( num * 100.0 / queries . size ( ) ) < < " %) " ;
std : : cerr < < " . \n " ;
}
2017-04-01 07:20:54 +00:00
2017-02-01 20:30:46 +00:00
/// Try push new query and check cancellation conditions
bool tryPushQueryInteractively ( const String & query , InterruptListener & interrupt_listener )
2014-03-10 09:33:18 +00:00
{
2020-06-19 23:03:13 +00:00
bool inserted = false ;
2017-04-01 07:20:54 +00:00
2020-06-19 23:03:13 +00:00
while ( ! inserted )
2020-06-19 22:41:15 +00:00
{
2020-06-19 23:03:13 +00:00
inserted = queue . tryPush ( query , 100 ) ;
2017-04-01 07:20:54 +00:00
2020-06-19 23:03:13 +00:00
if ( shutdown )
{
/// An exception occurred in a worker
return false ;
}
2017-04-01 07:20:54 +00:00
2020-06-19 23:03:13 +00:00
if ( max_time > 0 & & total_watch . elapsedSeconds ( ) > = max_time )
{
2021-07-23 14:35:22 +00:00
std : : cout < < " Stopping launch of queries. "
< < " Requested time limit " < < max_time < < " seconds is exhausted. \n " ;
2020-06-19 23:03:13 +00:00
return false ;
}
if ( interrupt_listener . check ( ) )
{
std : : cout < < " Stopping launch of queries. SIGINT received. " < < std : : endl ;
return false ;
}
2023-08-10 21:39:06 +00:00
double seconds = delay_watch . elapsedSeconds ( ) ;
if ( delay > 0 & & seconds > delay )
2020-06-19 23:03:13 +00:00
{
printNumberOfQueriesExecuted ( queries_executed ) ;
2023-08-10 21:39:06 +00:00
cumulative
? report ( comparison_info_total , total_watch . elapsedSeconds ( ) )
: report ( comparison_info_per_interval , seconds ) ;
2020-06-19 23:03:13 +00:00
delay_watch . restart ( ) ;
}
2019-01-04 14:22:34 +00:00
}
2017-04-01 07:20:54 +00:00
2017-02-01 20:30:46 +00:00
return true ;
}
2017-04-01 07:20:54 +00:00
2018-11-28 14:41:51 +00:00
void runBenchmark ( )
2017-02-01 20:30:46 +00:00
{
2017-09-09 23:17:38 +00:00
pcg64 generator ( randomSeed ( ) ) ;
2017-02-01 20:30:46 +00:00
std : : uniform_int_distribution < size_t > distribution ( 0 , queries . size ( ) - 1 ) ;
2017-04-01 07:20:54 +00:00
2019-10-17 14:41:27 +00:00
try
2019-08-05 13:00:27 +00:00
{
2019-10-17 14:41:27 +00:00
for ( size_t i = 0 ; i < concurrency ; + + i )
2023-08-10 21:39:06 +00:00
pool . scheduleOrThrowOnError ( [ this ] ( ) mutable { thread ( ) ; } ) ;
2019-10-17 14:41:27 +00:00
}
catch ( . . . )
{
2021-07-21 06:29:28 +00:00
shutdown = true ;
2019-10-17 14:41:27 +00:00
pool . wait ( ) ;
throw ;
2019-08-05 13:00:27 +00:00
}
2017-04-01 07:20:54 +00:00
2017-02-01 20:30:46 +00:00
InterruptListener interrupt_listener ;
delay_watch . restart ( ) ;
2017-04-01 07:20:54 +00:00
2017-02-01 20:30:46 +00:00
/// 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 ( ) ;
2017-04-01 07:20:54 +00:00
2017-02-01 20:30:46 +00:00
if ( ! tryPushQueryInteractively ( queries [ query_index ] , interrupt_listener ) )
2018-10-11 18:25:05 +00:00
{
2020-06-19 23:03:13 +00:00
shutdown = true ;
2017-02-01 20:30:46 +00:00
break ;
2018-10-11 18:25:05 +00:00
}
2017-02-01 20:30:46 +00:00
}
2017-04-01 07:20:54 +00:00
2021-12-29 19:09:07 +00:00
/// Now we don't block the Ctrl+C signal and second signal will terminate the program without waiting.
interrupt_listener . unblock ( ) ;
2017-02-01 20:30:46 +00:00
pool . wait ( ) ;
2019-08-05 13:00:27 +00:00
total_watch . stop ( ) ;
2017-04-01 07:20:54 +00:00
2019-08-05 13:00:27 +00:00
printNumberOfQueriesExecuted ( queries_executed ) ;
2023-08-10 21:39:06 +00:00
report ( comparison_info_total , total_watch . elapsedSeconds ( ) ) ;
2014-03-10 09:33:18 +00:00
}
2017-04-01 07:20:54 +00:00
2023-08-10 21:39:06 +00:00
void thread ( )
2014-03-10 09:33:18 +00:00
{
2015-12-26 03:28:03 +00:00
Query query ;
2017-04-01 07:20:54 +00:00
2019-08-06 13:19:06 +00:00
/// Randomly choosing connection index
pcg64 generator ( randomSeed ( ) ) ;
2023-08-10 21:39:06 +00:00
std : : uniform_int_distribution < size_t > distribution ( 0 , connections . size ( ) - 1 ) ;
2019-08-06 13:19:06 +00:00
2020-06-12 18:28:07 +00:00
/// 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 ) )
2014-03-10 09:33:18 +00:00
{
2023-12-15 18:25:49 +00:00
throw ErrnoException ( ErrorCodes : : CANNOT_BLOCK_SIGNAL , " Cannot block signal " ) ;
2020-06-12 18:28:07 +00:00
}
while ( true )
{
2020-06-19 23:03:13 +00:00
bool extracted = false ;
while ( ! extracted )
2014-03-10 09:33:18 +00:00
{
2020-06-19 23:03:13 +00:00
extracted = queue . tryPop ( query , 100 ) ;
2021-07-23 14:35:22 +00:00
if ( shutdown | | ( max_iterations & & queries_executed = = max_iterations ) )
2020-06-19 23:03:13 +00:00
return ;
2020-06-12 18:28:07 +00:00
}
2020-06-19 22:41:15 +00:00
const auto connection_index = distribution ( generator ) ;
2020-06-12 18:28:07 +00:00
try
{
2023-08-10 21:39:06 +00:00
execute ( query , connection_index ) ;
2022-12-07 13:51:16 +00:00
consecutive_errors = 0 ;
2015-12-26 03:28:03 +00:00
}
2020-06-12 18:28:07 +00:00
catch ( . . . )
{
2021-07-23 14:35:22 +00:00
std : : lock_guard lock ( mutex ) ;
2021-07-23 14:41:32 +00:00
std : : cerr < < " An error occurred while processing the query " < < " ' " < < query < < " ' "
< < " : " < < getCurrentExceptionMessage ( false ) < < std : : endl ;
2022-12-07 13:51:16 +00:00
if ( ! ( continue_on_errors | | max_consecutive_errors > + + consecutive_errors ) )
2020-06-12 18:28:07 +00:00
{
2020-06-19 23:03:13 +00:00
shutdown = true ;
2020-06-12 18:28:07 +00:00
throw ;
}
2020-06-15 14:25:42 +00:00
else
{
2020-06-15 16:39:00 +00:00
std : : cerr < < getCurrentExceptionMessage ( print_stacktrace ,
2020-06-19 22:41:15 +00:00
true /*check embedded stack trace*/ ) < < std : : endl ;
2021-07-20 19:49:15 +00:00
size_t info_index = round_robin ? 0 : connection_index ;
2022-08-14 01:33:42 +00:00
+ + comparison_info_per_interval [ info_index ] - > errors ;
+ + comparison_info_total [ info_index ] - > errors ;
2020-06-15 14:25:42 +00:00
}
2020-06-12 18:28:07 +00:00
}
2020-06-15 14:25:42 +00:00
// Count failed queries toward executed, so that we'd reach
// max_iterations even if every run fails.
+ + queries_executed ;
2014-03-10 09:33:18 +00:00
}
}
2017-04-01 07:20:54 +00:00
2023-08-10 21:39:06 +00:00
void execute ( Query & query , size_t connection_index )
2014-03-10 09:33:18 +00:00
{
Stopwatch watch ;
2021-01-31 09:36:55 +00:00
2023-08-10 21:39:06 +00:00
ConnectionPool : : Entry entry = connections [ connection_index ] - > get (
ConnectionTimeouts : : getTCPTimeoutsWithoutFailover ( settings ) ) ;
2021-01-31 09:36:55 +00:00
if ( reconnect )
2023-08-10 21:39:06 +00:00
entry - > disconnect ( ) ;
2021-01-31 09:36:55 +00:00
2021-10-08 14:03:54 +00:00
RemoteQueryExecutor executor (
2023-08-10 21:39:06 +00:00
* entry , query , { } , global_context , nullptr , Scalars ( ) , Tables ( ) , query_processing_stage ) ;
2024-05-03 16:20:03 +00:00
2020-02-27 17:21:01 +00:00
if ( ! query_id . empty ( ) )
2021-10-08 14:03:54 +00:00
executor . setQueryId ( query_id ) ;
2024-05-03 16:20:03 +00:00
else if ( ! query_id_prefix . empty ( ) )
executor . setQueryId ( query_id_prefix + " _ " + Poco : : UUIDGenerator ( ) . createRandom ( ) . toString ( ) ) ;
2014-03-10 09:33:18 +00:00
2014-10-25 18:33:52 +00:00
Progress progress ;
2021-10-08 14:03:54 +00:00
executor . setProgressCallback ( [ & progress ] ( const Progress & value ) { progress . incrementPiecewiseAtomically ( value ) ; } ) ;
2017-04-01 07:20:54 +00:00
2022-03-08 15:42:29 +00:00
executor . sendQuery ( ClientInfo : : QueryKind : : INITIAL_QUERY ) ;
2021-10-15 20:18:20 +00:00
ProfileInfo info ;
2023-02-03 13:34:18 +00:00
while ( Block block = executor . readBlock ( ) )
2021-10-08 14:03:54 +00:00
info . update ( block ) ;
2019-08-05 13:00:27 +00:00
2021-10-08 14:03:54 +00:00
executor . finish ( ) ;
2014-05-06 17:34:22 +00:00
2023-08-10 21:39:06 +00:00
double duration = ( display_client_side_time | | progress . elapsed_ns = = 0 )
2022-08-14 01:33:42 +00:00
? watch . elapsedSeconds ( )
: progress . elapsed_ns / 1e9 ;
2014-03-10 09:33:18 +00:00
2019-01-04 12:10:00 +00:00
std : : lock_guard lock ( mutex ) ;
2014-05-06 17:08:51 +00:00
2021-07-20 19:49:15 +00:00
size_t info_index = round_robin ? 0 : connection_index ;
2023-08-10 21:39:06 +00:00
comparison_info_per_interval [ info_index ] - > add ( duration , progress . read_rows , progress . read_bytes , info . rows , info . bytes ) ;
comparison_info_total [ info_index ] - > add ( duration , progress . read_rows , progress . read_bytes , info . rows , info . bytes ) ;
t_test . add ( info_index , duration ) ;
2019-08-05 13:00:27 +00:00
}
2014-05-06 17:08:51 +00:00
2023-08-10 21:39:06 +00:00
void report ( MultiStats & infos , double seconds )
2014-03-10 09:33:18 +00:00
{
2019-01-04 12:10:00 +00:00
std : : lock_guard lock ( mutex ) ;
2014-03-10 09:33:18 +00:00
2019-08-05 13:00:27 +00:00
std : : cerr < < " \n " ;
2019-08-21 08:53:41 +00:00
for ( size_t i = 0 ; i < infos . size ( ) ; + + i )
2019-08-05 13:00:27 +00:00
{
2019-08-21 08:53:41 +00:00
const auto & info = infos [ i ] ;
2019-08-09 13:00:47 +00:00
2019-08-05 13:00:27 +00:00
/// Avoid zeros, nans or exceptions
if ( 0 = = info - > queries )
return ;
2021-07-20 19:49:15 +00:00
std : : string connection_description = connections [ i ] - > getDescription ( ) ;
if ( round_robin )
{
connection_description . clear ( ) ;
for ( const auto & conn : connections )
{
if ( ! connection_description . empty ( ) )
2021-07-24 16:10:37 +00:00
connection_description + = " , " ;
2021-07-20 19:49:15 +00:00
connection_description + = conn - > getDescription ( ) ;
}
}
2019-08-05 13:00:27 +00:00
std : : cerr
2021-07-20 19:49:15 +00:00
< < connection_description < < " , "
2023-08-10 21:39:06 +00:00
< < " queries: " < < info - > queries < < " , " ;
2020-06-19 22:41:15 +00:00
if ( info - > errors )
{
2023-08-10 21:39:06 +00:00
std : : cerr < < " errors: " < < info - > errors < < " , " ;
2020-06-19 22:41:15 +00:00
}
std : : cerr
2019-08-08 18:49:29 +00:00
< < " QPS: " < < ( info - > queries / seconds ) < < " , "
< < " RPS: " < < ( info - > read_rows / seconds ) < < " , "
< < " MiB/s: " < < ( info - > read_bytes / seconds / 1048576 ) < < " , "
< < " result RPS: " < < ( info - > result_rows / seconds ) < < " , "
< < " result MiB/s: " < < ( info - > result_bytes / seconds / 1048576 ) < < " . "
2019-08-05 13:00:27 +00:00
< < " \n " ;
}
std : : cerr < < " \n " ;
2014-03-10 09:33:18 +00:00
2016-09-28 16:49:59 +00:00
auto print_percentile = [ & ] ( double percent )
{
2019-08-05 13:00:27 +00:00
std : : cerr < < percent < < " % \t \t " ;
2019-08-09 15:18:21 +00:00
for ( const auto & info : infos )
2019-08-05 13:00:27 +00:00
{
2020-05-23 19:43:13 +00:00
std : : cerr < < info - > sampler . quantileNearest ( percent / 100.0 ) < < " sec. \t " ;
2019-08-05 13:00:27 +00:00
}
std : : cerr < < " \n " ;
2016-09-28 16:49:59 +00:00
} ;
2014-04-06 23:18:07 +00:00
2016-09-28 16:49:59 +00:00
for ( int percent = 0 ; percent < = 90 ; percent + = 10 )
print_percentile ( percent ) ;
2016-09-30 12:39:18 +00:00
2016-09-28 16:49:59 +00:00
print_percentile ( 95 ) ;
print_percentile ( 99 ) ;
print_percentile ( 99.9 ) ;
print_percentile ( 99.99 ) ;
2014-05-06 18:02:57 +00:00
2019-08-21 08:53:41 +00:00
std : : cerr < < " \n " < < t_test . compareAndReport ( confidence ) . second < < " \n " ;
2019-08-06 10:54:35 +00:00
if ( ! cumulative )
2019-08-08 18:49:29 +00:00
{
2019-08-06 10:54:35 +00:00
for ( auto & info : infos )
info - > clear ( ) ;
2019-08-08 18:49:29 +00:00
}
2014-03-10 09:33:18 +00:00
}
2016-09-28 16:49:59 +00:00
2017-02-01 20:30:46 +00:00
public :
2020-01-21 15:34:55 +00:00
~ Benchmark ( ) override
2017-02-01 20:30:46 +00:00
{
2020-06-19 23:03:13 +00:00
shutdown = true ;
2017-02-01 20:30:46 +00:00
}
2014-03-10 09:33:18 +00:00
} ;
}
2017-03-24 15:05:54 +00:00
int mainEntryClickHouseBenchmark ( int argc , char * * argv )
2014-03-10 09:33:18 +00:00
{
using namespace DB ;
2017-02-07 16:36:12 +00:00
bool print_stacktrace = true ;
2017-04-01 07:20:54 +00:00
2014-03-10 09:33:18 +00:00
try
{
2016-09-28 16:49:59 +00:00
using boost : : program_options : : value ;
2017-04-01 07:20:54 +00:00
2022-04-21 14:39:05 +00:00
/// Note: according to the standard, subsequent calls to getenv can mangle previous result.
/// So we copy the results to std::string.
std : : optional < std : : string > env_user_str ;
std : : optional < std : : string > env_password_str ;
2022-08-03 19:44:08 +00:00
std : : optional < std : : string > env_quota_key_str ;
2022-04-21 14:39:05 +00:00
2022-08-21 18:24:17 +00:00
const char * env_user = getenv ( " CLICKHOUSE_USER " ) ; // NOLINT(concurrency-mt-unsafe)
2022-04-21 14:39:05 +00:00
if ( env_user ! = nullptr )
env_user_str . emplace ( std : : string ( env_user ) ) ;
2022-08-21 18:24:17 +00:00
const char * env_password = getenv ( " CLICKHOUSE_PASSWORD " ) ; // NOLINT(concurrency-mt-unsafe)
2022-04-21 14:39:05 +00:00
if ( env_password ! = nullptr )
env_password_str . emplace ( std : : string ( env_password ) ) ;
2022-08-21 18:24:17 +00:00
const char * env_quota_key = getenv ( " CLICKHOUSE_QUOTA_KEY " ) ; // NOLINT(concurrency-mt-unsafe)
2022-08-03 19:44:08 +00:00
if ( env_quota_key ! = nullptr )
env_quota_key_str . emplace ( std : : string ( env_quota_key ) ) ;
2019-08-23 15:47:27 +00:00
boost : : program_options : : options_description desc = createOptionsDescription ( " Allowed options " , getTerminalWidth ( ) ) ;
2014-03-10 09:33:18 +00:00
desc . add_options ( )
2018-10-11 21:12:14 +00:00
( " help " , " produce help message " )
2022-04-21 14:39:05 +00:00
( " query,q " , value < std : : string > ( ) - > default_value ( " " ) , " query to execute " )
2018-10-11 21:12:14 +00:00
( " concurrency,c " , value < unsigned > ( ) - > default_value ( 1 ) , " number of parallel queries " )
2020-08-15 13:57:17 +00:00
( " delay,d " , value < double > ( ) - > default_value ( 1 ) , " delay between intermediate reports in seconds (set 0 to disable reports) " )
2021-06-04 06:43:56 +00:00
( " stage " , value < std : : string > ( ) - > default_value ( " complete " ) , " request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit " )
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 " )
2022-08-14 01:33:42 +00:00
( " randomize,r " , " randomize order of execution " )
2021-07-20 19:49:15 +00:00
( " host,h " , value < Strings > ( ) - > multitoken ( ) , " list of hosts " )
2022-04-21 14:39:05 +00:00
( " port " , value < Ports > ( ) - > multitoken ( ) , " list of ports " )
2022-08-14 01:33:42 +00:00
( " roundrobin " , " Instead of comparing queries for different --host/--port just pick one random --host/--port for every query and send query to it. " )
( " cumulative " , " prints cumulative data instead of data per interval " )
( " secure,s " , " Use TLS connection " )
2022-04-21 14:39:05 +00:00
( " user,u " , value < std : : string > ( ) - > default_value ( env_user_str . value_or ( " default " ) ) , " " )
( " password " , value < std : : string > ( ) - > default_value ( env_password_str . value_or ( " " ) ) , " " )
2022-08-14 01:33:42 +00:00
( " quota_key " , value < std : : string > ( ) - > default_value ( env_quota_key_str . value_or ( " " ) ) , " " )
2022-08-14 02:16:48 +00:00
( " database " , value < std : : string > ( ) - > default_value ( " default " ) , " " )
( " stacktrace " , " print stack traces of exceptions " )
( " confidence " , value < size_t > ( ) - > default_value ( 5 ) , " set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default) " )
( " query_id " , value < std : : string > ( ) - > default_value ( " " ) , " " )
2024-05-03 16:20:03 +00:00
( " query_id_prefix " , value < std : : string > ( ) - > default_value ( " " ) , " " )
2022-12-07 13:51:16 +00:00
( " max-consecutive-errors " , value < size_t > ( ) - > default_value ( 0 ) , " set number of allowed consecutive errors " )
2022-12-24 19:40:23 +00:00
( " ignore-error,continue_on_errors " , " continue testing even if a query fails " )
2022-08-14 02:16:48 +00:00
( " reconnect " , " establish new connection for every query " )
( " client-side-time " , " display the time including network communication instead of server-side time; note that for server versions before 22.8 we always display client-side time " )
2014-03-10 09:33:18 +00:00
;
2017-04-01 07:20:54 +00:00
2019-04-25 14:08:20 +00:00
Settings settings ;
2024-03-19 16:04:29 +00:00
addProgramOptions ( settings , desc ) ;
2019-04-25 14:08:20 +00:00
2014-03-10 09:33:18 +00:00
boost : : program_options : : variables_map options ;
boost : : program_options : : store ( boost : : program_options : : parse_command_line ( argc , argv , desc ) , options ) ;
2019-04-25 14:08:20 +00:00
boost : : program_options : : notify ( options ) ;
2017-04-01 07:20:54 +00:00
2020-06-14 19:23:05 +00:00
clearPasswordFromCommandLine ( argc , argv ) ;
2014-03-10 09:33:18 +00:00
if ( options . count ( " help " ) )
{
2023-04-20 22:54:34 +00:00
std : : cout < < " Usage: " < < argv [ 0 ] < < " [options] < queries.txt \n " ;
2015-12-26 03:28:03 +00:00
std : : cout < < desc < < " \n " ;
2024-02-11 19:00:37 +00:00
std : : cout < < " \n See also: https://clickhouse.com/docs/en/operations/utilities/clickhouse-benchmark/ \n " ;
return 0 ;
2014-03-10 09:33:18 +00:00
}
2017-04-01 07:20:54 +00:00
2017-02-07 16:36:12 +00:00
print_stacktrace = options . count ( " stacktrace " ) ;
2017-04-01 07:20:54 +00:00
2020-07-12 18:54:13 +00:00
/// NOTE Maybe clickhouse-benchmark should also respect .xml configuration of clickhouse-client.
UInt16 default_port = options . count ( " secure " ) ? DBMS_DEFAULT_SECURE_PORT : DBMS_DEFAULT_PORT ;
2018-09-06 18:05:33 +00:00
UseSSL use_ssl ;
2020-07-12 18:54:13 +00:00
Ports ports = options . count ( " port " )
? options [ " port " ] . as < Ports > ( )
: Ports ( { default_port } ) ;
2019-08-21 08:53:41 +00:00
Strings hosts = options . count ( " host " ) ? options [ " host " ] . as < Strings > ( ) : Strings ( { " localhost " } ) ;
2018-09-06 18:05:33 +00:00
2014-03-10 09:33:18 +00:00
Benchmark benchmark (
options [ " concurrency " ] . as < unsigned > ( ) ,
2014-04-06 01:19:43 +00:00
options [ " delay " ] . as < double > ( ) ,
2019-08-21 08:53:41 +00:00
std : : move ( hosts ) ,
std : : move ( ports ) ,
2021-07-20 19:49:15 +00:00
options . count ( " roundrobin " ) ,
2019-08-06 10:54:35 +00:00
options . count ( " cumulative " ) ,
2018-11-28 15:31:09 +00:00
options . count ( " secure " ) ,
2014-03-10 09:33:18 +00:00
options [ " database " ] . as < std : : string > ( ) ,
options [ " user " ] . as < std : : string > ( ) ,
2015-02-16 20:12:59 +00:00
options [ " password " ] . as < std : : string > ( ) ,
2022-08-03 19:44:08 +00:00
options [ " quota_key " ] . as < std : : string > ( ) ,
2015-12-26 01:04:12 +00:00
options [ " stage " ] . as < std : : string > ( ) ,
2022-08-14 01:33:42 +00:00
options . count ( " randomize " ) ,
2016-09-28 16:49:59 +00:00
options [ " iterations " ] . as < size_t > ( ) ,
options [ " timelimit " ] . as < double > ( ) ,
2019-08-09 10:46:57 +00:00
options [ " confidence " ] . as < size_t > ( ) ,
2020-02-27 17:21:01 +00:00
options [ " query_id " ] . as < std : : string > ( ) ,
2024-05-03 16:20:03 +00:00
options [ " query_id_prefix " ] . as < std : : string > ( ) ,
2020-12-06 08:53:02 +00:00
options [ " query " ] . as < std : : string > ( ) ,
2022-12-07 13:51:16 +00:00
options [ " max-consecutive-errors " ] . as < size_t > ( ) ,
2022-12-24 19:40:23 +00:00
options . count ( " ignore-error " ) ,
2021-01-31 09:36:55 +00:00
options . count ( " reconnect " ) ,
2022-08-14 01:33:42 +00:00
options . count ( " client-side-time " ) ,
2020-06-15 16:39:00 +00:00
print_stacktrace ,
2015-02-16 20:12:59 +00:00
settings ) ;
2018-11-28 14:41:51 +00:00
return benchmark . run ( ) ;
2014-03-10 09:33:18 +00:00
}
catch ( . . . )
{
2017-02-07 16:36:12 +00:00
std : : cerr < < getCurrentExceptionMessage ( print_stacktrace , true ) < < std : : endl ;
2017-02-01 20:30:46 +00:00
return getCurrentExceptionCode ( ) ;
2014-03-10 09:33:18 +00:00
}
}