2020-02-17 14:27:09 +00:00
# include <unistd.h>
2014-03-10 09:33:18 +00:00
# include <stdlib.h>
# include <fcntl.h>
# include <signal.h>
2016-01-12 02:55:39 +00:00
# include <time.h>
2014-03-10 09:33:18 +00:00
# include <iostream>
# include <fstream>
# include <iomanip>
2016-01-12 02:55:39 +00:00
# include <random>
2017-09-09 23:17:38 +00:00
# include <pcg_random.hpp>
2014-03-10 09:33:18 +00:00
# include <Poco/File.h>
# 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 <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>
2017-12-27 17:58:52 +00:00
# include <IO/ConnectionTimeouts.h>
2020-12-10 22:05:02 +00:00
# include <IO/ConnectionTimeoutsContext.h>
2018-09-06 18:05:33 +00:00
# include <IO/UseSSL.h>
2017-04-01 09:19:00 +00:00
# include <DataStreams/RemoteBlockInputStream.h>
# include <Interpreters/Context.h>
# include <Client/Connection.h>
2018-06-05 20:09:51 +00:00
# include <Common/InterruptListener.h>
2018-11-28 14:41:51 +00:00
# include <Common/Config/configReadClient.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>
2014-03-10 09:33: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
*/
namespace DB
{
2019-08-21 08:53:41 +00:00
using Ports = std : : vector < UInt16 > ;
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 :
2019-08-21 08:53:41 +00:00
Benchmark ( unsigned concurrency_ , double delay_ , Strings & & hosts_ , Ports & & ports_ ,
2019-08-06 10:54:35 +00:00
bool cumulative_ , bool secure_ , const String & default_database_ ,
2017-04-01 07:20:54 +00:00
const String & user_ , const String & password_ , const String & stage ,
bool randomize_ , size_t max_iterations_ , double max_time_ ,
2020-06-12 18:28:07 +00:00
const String & json_path_ , size_t confidence_ ,
2020-12-06 08:53:02 +00:00
const String & query_id_ , const String & query_to_execute_ , bool continue_on_errors_ ,
2021-01-31 09:36:55 +00:00
bool reconnect_ , bool print_stacktrace_ , const Settings & settings_ )
2017-04-01 07:20:54 +00:00
:
2020-06-19 23:03:13 +00:00
concurrency ( concurrency_ ) , delay ( delay_ ) , queue ( concurrency ) , randomize ( randomize_ ) ,
2019-08-06 10:54:35 +00:00
cumulative ( cumulative_ ) , max_iterations ( max_iterations_ ) , max_time ( max_time_ ) ,
2020-06-12 18:28:07 +00:00
json_path ( json_path_ ) , confidence ( confidence_ ) , query_id ( query_id_ ) ,
2021-01-31 09:36:55 +00:00
query_to_execute ( query_to_execute_ ) , continue_on_errors ( continue_on_errors_ ) , reconnect ( reconnect_ ) ,
2020-06-15 16:39:00 +00:00
print_stacktrace ( print_stacktrace_ ) , settings ( settings_ ) ,
2020-04-17 09:47:40 +00:00
shared_context ( Context : : createShared ( ) ) , global_context ( Context : : createGlobal ( shared_context . get ( ) ) ) ,
pool ( concurrency )
2017-04-01 07:20:54 +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 ) ;
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 ] ;
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 ,
default_database_ , user_ , password_ ,
" " , /* cluster */
" " , /* cluster_secret */
" 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 ( ) ;
2020-12-10 22:05:02 +00:00
global_context . setSettings ( settings ) ;
2019-07-08 02:14:32 +00:00
2017-04-01 07:20:54 +00:00
std : : cerr < < std : : fixed < < std : : setprecision ( 3 ) ;
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 ;
const char * home_path_cstr = getenv ( " HOME " ) ;
if ( home_path_cstr )
home_path = home_path_cstr ;
configReadClient ( config ( ) , home_path ) ;
}
2020-01-21 08:54:26 +00:00
int main ( const std : : vector < std : : string > & ) override
2018-11-28 14:41:51 +00:00
{
2017-04-01 07:20:54 +00:00
if ( ! json_path . empty ( ) & & Poco : : File ( json_path ) . exists ( ) ) /// Clear file with previous results
Poco : : File ( json_path ) . remove ( ) ;
readQueries ( ) ;
2018-11-28 14:41:51 +00:00
runBenchmark ( ) ;
2018-11-28 14:50:02 +00:00
return 0 ;
2017-04-01 07:20:54 +00:00
}
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
unsigned concurrency ;
double delay ;
2019-08-05 13:00:27 +00:00
using Query = std : : string ;
2017-04-01 07:20:54 +00:00
using Queries = std : : vector < Query > ;
Queries queries ;
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
2017-04-01 07:20:54 +00:00
bool randomize ;
2019-08-06 10:54:35 +00:00
bool cumulative ;
2017-04-01 07:20:54 +00:00
size_t max_iterations ;
double max_time ;
String json_path ;
2020-02-27 17:21:01 +00:00
size_t confidence ;
2020-12-06 08:53:02 +00:00
String query_id ;
String query_to_execute ;
2020-06-12 18:28:07 +00:00
bool continue_on_errors ;
2021-01-31 09:36:55 +00:00
bool reconnect ;
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 ;
2017-06-19 20:31:23 +00:00
Context global_context ;
2017-04-01 07:20:54 +00:00
QueryProcessingStage : : Enum query_processing_stage ;
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 } ;
2017-04-01 07:20:54 +00:00
struct Stats
{
std : : atomic < size_t > queries { 0 } ;
2020-06-19 22:41:15 +00:00
size_t errors = 0 ;
2017-04-01 07:20:54 +00:00
size_t read_rows = 0 ;
size_t read_bytes = 0 ;
size_t result_rows = 0 ;
size_t result_bytes = 0 ;
2019-08-06 10:54:35 +00:00
double work_time = 0 ;
2017-04-01 07:20:54 +00:00
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 ;
2019-08-06 10:54:35 +00:00
work_time + = seconds ;
2017-04-01 07:20:54 +00:00
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 ;
2019-08-06 10:54:35 +00:00
work_time = 0 ;
2017-04-01 07:20:54 +00:00
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 ;
2019-08-28 00:15:08 +00:00
StudentTTest t_test ;
2019-08-05 13:00:27 +00:00
Stopwatch total_watch ;
2017-04-01 07:20:54 +00:00
Stopwatch delay_watch ;
std : : mutex mutex ;
ThreadPool pool ;
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 ( ) )
throw Exception ( " Empty list of queries. " , ErrorCodes : : EMPTY_DATA_PASSED ) ;
2020-12-07 08:35:44 +00:00
}
else
{
2020-12-06 08:53:02 +00:00
queries . emplace_back ( query_to_execute ) ;
2017-04-01 07:20:54 +00:00
}
std : : cerr < < " Loaded " < < queries . size ( ) < < " queries. \n " ;
}
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 " ;
}
/// Try push new query and check cancellation conditions
bool tryPushQueryInteractively ( const String & query , InterruptListener & interrupt_listener )
{
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 )
{
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 received. " < < std : : endl ;
return false ;
}
if ( delay > 0 & & delay_watch . elapsedSeconds ( ) > delay )
{
printNumberOfQueriesExecuted ( queries_executed ) ;
cumulative ? report ( comparison_info_total ) : report ( comparison_info_per_interval ) ;
delay_watch . restart ( ) ;
}
2019-01-04 14:22:34 +00:00
}
2017-04-01 07:20:54 +00:00
return true ;
}
2018-11-28 14:41:51 +00:00
void runBenchmark ( )
2017-04-01 07:20:54 +00:00
{
2017-09-09 23:17:38 +00:00
pcg64 generator ( randomSeed ( ) ) ;
2017-04-01 07:20:54 +00:00
std : : uniform_int_distribution < size_t > distribution ( 0 , queries . size ( ) - 1 ) ;
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 )
{
EntryPtrs connection_entries ;
connection_entries . reserve ( connections . size ( ) ) ;
2019-08-05 13:00:27 +00:00
2019-10-17 14:41:27 +00:00
for ( const auto & connection : connections )
connection_entries . emplace_back ( std : : make_shared < Entry > (
connection - > get ( ConnectionTimeouts : : getTCPTimeoutsWithoutFailover ( settings ) ) ) ) ;
2019-08-05 13:00:27 +00:00
2020-05-18 08:08:55 +00:00
pool . scheduleOrThrowOnError ( [ this , connection_entries ] ( ) mutable { thread ( connection_entries ) ; } ) ;
2019-10-17 14:41:27 +00:00
}
}
catch ( . . . )
{
pool . wait ( ) ;
throw ;
2019-08-05 13:00:27 +00:00
}
2017-04-01 07:20:54 +00:00
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
{
2020-06-19 23:03:13 +00:00
shutdown = true ;
2017-04-01 07:20:54 +00:00
break ;
2018-10-11 18:25:05 +00:00
}
2017-04-01 07:20:54 +00:00
}
pool . wait ( ) ;
2019-08-05 13:00:27 +00:00
total_watch . stop ( ) ;
2017-04-01 07:20:54 +00:00
if ( ! json_path . empty ( ) )
2019-08-05 13:00:27 +00:00
reportJSON ( comparison_info_total , json_path ) ;
2017-04-01 07:20:54 +00:00
2019-08-05 13:00:27 +00:00
printNumberOfQueriesExecuted ( queries_executed ) ;
report ( comparison_info_total ) ;
2017-04-01 07:20:54 +00:00
}
2019-08-05 13:00:27 +00:00
void thread ( EntryPtrs & connection_entries )
2017-04-01 07:20:54 +00:00
{
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 ) ;
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 ) )
2017-04-01 07:20:54 +00:00
{
2020-06-12 18:28:07 +00:00
throwFromErrno ( " Cannot block signal. " , ErrorCodes : : CANNOT_BLOCK_SIGNAL ) ;
}
while ( true )
{
2020-06-19 23:03:13 +00:00
bool extracted = false ;
while ( ! extracted )
2017-04-01 07:20:54 +00:00
{
2020-06-19 23:03:13 +00:00
extracted = queue . tryPop ( query , 100 ) ;
if ( shutdown
| | ( max_iterations & & queries_executed = = max_iterations ) )
{
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
{
2020-06-19 22:41:15 +00:00
execute ( connection_entries , query , connection_index ) ;
2017-04-01 07:20:54 +00:00
}
2020-06-12 18:28:07 +00:00
catch ( . . . )
{
2020-06-19 23:04:42 +00:00
std : : cerr < < " An error occurred while processing the query ' "
2020-06-19 22:41:15 +00:00
< < query < < " '. \n " ;
2020-06-12 18:28:07 +00:00
if ( ! continue_on_errors )
{
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 ;
comparison_info_per_interval [ connection_index ] - > errors + + ;
comparison_info_total [ connection_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 ;
2017-04-01 07:20:54 +00:00
}
}
2019-08-06 13:19:06 +00:00
void execute ( EntryPtrs & connection_entries , Query & query , size_t connection_index )
2017-04-01 07:20:54 +00:00
{
Stopwatch watch ;
2021-01-31 09:36:55 +00:00
Connection & connection = * * connection_entries [ connection_index ] ;
if ( reconnect )
connection . disconnect ( ) ;
2019-03-02 21:40:40 +00:00
RemoteBlockInputStream stream (
2021-01-31 09:36:55 +00:00
connection , query , { } , global_context , nullptr , Scalars ( ) , Tables ( ) , query_processing_stage ) ;
2020-02-27 17:21:01 +00:00
if ( ! query_id . empty ( ) )
stream . setQueryId ( query_id ) ;
2014-03-10 09:33:18 +00:00
2017-04-01 07:20:54 +00:00
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 ( ) ) ;
2017-04-01 07:20:54 +00:00
stream . readSuffix ( ) ;
2014-03-10 09:33:18 +00:00
2017-04-01 07:20:54 +00:00
const BlockStreamProfileInfo & info = stream . getProfileInfo ( ) ;
2014-05-06 17:34:22 +00:00
2017-04-01 07:20:54 +00:00
double seconds = watch . elapsedSeconds ( ) ;
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
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 ) ;
2019-08-21 08:53:41 +00:00
t_test . add ( connection_index , seconds ) ;
2019-08-05 13:00:27 +00:00
}
2014-05-06 17:08:51 +00:00
2019-08-05 13:00:27 +00:00
void report ( MultiStats & infos )
2017-04-01 07:20:54 +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 ;
2019-08-08 18:49:29 +00:00
double seconds = info - > work_time / concurrency ;
2019-08-05 13:00:27 +00:00
std : : cerr
2019-08-21 08:53:41 +00:00
< < connections [ i ] - > getDescription ( ) < < " , "
2020-06-19 22:41:15 +00:00
< < " queries " < < info - > queries < < " , " ;
if ( info - > errors )
{
std : : cerr < < " errors " < < info - > errors < < " , " ;
}
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
2017-04-01 07:20:54 +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 " ;
2017-04-01 07:20:54 +00:00
} ;
2014-04-06 23:18:07 +00:00
2017-04-01 07:20:54 +00:00
for ( int percent = 0 ; percent < = 90 ; percent + = 10 )
print_percentile ( percent ) ;
2016-09-30 12:39:18 +00:00
2017-04-01 07:20:54 +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
}
2017-04-01 07:20:54 +00:00
}
2016-09-28 16:49:59 +00:00
2019-08-05 13:00:27 +00:00
void reportJSON ( MultiStats & infos , const std : : string & filename )
2017-04-01 07:20:54 +00:00
{
WriteBufferFromFile json_out ( filename ) ;
2016-09-28 16:49:59 +00:00
2019-01-04 12:10:00 +00:00
std : : lock_guard lock ( mutex ) ;
2016-09-28 16:49:59 +00:00
2017-04-01 07:20:54 +00:00
auto print_key_value = [ & ] ( auto key , auto value , bool with_comma = true )
{
json_out < < double_quote < < key < < " : " < < value < < ( with_comma ? " , \n " : " \n " ) ;
} ;
2016-09-28 16:49:59 +00:00
2019-08-05 13:00:27 +00:00
auto print_percentile = [ & json_out ] ( Stats & info , auto percent , bool with_comma = true )
2017-04-01 07:20:54 +00:00
{
2020-05-23 19:43:13 +00:00
json_out < < " \" " < < percent < < " \" : " < < info . sampler . quantileNearest ( percent / 100.0 ) < < ( with_comma ? " , \n " : " \n " ) ;
2017-04-01 07:20:54 +00:00
} ;
2016-09-28 16:49:59 +00:00
2017-04-01 07:20:54 +00:00
json_out < < " { \n " ;
2016-09-30 12:39:18 +00:00
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 ] ;
2016-09-30 12:39:18 +00:00
2019-08-21 08:53:41 +00:00
json_out < < double_quote < < connections [ i ] - > getDescription ( ) < < " : { \n " ;
2020-06-11 17:53:33 +00:00
json_out < < double_quote < < " statistics " < < " : { \n " ;
2016-09-30 12:39:18 +00:00
2019-08-06 10:54:35 +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 ) ;
2020-06-19 22:41:15 +00:00
print_key_value ( " num_queries " , info - > queries . load ( ) ) ;
print_key_value ( " num_errors " , info - > errors , false ) ;
2016-09-30 12:39:18 +00:00
2019-08-05 13:00:27 +00:00
json_out < < " }, \n " ;
2020-06-11 17:53:33 +00:00
json_out < < double_quote < < " query_time_percentiles " < < " : { \n " ;
2016-09-30 12:39:18 +00:00
2020-06-19 22:41:15 +00:00
if ( info - > queries ! = 0 )
{
for ( int percent = 0 ; percent < = 90 ; percent + = 10 )
print_percentile ( * info , percent ) ;
2016-09-30 12:39:18 +00:00
2020-06-19 22:41:15 +00:00
print_percentile ( * info , 95 ) ;
print_percentile ( * info , 99 ) ;
print_percentile ( * info , 99.9 ) ;
print_percentile ( * info , 99.99 , false ) ;
}
2016-09-28 16:49:59 +00:00
2019-08-05 13:00:27 +00:00
json_out < < " } \n " ;
2019-08-21 08:53:41 +00:00
json_out < < ( i = = infos . size ( ) - 1 ? " } \n " : " }, \n " ) ;
2019-08-05 13:00:27 +00:00
}
2016-09-28 16:49:59 +00:00
2017-04-01 07:20:54 +00:00
json_out < < " } \n " ;
}
2017-02-01 20:30:46 +00:00
public :
2020-01-21 15:34:55 +00:00
~ Benchmark ( ) override
2017-04-01 07:20:54 +00:00
{
2020-06-19 23:03:13 +00:00
shutdown = true ;
2017-04-01 07:20:54 +00:00
}
2014-03-10 09:33:18 +00:00
} ;
}
2018-09-01 02:22:30 +00:00
# ifndef __clang__
# pragma GCC optimize("-fno-var-tracking-assignments")
# endif
2019-12-15 06:34:43 +00:00
# pragma GCC diagnostic ignored "-Wmissing-declarations"
2018-09-01 02:22:30 +00:00
2017-03-24 15:05:54 +00:00
int mainEntryClickHouseBenchmark ( int argc , char * * argv )
2014-03-10 09:33:18 +00:00
{
2017-04-01 07:20:54 +00:00
using namespace DB ;
bool print_stacktrace = true ;
try
{
using boost : : program_options : : value ;
2019-08-23 15:47:27 +00:00
boost : : program_options : : options_description desc = createOptionsDescription ( " Allowed options " , getTerminalWidth ( ) ) ;
2017-04-01 07:20:54 +00:00
desc . add_options ( )
2018-10-11 21:12:14 +00:00
( " help " , " produce help message " )
2020-12-06 08:53:02 +00:00
( " query " , 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) " )
( " 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 " )
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 " )
2019-08-21 08:53:41 +00:00
( " host,h " , value < Strings > ( ) - > multitoken ( ) , " " )
( " port,p " , value < Ports > ( ) - > multitoken ( ) , " " )
2019-08-06 10:54:35 +00:00
( " cumulative " , " prints cumulative data instead of data per interval " )
2018-11-28 21:21:22 +00:00
( " 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 " )
2020-06-14 19:23:05 +00:00
( " 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) " )
2020-02-27 17:21:01 +00:00
( " query_id " , value < std : : string > ( ) - > default_value ( " " ) , " " )
2020-06-12 18:28:07 +00:00
( " continue_on_errors " , " continue testing even if a query fails " )
2021-01-31 09:36:55 +00:00
( " reconnect " , " establish new connection for every query " )
2017-04-01 07:20:54 +00:00
;
2019-04-25 14:08:20 +00:00
Settings settings ;
settings . addProgramOptions ( desc ) ;
2017-04-01 07:20:54 +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 ) ;
2017-04-01 07:20:54 +00:00
if ( options . count ( " help " ) )
{
std : : cout < < " Usage: " < < argv [ 0 ] < < " [options] < queries.txt \n " ;
std : : cout < < desc < < " \n " ;
return 1 ;
}
print_stacktrace = options . count ( " stacktrace " ) ;
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
2017-04-01 07:20:54 +00:00
Benchmark benchmark (
options [ " concurrency " ] . as < unsigned > ( ) ,
options [ " delay " ] . as < double > ( ) ,
2019-08-21 08:53:41 +00:00
std : : move ( hosts ) ,
std : : move ( ports ) ,
2019-08-06 10:54:35 +00:00
options . count ( " cumulative " ) ,
2018-11-28 15:31:09 +00:00
options . count ( " secure " ) ,
2017-04-01 07:20:54 +00:00
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 > ( ) ,
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 > ( ) ,
2020-12-06 08:53:02 +00:00
options [ " query " ] . as < std : : string > ( ) ,
2021-01-31 09:36:55 +00:00
options . count ( " continue_on_errors " ) ,
options . count ( " reconnect " ) ,
2020-06-15 16:39:00 +00:00
print_stacktrace ,
2017-04-01 07:20:54 +00:00
settings ) ;
2018-11-28 14:41:51 +00:00
return benchmark . run ( ) ;
2017-04-01 07:20:54 +00:00
}
catch ( . . . )
{
std : : cerr < < getCurrentExceptionMessage ( print_stacktrace , true ) < < std : : endl ;
return getCurrentExceptionCode ( ) ;
}
2014-03-10 09:33:18 +00:00
}