2021-07-22 21:27:26 +00:00
# include <Client/ClientBase.h>
2022-12-10 10:16:31 +00:00
# include <Client/LineReader.h>
# include <Client/ClientBaseHelpers.h>
# include <Client/TestHint.h>
# include <Client/InternalTextLogs.h>
# include <Client/TestTags.h>
2021-07-11 11:36:27 +00:00
2022-12-10 10:16:31 +00:00
# include <base/argsToConfig.h>
# include <base/safeExit.h>
# include <Core/Block.h>
2024-03-19 16:04:29 +00:00
# include <Core/BaseSettingsProgramOptions.h>
2022-12-10 10:16:31 +00:00
# include <Core/Protocol.h>
2021-12-21 13:41:53 +00:00
# include <Common/DateLUT.h>
2021-12-31 07:58:44 +00:00
# include <Common/MemoryTracker.h>
2022-04-27 15:05:45 +00:00
# include <Common/scope_guard_safe.h>
2022-01-18 06:51:13 +00:00
# include <Common/Exception.h>
# include <Common/getNumberOfPhysicalCPUCores.h>
# include <Common/typeid_cast.h>
2021-07-11 11:36:27 +00:00
# include <Common/TerminalSize.h>
# include <Common/clearPasswordFromCommandLine.h>
2024-05-19 08:02:06 +00:00
# include <Common/StringUtils.h>
2021-07-11 20:35:29 +00:00
# include <Common/filesystemHelpers.h>
2021-08-18 14:39:04 +00:00
# include <Common/NetException.h>
2024-03-08 20:37:30 +00:00
# include <Common/SignalHandlers.h>
2024-05-13 07:21:01 +00:00
# include <Common/tryGetFileNameByFileDescriptor.h>
2022-12-10 10:16:31 +00:00
# include <Columns/ColumnString.h>
# include <Columns/ColumnsNumber.h>
# include <Formats/FormatFactory.h>
2021-07-22 21:27:26 +00:00
2021-07-11 23:17:14 +00:00
# include <Parsers/parseQuery.h>
# include <Parsers/ParserQuery.h>
# include <Parsers/formatAST.h>
# include <Parsers/ASTInsertQuery.h>
2021-07-12 09:30:16 +00:00
# include <Parsers/ASTCreateQuery.h>
2022-05-01 15:24:41 +00:00
# include <Parsers/ASTCreateFunctionQuery.h>
2022-11-22 02:41:23 +00:00
# include <Parsers/Access/ASTCreateUserQuery.h>
2023-04-10 02:20:57 +00:00
# include <Parsers/Access/ASTAuthenticationData.h>
2021-07-12 09:30:16 +00:00
# include <Parsers/ASTDropQuery.h>
2023-03-24 02:44:52 +00:00
# include <Parsers/ASTSelectQuery.h>
2021-07-12 09:30:16 +00:00
# include <Parsers/ASTSetQuery.h>
# include <Parsers/ASTUseQuery.h>
# include <Parsers/ASTSelectWithUnionQuery.h>
# include <Parsers/ASTQueryWithOutput.h>
# include <Parsers/ASTLiteral.h>
# include <Parsers/ASTIdentifier.h>
2021-12-26 16:10:25 +00:00
# include <Parsers/ASTColumnDeclaration.h>
2022-07-07 22:16:01 +00:00
# include <Parsers/ASTFunction.h>
2023-07-20 10:54:42 +00:00
# include <Parsers/PRQL/ParserPRQLQuery.h>
2022-12-10 10:16:31 +00:00
# include <Parsers/Kusto/ParserKQLStatement.h>
2023-08-01 06:55:41 +00:00
# include <Parsers/Kusto/parseKQLQuery.h>
2021-07-12 09:30:16 +00:00
2021-10-13 10:01:08 +00:00
# include <Processors/Formats/Impl/NullFormat.h>
2021-08-18 14:39:04 +00:00
# include <Processors/Formats/IInputFormat.h>
2022-05-23 19:47:32 +00:00
# include <Processors/QueryPlan/QueryPlan.h>
# include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
# include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
2021-08-18 14:39:04 +00:00
# include <Processors/Executors/PullingAsyncPipelineExecutor.h>
# include <Processors/Transforms/AddingDefaultsTransform.h>
2022-12-10 10:16:31 +00:00
# include <QueryPipeline/QueryPipeline.h>
# include <QueryPipeline/QueryPipelineBuilder.h>
2021-08-17 19:59:51 +00:00
# include <Interpreters/ReplaceQueryParameterVisitor.h>
2021-12-14 07:25:30 +00:00
# include <Interpreters/ProfileEventsExt.h>
2021-07-11 23:17:14 +00:00
# include <IO/WriteBufferFromOStream.h>
2022-10-05 00:29:52 +00:00
# include <IO/WriteBufferFromFileDescriptor.h>
2021-08-18 21:01:17 +00:00
# include <IO/CompressionMethod.h>
2022-07-14 20:14:46 +00:00
# include <IO/ForkWriteBuffer.h>
2022-12-10 10:16:31 +00:00
# include <Access/AccessControl.h>
# include <Storages/ColumnsDescription.h>
2022-11-03 03:16:55 +00:00
# include <boost/algorithm/string/case_conv.hpp>
2023-04-22 14:21:42 +00:00
# include <boost/algorithm/string/replace.hpp>
2024-03-20 21:12:57 +00:00
# include <boost/algorithm/string/split.hpp>
2022-12-10 10:16:31 +00:00
# include <iostream>
# include <filesystem>
2023-10-16 20:31:42 +00:00
# include <limits>
2022-12-10 10:16:31 +00:00
# include <map>
2023-07-20 10:54:42 +00:00
# include <memory>
2022-12-10 10:16:31 +00:00
# include <unordered_map>
2023-11-13 09:09:23 +00:00
# include <Common/config_version.h>
2022-12-10 10:16:31 +00:00
# include "config.h"
2021-08-17 19:59:51 +00:00
2021-07-11 20:35:29 +00:00
namespace fs = std : : filesystem ;
2021-10-31 15:11:46 +00:00
using namespace std : : literals ;
2021-07-11 11:36:27 +00:00
2021-12-31 07:58:44 +00:00
namespace CurrentMetrics
{
extern const Metric MemoryTracking ;
}
2021-07-11 11:36:27 +00:00
namespace DB
{
namespace ErrorCodes
{
2021-07-18 14:42:41 +00:00
extern const int BAD_ARGUMENTS ;
2021-08-17 19:59:51 +00:00
extern const int DEADLOCK_AVOIDED ;
extern const int CLIENT_OUTPUT_FORMAT_SPECIFIED ;
extern const int UNKNOWN_PACKET_FROM_SERVER ;
2021-08-18 14:39:04 +00:00
extern const int NO_DATA_TO_INSERT ;
extern const int UNEXPECTED_PACKET_FROM_SERVER ;
2021-09-19 21:42:28 +00:00
extern const int INVALID_USAGE_OF_INPUT ;
2021-10-02 13:05:19 +00:00
extern const int CANNOT_SET_SIGNAL_HANDLER ;
2021-10-14 13:34:05 +00:00
extern const int UNRECOGNIZED_ARGUMENTS ;
2021-10-28 21:23:58 +00:00
extern const int LOGICAL_ERROR ;
2022-10-01 22:38:41 +00:00
extern const int CANNOT_OPEN_FILE ;
2023-06-16 08:10:06 +00:00
extern const int FILE_ALREADY_EXISTS ;
2023-08-10 04:11:07 +00:00
extern const int USER_SESSION_LIMIT_EXCEEDED ;
2023-10-20 16:39:48 +00:00
extern const int NOT_IMPLEMENTED ;
2023-10-24 13:44:00 +00:00
extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR ;
2024-04-22 01:36:34 +00:00
extern const int USER_EXPIRED ;
2021-07-11 11:36:27 +00:00
}
}
2021-09-14 13:24:57 +00:00
namespace ProfileEvents
{
extern const Event UserTimeMicroseconds ;
extern const Event SystemTimeMicroseconds ;
2021-07-31 12:34:29 +00:00
}
2023-03-14 11:36:26 +00:00
namespace
{
constexpr UInt64 THREAD_GROUP_ID = 0 ;
}
2021-07-31 12:34:29 +00:00
namespace DB
{
2021-07-11 20:35:29 +00:00
2022-11-12 02:55:26 +00:00
ProgressOption toProgressOption ( std : : string progress )
{
boost : : to_upper ( progress ) ;
if ( progress = = " OFF " | | progress = = " FALSE " | | progress = = " 0 " | | progress = = " NO " )
return ProgressOption : : OFF ;
if ( progress = = " TTY " | | progress = = " ON " | | progress = = " TRUE " | | progress = = " 1 " | | progress = = " YES " )
return ProgressOption : : TTY ;
if ( progress = = " ERR " )
return ProgressOption : : ERR ;
if ( progress = = " DEFAULT " )
return ProgressOption : : DEFAULT ;
throw boost : : program_options : : validation_error ( boost : : program_options : : validation_error : : invalid_option_value ) ;
}
2022-11-03 03:16:55 +00:00
std : : istream & operator > > ( std : : istream & in , ProgressOption & progress )
{
std : : string token ;
in > > token ;
2022-11-12 02:55:26 +00:00
progress = toProgressOption ( token ) ;
2022-11-03 03:16:55 +00:00
return in ;
}
2022-05-17 11:48:06 +00:00
static ClientInfo : : QueryKind parseQueryKind ( const String & query_kind )
{
if ( query_kind = = " initial_query " )
return ClientInfo : : QueryKind : : INITIAL_QUERY ;
if ( query_kind = = " secondary_query " )
return ClientInfo : : QueryKind : : SECONDARY_QUERY ;
if ( query_kind = = " no_query " )
return ClientInfo : : QueryKind : : NO_QUERY ;
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " Unknown query kind {} " , query_kind ) ;
}
2021-12-14 07:25:30 +00:00
static void incrementProfileEventsBlock ( Block & dst , const Block & src )
{
if ( ! dst )
{
2022-02-19 10:45:15 +00:00
dst = src . cloneEmpty ( ) ;
2021-12-14 07:25:30 +00:00
}
assertBlocksHaveEqualStructure ( src , dst , " ProfileEvents " ) ;
std : : unordered_map < String , size_t > name_pos ;
for ( size_t i = 0 ; i < dst . columns ( ) ; + + i )
name_pos [ dst . getByPosition ( i ) . name ] = i ;
size_t dst_rows = dst . rows ( ) ;
MutableColumns mutable_columns = dst . mutateColumns ( ) ;
auto & dst_column_host_name = typeid_cast < ColumnString & > ( * mutable_columns [ name_pos [ " host_name " ] ] ) ;
auto & dst_array_current_time = typeid_cast < ColumnUInt32 & > ( * mutable_columns [ name_pos [ " current_time " ] ] ) . getData ( ) ;
auto & dst_array_type = typeid_cast < ColumnInt8 & > ( * mutable_columns [ name_pos [ " type " ] ] ) . getData ( ) ;
auto & dst_column_name = typeid_cast < ColumnString & > ( * mutable_columns [ name_pos [ " name " ] ] ) ;
auto & dst_array_value = typeid_cast < ColumnInt64 & > ( * mutable_columns [ name_pos [ " value " ] ] ) . getData ( ) ;
const auto & src_column_host_name = typeid_cast < const ColumnString & > ( * src . getByName ( " host_name " ) . column ) ;
const auto & src_array_current_time = typeid_cast < const ColumnUInt32 & > ( * src . getByName ( " current_time " ) . column ) . getData ( ) ;
2022-04-12 16:51:39 +00:00
const auto & src_array_thread_id = typeid_cast < const ColumnUInt64 & > ( * src . getByName ( " thread_id " ) . column ) . getData ( ) ;
2021-12-14 07:25:30 +00:00
const auto & src_column_name = typeid_cast < const ColumnString & > ( * src . getByName ( " name " ) . column ) ;
const auto & src_array_value = typeid_cast < const ColumnInt64 & > ( * src . getByName ( " value " ) . column ) . getData ( ) ;
struct Id
{
StringRef name ;
StringRef host_name ;
bool operator < ( const Id & rhs ) const
{
2022-02-19 10:45:15 +00:00
return std : : tie ( name , host_name )
< std : : tie ( rhs . name , rhs . host_name ) ;
2021-12-14 07:25:30 +00:00
}
} ;
std : : map < Id , UInt64 > rows_by_name ;
2023-03-14 11:36:26 +00:00
2021-12-14 07:25:30 +00:00
for ( size_t src_row = 0 ; src_row < src . rows ( ) ; + + src_row )
{
2023-03-15 21:12:29 +00:00
/// Filter out threads stats, use stats from thread group
/// Exactly stats from thread group is stored to the table system.query_log
/// The stats from threads are less useful.
/// They take more records, they need to be combined,
/// there even could be several records from one thread.
/// Server doesn't send it any more to the clients, so this code left for compatible
2023-03-14 11:36:26 +00:00
auto thread_id = src_array_thread_id [ src_row ] ;
if ( thread_id ! = THREAD_GROUP_ID )
continue ;
2021-12-14 07:25:30 +00:00
Id id {
src_column_name . getDataAt ( src_row ) ,
src_column_host_name . getDataAt ( src_row ) ,
} ;
rows_by_name [ id ] = src_row ;
}
/// Merge src into dst.
for ( size_t dst_row = 0 ; dst_row < dst_rows ; + + dst_row )
{
Id id {
dst_column_name . getDataAt ( dst_row ) ,
dst_column_host_name . getDataAt ( dst_row ) ,
} ;
if ( auto it = rows_by_name . find ( id ) ; it ! = rows_by_name . end ( ) )
{
size_t src_row = it - > second ;
2022-04-12 16:51:39 +00:00
2021-12-14 07:25:30 +00:00
dst_array_current_time [ dst_row ] = src_array_current_time [ src_row ] ;
2023-09-26 18:44:14 +00:00
switch ( static_cast < ProfileEvents : : Type > ( dst_array_type [ dst_row ] ) )
2021-12-14 07:25:30 +00:00
{
case ProfileEvents : : Type : : INCREMENT :
dst_array_value [ dst_row ] + = src_array_value [ src_row ] ;
break ;
case ProfileEvents : : Type : : GAUGE :
dst_array_value [ dst_row ] = src_array_value [ src_row ] ;
break ;
}
rows_by_name . erase ( it ) ;
}
}
/// Copy rows from src that dst does not contains.
for ( const auto & [ id , pos ] : rows_by_name )
{
2022-04-12 16:51:39 +00:00
for ( size_t col = 0 ; col < src . columns ( ) ; + + col )
2022-02-19 10:45:15 +00:00
{
2022-04-12 16:51:39 +00:00
mutable_columns [ col ] - > insert ( ( * src . getByPosition ( col ) . column ) [ pos ] ) ;
2022-02-19 10:45:15 +00:00
}
}
2021-12-14 07:25:30 +00:00
dst . setColumns ( std : : move ( mutable_columns ) ) ;
}
2021-10-02 08:10:34 +00:00
2023-03-15 06:06:55 +00:00
std : : atomic < Int32 > exit_after_signals = 0 ;
2021-10-02 08:10:34 +00:00
class QueryInterruptHandler : private boost : : noncopyable
{
public :
2023-03-15 06:06:55 +00:00
/// Store how much interrupt signals can be before stopping the query
/// by default stop after the first interrupt signal.
static void start ( Int32 signals_before_stop = 1 ) { exit_after_signals . store ( signals_before_stop ) ; }
/// Set value not greater then 0 to mark the query as stopped.
2024-05-10 02:53:29 +00:00
static void stop ( ) { exit_after_signals . store ( 0 ) ; }
2023-03-15 06:06:55 +00:00
2022-04-08 05:48:20 +00:00
/// Return true if the query was stopped.
2023-03-15 13:05:38 +00:00
/// Query was stopped if it received at least "signals_before_stop" interrupt signals.
2023-03-15 06:06:55 +00:00
static bool try_stop ( ) { return exit_after_signals . fetch_sub ( 1 ) < = 0 ; }
static bool cancelled ( ) { return exit_after_signals . load ( ) < = 0 ; }
/// Return how much interrupt signals remain before stop.
static Int32 cancelled_status ( ) { return exit_after_signals . load ( ) ; }
2021-10-02 08:10:34 +00:00
} ;
2021-10-01 13:47:39 +00:00
2023-06-01 13:45:00 +00:00
/// This signal handler is set for SIGINT and SIGQUIT.
2021-10-01 17:31:00 +00:00
void interruptSignalHandler ( int signum )
2021-10-01 13:47:39 +00:00
{
2023-03-15 06:06:55 +00:00
if ( QueryInterruptHandler : : try_stop ( ) )
Fix signal-unsafe TSan report in client
CI founds [1]:
WARNING: ThreadSanitizer: signal-unsafe call inside of a signal (pid=2975)
0 malloc (clickhouse+0xab36d8d)
1 _dl_exception_create_format (ld-linux-x86-64.so.2+0x18ea8)
2 DB::interruptSignalHandler(int) obj-x86_64-linux-gnu/../src/Client/ClientBase.cpp:236:9 (clickhouse+0x1a1d603e)
3 __tsan::CallUserSignalHandler(__tsan::ThreadState*, bool, bool, bool, int, __sanitizer::__sanitizer_siginfo*, void*) crtstuff.c (clickhouse+0xab3ee5f)
4 unsigned long std::__1::__cxx_atomic_load(std::__1::__cxx_atomic_base_impl const*, std::__1::memory_order) obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:1006:12 (clickhouse+0x1da6c41d)
5 std::__1::__atomic_base::load(std::__1::memory_order) const obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:1615:17 (clickhouse+0x1da6c41d)
6 cctz::TimeZoneInfo::MakeTime(cctz::detail::civil_time const&) const obj-x86_64-linux-gnu/../contrib/cctz/src/time_zone_info.cc:844:47 (clickhouse+0x1da6c41d)
7 cctz::time_zone::Impl::MakeTime(cctz::detail::civil_time const&) const obj-x86_64-linux-gnu/../contrib/cctz/src/time_zone_impl.h:57:19 (clickhouse+0x1da64777)
8 cctz::time_zone::lookup(cctz::detail::civil_time const&) const obj-x86_64-linux-gnu/../contrib/cctz/src/time_zone_lookup.cc:72:27 (clickhouse+0x1da64777)
9 DateLUTImpl::DateLUTImpl(std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../src/Common/DateLUTImpl.cpp:70:63 (clickhouse+0xac8910b)
10 DateLUT::getImplementation(std::__1::basic_string, std::__1::allocator > const&) const obj-x86_64-linux-gnu/../src/Common/DateLUT.cpp:155:55 (clickhouse+0xac87404)
11 DateLUT::DateLUT() obj-x86_64-linux-gnu/../src/Common/DateLUT.cpp:145:25 (clickhouse+0xac8697f)
12 DateLUT::getInstance() obj-x86_64-linux-gnu/../src/Common/DateLUT.cpp:162:20 (clickhouse+0xac87530)
13 DateLUT::instance(std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../src/Common/DateLUT.h:29:33 (clickhouse+0x188481f9)
14 TimezoneMixin::TimezoneMixin(std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../src/DataTypes/TimezoneMixin.h:18:21 (clickhouse+0x188481f9)
15 DB::DataTypeDateTime::DataTypeDateTime(std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../src/DataTypes/DataTypeDateTime.cpp:11:7 (clickhouse+0x18847e55)
16 DB::(anonymous namespace)::FunctionEmptyArray::getNameImpl() emptyArray.cpp (clickhouse+0x1602abb1)
17 DB::registerFunctionsEmptyArray(DB::FunctionFactory&) (clickhouse+0x16023b6f)
18 DB::registerFunctionsArray(DB::FunctionFactory&) (clickhouse+0x15de7b99)
19 DB::registerFunctions() (clickhouse+0xe4e7bc6)
20 DB::Client::main(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&) obj-x86_64-linux-gnu/../programs/client/Client.cpp:402:5 (clickhouse+0xacb2649)
21 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse+0x1d96868a)
22 mainEntryClickHouseClient(int, char**) obj-x86_64-linux-gnu/../programs/client/Client.cpp:1237:23 (clickhouse+0xacbdd7c)
23 main obj-x86_64-linux-gnu/../programs/main.cpp:378:12 (clickhouse+0xabae77a)
[1]: https://s3.amazonaws.com/clickhouse-test-reports/34924/66cbb468eb6990b74ef4d08beefbe48d32bf4bd5/stateless_tests__thread__actions__[1/3].html
Fixes: #34923
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-03-02 10:01:33 +00:00
safeExit ( 128 + signum ) ;
2021-10-01 13:47:39 +00:00
}
2021-10-30 18:02:33 +00:00
2022-04-02 15:12:51 +00:00
/// To cancel the query on local format error.
class LocalFormatError : public DB : : Exception
{
public :
using Exception : : Exception ;
} ;
2024-03-08 20:37:30 +00:00
ClientBase : : ~ ClientBase ( )
{
writeSignalIDtoSignalPipe ( SignalListener : : StopThread ) ;
signal_listener_thread . join ( ) ;
HandledSignals : : instance ( ) . reset ( ) ;
}
2024-06-25 14:23:37 +00:00
ClientBase : : ClientBase (
int in_fd_ ,
int out_fd_ ,
int err_fd_ ,
std : : istream & input_stream_ ,
std : : ostream & output_stream_ ,
std : : ostream & error_stream_
)
: std_in ( in_fd_ )
, std_out ( out_fd_ )
, progress_indication ( output_stream_ , in_fd_ , err_fd_ )
, in_fd ( in_fd_ )
, out_fd ( out_fd_ )
, err_fd ( err_fd_ )
, input_stream ( input_stream_ )
, output_stream ( output_stream_ )
, error_stream ( error_stream_ )
{
stdin_is_a_tty = isatty ( in_fd ) ;
stdout_is_a_tty = isatty ( out_fd ) ;
stderr_is_a_tty = isatty ( err_fd ) ;
terminal_width = getTerminalWidth ( in_fd , err_fd ) ;
}
2021-10-30 18:02:33 +00:00
2021-10-02 08:10:34 +00:00
void ClientBase : : setupSignalHandler ( )
{
2022-04-08 05:48:20 +00:00
QueryInterruptHandler : : stop ( ) ;
2021-10-02 08:10:34 +00:00
2022-02-28 09:12:28 +00:00
struct sigaction new_act ;
memset ( & new_act , 0 , sizeof ( new_act ) ) ;
2021-10-02 13:05:19 +00:00
2022-02-28 09:12:28 +00:00
new_act . sa_handler = interruptSignalHandler ;
new_act . sa_flags = 0 ;
2021-10-02 13:05:19 +00:00
# if defined(OS_DARWIN)
2021-10-03 06:23:05 +00:00
sigemptyset ( & new_act . sa_mask ) ;
2021-10-02 13:05:19 +00:00
# else
2022-02-28 09:12:28 +00:00
if ( sigemptyset ( & new_act . sa_mask ) )
2023-12-15 18:25:49 +00:00
throw ErrnoException ( ErrorCodes : : CANNOT_SET_SIGNAL_HANDLER , " Cannot set signal handler " ) ;
2021-10-02 13:05:19 +00:00
# endif
2022-02-28 09:12:28 +00:00
if ( sigaction ( SIGINT , & new_act , nullptr ) )
2023-12-15 18:25:49 +00:00
throw ErrnoException ( ErrorCodes : : CANNOT_SET_SIGNAL_HANDLER , " Cannot set signal handler " ) ;
2023-06-01 13:45:00 +00:00
if ( sigaction ( SIGQUIT , & new_act , nullptr ) )
2023-12-15 18:25:49 +00:00
throw ErrnoException ( ErrorCodes : : CANNOT_SET_SIGNAL_HANDLER , " Cannot set signal handler " ) ;
2021-10-02 08:10:34 +00:00
}
2021-10-01 13:47:39 +00:00
2024-06-25 14:23:37 +00:00
ASTPtr ClientBase : : parseQuery ( const char * & pos , const char * end , const Settings & settings , bool allow_multi_statements )
2021-07-12 09:30:16 +00:00
{
2023-04-10 04:38:13 +00:00
std : : unique_ptr < IParserBase > parser ;
2021-07-12 09:30:16 +00:00
ASTPtr res ;
size_t max_length = 0 ;
if ( ! allow_multi_statements )
max_length = settings . max_query_size ;
2022-08-17 13:03:41 +00:00
const Dialect & dialect = settings . dialect ;
2022-06-08 17:14:03 +00:00
2022-08-17 13:03:41 +00:00
if ( dialect = = Dialect : : kusto )
2024-03-26 09:53:08 +00:00
parser = std : : make_unique < ParserKQLStatement > ( end , settings . allow_settings_after_format_in_insert ) ;
2023-07-20 10:54:42 +00:00
else if ( dialect = = Dialect : : prql )
2024-03-17 18:53:58 +00:00
parser = std : : make_unique < ParserPRQLQuery > ( max_length , settings . max_parser_depth , settings . max_parser_backtracks ) ;
2022-06-08 17:14:03 +00:00
else
2024-03-26 09:53:08 +00:00
parser = std : : make_unique < ParserQuery > ( end , settings . allow_settings_after_format_in_insert ) ;
2022-06-08 17:14:03 +00:00
2021-07-12 09:30:16 +00:00
if ( is_interactive | | ignore_error )
{
String message ;
2023-08-01 06:55:41 +00:00
if ( dialect = = Dialect : : kusto )
2024-03-17 18:53:58 +00:00
res = tryParseKQLQuery ( * parser , pos , end , message , true , " " , allow_multi_statements , max_length , settings . max_parser_depth , settings . max_parser_backtracks , true ) ;
2023-08-01 06:55:41 +00:00
else
2024-03-17 18:53:58 +00:00
res = tryParseQuery ( * parser , pos , end , message , true , " " , allow_multi_statements , max_length , settings . max_parser_depth , settings . max_parser_backtracks , true ) ;
2022-06-29 20:02:14 +00:00
if ( ! res )
{
2024-06-25 14:23:37 +00:00
error_stream < < std : : endl < < message < < std : : endl < < std : : endl ;
2022-07-16 14:49:24 +00:00
return nullptr ;
2021-07-12 09:30:16 +00:00
}
}
else
{
2023-08-01 06:55:41 +00:00
if ( dialect = = Dialect : : kusto )
2024-03-17 18:53:58 +00:00
res = parseKQLQueryAndMovePosition ( * parser , pos , end , " " , allow_multi_statements , max_length , settings . max_parser_depth , settings . max_parser_backtracks ) ;
2023-08-01 06:55:41 +00:00
else
2024-03-17 18:53:58 +00:00
res = parseQueryAndMovePosition ( * parser , pos , end , " " , allow_multi_statements , max_length , settings . max_parser_depth , settings . max_parser_backtracks ) ;
2021-07-12 09:30:16 +00:00
}
if ( is_interactive )
{
2024-06-25 14:23:37 +00:00
output_stream < < std : : endl ;
WriteBufferFromOStream res_buf ( output_stream , 4096 ) ;
2021-07-12 09:30:16 +00:00
formatAST ( * res , res_buf ) ;
2023-06-28 08:51:15 +00:00
res_buf . finalize ( ) ;
2024-06-25 14:23:37 +00:00
output_stream < < std : : endl < < std : : endl ;
2021-07-12 09:30:16 +00:00
}
return res ;
}
2021-10-30 18:02:33 +00:00
/// Consumes trailing semicolons and tries to consume the same-line trailing comment.
2024-03-17 18:53:58 +00:00
void ClientBase : : adjustQueryEnd ( const char * & this_query_end , const char * all_queries_end , uint32_t max_parser_depth , uint32_t max_parser_backtracks )
2021-07-12 09:30:16 +00:00
{
// We have to skip the trailing semicolon that might be left
// after VALUES parsing or just after a normal semicolon-terminated query.
Tokens after_query_tokens ( this_query_end , all_queries_end ) ;
2024-03-17 18:53:58 +00:00
IParser : : Pos after_query_iterator ( after_query_tokens , max_parser_depth , max_parser_backtracks ) ;
2021-07-12 09:30:16 +00:00
while ( after_query_iterator . isValid ( ) & & after_query_iterator - > type = = TokenType : : Semicolon )
{
this_query_end = after_query_iterator - > end ;
+ + after_query_iterator ;
}
// Now we have to do some extra work to add the trailing
// same-line comment to the query, but preserve the leading
// comments of the next query. The trailing comment is important
// because the test hints are usually written this way, e.g.:
// select nonexistent_column; -- { serverError 12345 }.
// The token iterator skips comments and whitespace, so we have
// to find the newline in the string manually. If it's earlier
// than the next significant token, it means that the text before
// newline is some trailing whitespace or comment, and we should
// add it to our query. There are also several special cases
// that are described below.
const auto * newline = find_first_symbols < ' \n ' > ( this_query_end , all_queries_end ) ;
const char * next_query_begin = after_query_iterator - > begin ;
// We include the entire line if the next query starts after
// it. This is a generic case of trailing in-line comment.
// The "equals" condition is for case of end of input (they both equal
// all_queries_end);
if ( newline < = next_query_begin )
{
assert ( newline > = this_query_end ) ;
this_query_end = newline ;
}
else
{
// Many queries on one line, can't do anything. By the way, this
// syntax is probably going to work as expected:
// select nonexistent /* { serverError 12345 } */; select 1
}
}
2021-08-17 19:59:51 +00:00
/// Convert external tables to ExternalTableData and send them using the connection.
void ClientBase : : sendExternalTables ( ASTPtr parsed_query )
{
const auto * select = parsed_query - > as < ASTSelectWithUnionQuery > ( ) ;
if ( ! select & & ! external_tables . empty ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " External tables could be sent only with select query " ) ;
2021-08-17 19:59:51 +00:00
std : : vector < ExternalTableDataPtr > data ;
for ( auto & table : external_tables )
data . emplace_back ( table . getData ( global_context ) ) ;
2024-04-25 11:37:33 +00:00
connection - > sendExternalTablesData ( data ) ;
2021-08-17 19:59:51 +00:00
}
void ClientBase : : onData ( Block & block , ASTPtr parsed_query )
{
if ( ! block )
return ;
2023-10-21 01:14:22 +00:00
processed_rows + = block . rows ( ) ;
2021-08-17 19:59:51 +00:00
/// Even if all blocks are empty, we still need to initialize the output stream to write empty resultset.
2022-05-09 19:13:02 +00:00
initOutputFormat ( block , parsed_query ) ;
2021-08-17 19:59:51 +00:00
/// The header block containing zero rows was used to initialize
2021-10-11 16:11:50 +00:00
/// output_format, do not output it.
2021-08-17 19:59:51 +00:00
/// Also do not output too much data if we're fuzzing.
if ( block . rows ( ) = = 0 | | ( query_fuzzer_runs ! = 0 & & processed_rows > = 100 ) )
return ;
2023-04-11 11:03:03 +00:00
2021-10-30 18:02:33 +00:00
/// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker.
2022-10-23 02:46:07 +00:00
if ( need_render_progress & & tty_buf & & ( ! select_into_file | | select_into_file_and_stdout ) )
2022-10-01 21:19:36 +00:00
progress_indication . clearProgressOutput ( * tty_buf ) ;
2021-08-17 19:59:51 +00:00
2022-04-12 12:15:21 +00:00
try
{
2023-10-21 01:14:22 +00:00
output_format - > write ( materializeBlock ( block ) ) ;
2022-04-12 12:15:21 +00:00
written_first_block = true ;
}
catch ( const Exception & )
{
/// Catch client errors like NO_ROW_DELIMITER
2023-01-23 13:16:14 +00:00
throw LocalFormatError ( getCurrentExceptionMessageAndPattern ( print_stack_trace ) , getCurrentExceptionCode ( ) ) ;
2022-04-12 12:15:21 +00:00
}
2021-08-17 19:59:51 +00:00
/// Received data block is immediately displayed to the user.
2021-10-11 16:11:50 +00:00
output_format - > flush ( ) ;
2021-08-17 19:59:51 +00:00
/// Restore progress bar after data block.
2022-10-23 02:46:07 +00:00
if ( need_render_progress & & tty_buf )
2021-10-30 18:02:33 +00:00
{
2022-07-05 18:03:51 +00:00
if ( select_into_file & & ! select_into_file_and_stdout )
2024-06-25 14:23:37 +00:00
error_stream < < " \r " ;
2022-10-01 21:19:36 +00:00
progress_indication . writeProgress ( * tty_buf ) ;
2021-10-30 18:02:33 +00:00
}
2021-08-17 19:59:51 +00:00
}
void ClientBase : : onLogData ( Block & block )
{
initLogsOutputStream ( ) ;
2022-10-23 02:46:07 +00:00
if ( need_render_progress & & tty_buf )
2022-10-01 21:19:36 +00:00
progress_indication . clearProgressOutput ( * tty_buf ) ;
2021-10-12 18:03:54 +00:00
logs_out_stream - > writeLogs ( block ) ;
2021-08-17 19:59:51 +00:00
logs_out_stream - > flush ( ) ;
}
void ClientBase : : onTotals ( Block & block , ASTPtr parsed_query )
{
2022-05-09 19:13:02 +00:00
initOutputFormat ( block , parsed_query ) ;
2023-02-21 18:15:16 +00:00
output_format - > setTotals ( materializeBlock ( block ) ) ;
2021-08-17 19:59:51 +00:00
}
void ClientBase : : onExtremes ( Block & block , ASTPtr parsed_query )
{
2022-05-09 19:13:02 +00:00
initOutputFormat ( block , parsed_query ) ;
2023-02-21 18:15:16 +00:00
output_format - > setExtremes ( materializeBlock ( block ) ) ;
2021-08-17 19:59:51 +00:00
}
void ClientBase : : onReceiveExceptionFromServer ( std : : unique_ptr < Exception > & & e )
{
have_error = true ;
server_exception = std : : move ( e ) ;
resetOutput ( ) ;
}
2021-10-15 20:18:20 +00:00
void ClientBase : : onProfileInfo ( const ProfileInfo & profile_info )
2021-08-17 19:59:51 +00:00
{
2021-10-11 16:11:50 +00:00
if ( profile_info . hasAppliedLimit ( ) & & output_format )
output_format - > setRowsBeforeLimit ( profile_info . getRowsBeforeLimit ( ) ) ;
2021-08-17 19:59:51 +00:00
}
2022-05-09 19:13:02 +00:00
void ClientBase : : initOutputFormat ( const Block & block , ASTPtr parsed_query )
2022-04-02 15:12:51 +00:00
try
2021-08-17 19:59:51 +00:00
{
2021-10-11 16:11:50 +00:00
if ( ! output_format )
2021-08-17 19:59:51 +00:00
{
/// Ignore all results when fuzzing as they can be huge.
if ( query_fuzzer_runs )
{
2021-10-11 16:11:50 +00:00
output_format = std : : make_shared < NullOutputFormat > ( block ) ;
2021-08-17 19:59:51 +00:00
return ;
}
WriteBuffer * out_buf = nullptr ;
if ( ! pager . empty ( ) )
{
2022-05-27 20:51:37 +00:00
if ( SIG_ERR = = signal ( SIGPIPE , SIG_IGN ) )
2023-12-15 18:25:49 +00:00
throw ErrnoException ( ErrorCodes : : CANNOT_SET_SIGNAL_HANDLER , " Cannot set signal handler for SIGPIPE " ) ;
2023-09-30 16:43:20 +00:00
/// We need to reset signals that had been installed in the
/// setupSignalHandler() since terminal will send signals to both
/// processes and so signals will be delivered to the
/// clickhouse-client/local as well, which will be terminated when
/// signal will be delivered second time.
if ( SIG_ERR = = signal ( SIGINT , SIG_IGN ) )
2023-12-15 18:25:49 +00:00
throw ErrnoException ( ErrorCodes : : CANNOT_SET_SIGNAL_HANDLER , " Cannot set signal handler for SIGINT " ) ;
2023-09-30 16:43:20 +00:00
if ( SIG_ERR = = signal ( SIGQUIT , SIG_IGN ) )
2023-12-15 18:25:49 +00:00
throw ErrnoException ( ErrorCodes : : CANNOT_SET_SIGNAL_HANDLER , " Cannot set signal handler for SIGQUIT " ) ;
2021-09-04 18:19:01 +00:00
ShellCommand : : Config config ( pager ) ;
config . pipe_stdin_only = true ;
pager_cmd = ShellCommand : : execute ( config ) ;
2021-08-17 19:59:51 +00:00
out_buf = & pager_cmd - > in ;
}
else
{
out_buf = & std_out ;
}
2024-03-23 01:42:22 +00:00
String current_format = default_output_format ;
2021-08-17 19:59:51 +00:00
2021-10-30 18:02:33 +00:00
select_into_file = false ;
2022-07-05 18:03:51 +00:00
select_into_file_and_stdout = false ;
2021-08-17 19:59:51 +00:00
/// The query can specify output format or output file.
if ( const auto * query_with_output = dynamic_cast < const ASTQueryWithOutput * > ( parsed_query . get ( ) ) )
{
2022-01-07 05:16:41 +00:00
String out_file ;
2021-08-17 19:59:51 +00:00
if ( query_with_output - > out_file )
{
2021-10-30 18:02:33 +00:00
select_into_file = true ;
2021-08-17 19:59:51 +00:00
const auto & out_file_node = query_with_output - > out_file - > as < ASTLiteral & > ( ) ;
2022-01-07 05:16:41 +00:00
out_file = out_file_node . value . safeGet < std : : string > ( ) ;
2021-08-17 19:59:51 +00:00
2022-07-07 01:47:33 +00:00
std : : string compression_method_string ;
2021-09-16 18:44:42 +00:00
if ( query_with_output - > compression )
{
const auto & compression_method_node = query_with_output - > compression - > as < ASTLiteral & > ( ) ;
2022-07-07 01:47:33 +00:00
compression_method_string = compression_method_node . value . safeGet < std : : string > ( ) ;
}
2022-07-06 14:15:24 +00:00
2022-07-07 01:47:33 +00:00
CompressionMethod compression_method = chooseCompressionMethod ( out_file , compression_method_string ) ;
UInt64 compression_level = 3 ;
2022-07-06 14:15:24 +00:00
2022-07-07 01:47:33 +00:00
if ( query_with_output - > compression_level )
{
const auto & compression_level_node = query_with_output - > compression_level - > as < ASTLiteral & > ( ) ;
2023-06-16 08:10:06 +00:00
compression_level_node . value . tryGet < UInt64 > ( compression_level ) ;
2021-09-16 18:44:42 +00:00
}
2022-07-14 20:14:46 +00:00
2023-04-18 07:14:35 +00:00
auto flags = O_WRONLY | O_EXCL ;
2023-06-29 21:31:40 +00:00
auto file_exists = fs : : exists ( out_file ) ;
if ( file_exists & & query_with_output - > is_outfile_append )
2023-04-18 07:14:35 +00:00
flags | = O_APPEND ;
2023-06-29 21:31:40 +00:00
else if ( file_exists & & query_with_output - > is_outfile_truncate )
2023-06-13 08:18:36 +00:00
flags | = O_TRUNC ;
2023-04-18 07:14:35 +00:00
else
flags | = O_CREAT ;
2022-07-18 16:12:13 +00:00
out_file_buf = wrapWriteBufferWithCompressionMethod (
2023-04-18 07:14:35 +00:00
std : : make_unique < WriteBufferFromFile > ( out_file , DBMS_DEFAULT_BUFFER_SIZE , flags ) ,
2022-07-18 16:12:13 +00:00
compression_method ,
2022-10-07 10:46:45 +00:00
static_cast < int > ( compression_level )
2022-07-18 16:12:13 +00:00
) ;
2022-07-14 20:14:46 +00:00
if ( query_with_output - > is_into_outfile_with_stdout )
2022-07-05 18:03:51 +00:00
{
select_into_file_and_stdout = true ;
2022-07-18 16:12:13 +00:00
out_file_buf = std : : make_unique < ForkWriteBuffer > ( std : : vector < WriteBufferPtr > { std : : move ( out_file_buf ) ,
2022-07-14 20:14:46 +00:00
std : : make_shared < WriteBufferFromFileDescriptor > ( STDOUT_FILENO ) } ) ;
2022-07-05 18:03:51 +00:00
}
2021-08-17 19:59:51 +00:00
// We are writing to file, so default format is the same as in non-interactive mode.
if ( is_interactive & & is_default_format )
current_format = " TabSeparated " ;
}
if ( query_with_output - > format ! = nullptr )
{
if ( has_vertical_output_suffix )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : CLIENT_OUTPUT_FORMAT_SPECIFIED , " Output format already specified " ) ;
2021-08-17 19:59:51 +00:00
const auto & id = query_with_output - > format - > as < ASTIdentifier & > ( ) ;
current_format = id . name ( ) ;
}
2022-01-11 13:26:14 +00:00
else if ( query_with_output - > out_file )
2022-01-07 05:16:41 +00:00
{
2024-01-23 09:35:41 +00:00
auto format_name = FormatFactory : : instance ( ) . tryGetFormatFromFileName ( out_file ) ;
if ( format_name )
current_format = * format_name ;
2022-01-07 05:16:41 +00:00
}
2021-08-17 19:59:51 +00:00
}
if ( has_vertical_output_suffix )
current_format = " Vertical " ;
2022-07-29 19:10:23 +00:00
bool logs_into_stdout = server_logs_file = = " - " ;
bool extras_into_stdout = need_render_progress | | logs_into_stdout ;
bool select_only_into_file = select_into_file & & ! select_into_file_and_stdout ;
2024-05-13 07:21:01 +00:00
if ( ! out_file_buf & & default_output_compression_method ! = CompressionMethod : : None )
out_file_buf = wrapWriteBufferWithCompressionMethod ( out_buf , default_output_compression_method , 3 , 0 ) ;
2022-07-29 19:10:23 +00:00
/// It is not clear how to write progress and logs
/// intermixed with data with parallel formatting.
2021-10-30 18:02:33 +00:00
/// It may increase code complexity significantly.
2022-07-29 19:10:23 +00:00
if ( ! extras_into_stdout | | select_only_into_file )
2021-10-30 18:02:33 +00:00
output_format = global_context - > getOutputFormatParallelIfPossible (
current_format , out_file_buf ? * out_file_buf : * out_buf , block ) ;
2021-08-17 19:59:51 +00:00
else
2021-10-30 18:02:33 +00:00
output_format = global_context - > getOutputFormat (
current_format , out_file_buf ? * out_file_buf : * out_buf , block ) ;
2021-08-17 19:59:51 +00:00
2021-11-02 13:40:41 +00:00
output_format - > setAutoFlush ( ) ;
2021-08-17 19:59:51 +00:00
}
}
2022-04-02 15:12:51 +00:00
catch ( . . . )
{
2023-01-23 13:16:14 +00:00
throw LocalFormatError ( getCurrentExceptionMessageAndPattern ( print_stack_trace ) , getCurrentExceptionCode ( ) ) ;
2022-04-02 15:12:51 +00:00
}
2021-08-17 19:59:51 +00:00
void ClientBase : : initLogsOutputStream ( )
{
if ( ! logs_out_stream )
{
WriteBuffer * wb = out_logs_buf . get ( ) ;
2022-07-14 18:45:37 +00:00
bool color_logs = false ;
2021-08-17 19:59:51 +00:00
if ( ! out_logs_buf )
{
if ( server_logs_file . empty ( ) )
{
/// Use stderr by default
out_logs_buf = std : : make_unique < WriteBufferFromFileDescriptor > ( STDERR_FILENO ) ;
wb = out_logs_buf . get ( ) ;
2022-07-14 18:45:37 +00:00
color_logs = stderr_is_a_tty ;
2021-08-17 19:59:51 +00:00
}
else if ( server_logs_file = = " - " )
{
/// Use stdout if --server_logs_file=- specified
wb = & std_out ;
2022-07-14 18:45:37 +00:00
color_logs = stdout_is_a_tty ;
2021-08-17 19:59:51 +00:00
}
else
{
out_logs_buf
= std : : make_unique < WriteBufferFromFile > ( server_logs_file , DBMS_DEFAULT_BUFFER_SIZE , O_WRONLY | O_APPEND | O_CREAT ) ;
wb = out_logs_buf . get ( ) ;
}
}
2022-07-14 18:45:37 +00:00
logs_out_stream = std : : make_unique < InternalTextLogs > ( * wb , color_logs ) ;
2021-08-17 19:59:51 +00:00
}
}
2023-10-16 20:24:23 +00:00
void ClientBase : : adjustSettings ( )
{
Settings settings = global_context - > getSettings ( ) ;
/// NOTE: Do not forget to set changed=false to avoid sending it to the server (to avoid breakage read only profiles)
/// In case of multi-query we allow data after semicolon since it will be
/// parsed by the client and interpreted as new query
if ( is_multiquery & & ! global_context - > getSettingsRef ( ) . input_format_values_allow_data_after_semicolon . changed )
{
settings . input_format_values_allow_data_after_semicolon = true ;
settings . input_format_values_allow_data_after_semicolon . changed = false ;
}
2024-05-16 10:09:47 +00:00
/// Do not limit pretty format output in case of --pager specified or in case of stdout is not a tty.
if ( ! pager . empty ( ) | | ! stdout_is_a_tty )
2023-10-16 20:31:42 +00:00
{
2024-04-02 08:51:41 +00:00
if ( ! global_context - > getSettingsRef ( ) . output_format_pretty_max_rows . changed )
{
settings . output_format_pretty_max_rows = std : : numeric_limits < UInt64 > : : max ( ) ;
settings . output_format_pretty_max_rows . changed = false ;
}
if ( ! global_context - > getSettingsRef ( ) . output_format_pretty_max_value_width . changed )
{
settings . output_format_pretty_max_value_width = std : : numeric_limits < UInt64 > : : max ( ) ;
settings . output_format_pretty_max_value_width . changed = false ;
}
2023-10-16 20:31:42 +00:00
}
2023-10-16 20:24:23 +00:00
global_context - > setSettings ( settings ) ;
}
2024-03-23 01:42:22 +00:00
bool ClientBase : : isRegularFile ( int fd )
{
struct stat file_stat ;
return fstat ( fd , & file_stat ) = = 0 & & S_ISREG ( file_stat . st_mode ) ;
}
2024-05-13 07:21:01 +00:00
void ClientBase : : setDefaultFormatsAndCompressionFromConfiguration ( )
2024-03-23 01:42:22 +00:00
{
2024-06-25 14:23:37 +00:00
if ( getClientConfiguration ( ) . has ( " output-format " ) )
2024-03-23 01:42:22 +00:00
{
2024-06-25 14:23:37 +00:00
default_output_format = getClientConfiguration ( ) . getString ( " output-format " ) ;
2024-03-23 01:42:22 +00:00
is_default_format = false ;
}
2024-06-25 14:23:37 +00:00
else if ( getClientConfiguration ( ) . has ( " format " ) )
2024-03-23 01:42:22 +00:00
{
2024-06-25 14:23:37 +00:00
default_output_format = getClientConfiguration ( ) . getString ( " format " ) ;
2024-03-23 01:42:22 +00:00
is_default_format = false ;
}
2024-06-25 14:23:37 +00:00
else if ( getClientConfiguration ( ) . has ( " vertical " ) )
2024-03-23 01:42:22 +00:00
{
default_output_format = " Vertical " ;
is_default_format = false ;
}
else if ( isRegularFile ( STDOUT_FILENO ) )
{
std : : optional < String > format_from_file_name = FormatFactory : : instance ( ) . tryGetFormatFromFileDescriptor ( STDOUT_FILENO ) ;
if ( format_from_file_name )
default_output_format = * format_from_file_name ;
else
default_output_format = " TSV " ;
2024-05-13 07:21:01 +00:00
std : : optional < String > file_name = tryGetFileNameFromFileDescriptor ( STDOUT_FILENO ) ;
if ( file_name )
default_output_compression_method = chooseCompressionMethod ( * file_name , " " ) ;
2024-03-23 01:42:22 +00:00
}
2024-03-29 15:16:15 +00:00
else if ( is_interactive )
2024-03-23 01:42:22 +00:00
{
default_output_format = " PrettyCompact " ;
}
else
{
default_output_format = " TSV " ;
}
2024-06-25 14:23:37 +00:00
if ( getClientConfiguration ( ) . has ( " input-format " ) )
2024-03-23 01:42:22 +00:00
{
2024-06-25 14:23:37 +00:00
default_input_format = getClientConfiguration ( ) . getString ( " input-format " ) ;
2024-03-23 01:42:22 +00:00
}
2024-06-25 14:23:37 +00:00
else if ( getClientConfiguration ( ) . has ( " format " ) )
2024-03-23 01:42:22 +00:00
{
2024-06-25 14:23:37 +00:00
default_input_format = getClientConfiguration ( ) . getString ( " format " ) ;
2024-03-23 01:42:22 +00:00
}
2024-06-25 14:23:37 +00:00
else if ( getClientConfiguration ( ) . getString ( " table-file " , " - " ) ! = " - " )
2024-03-23 01:42:22 +00:00
{
2024-06-25 14:23:37 +00:00
auto file_name = getClientConfiguration ( ) . getString ( " table-file " ) ;
2024-03-23 01:42:22 +00:00
std : : optional < String > format_from_file_name = FormatFactory : : instance ( ) . tryGetFormatFromFileName ( file_name ) ;
if ( format_from_file_name )
default_input_format = * format_from_file_name ;
else
default_input_format = " TSV " ;
}
else
{
std : : optional < String > format_from_file_name = FormatFactory : : instance ( ) . tryGetFormatFromFileDescriptor ( STDIN_FILENO ) ;
if ( format_from_file_name )
default_input_format = * format_from_file_name ;
else
default_input_format = " TSV " ;
}
2024-06-25 14:23:37 +00:00
format_max_block_size = getClientConfiguration ( ) . getUInt64 ( " format_max_block_size " ,
2024-03-23 01:42:22 +00:00
global_context - > getSettingsRef ( ) . max_block_size ) ;
/// Setting value from cmd arg overrides one from config
if ( global_context - > getSettingsRef ( ) . max_insert_block_size . changed )
{
insert_format_max_block_size = global_context - > getSettingsRef ( ) . max_insert_block_size ;
}
else
{
2024-06-25 14:23:37 +00:00
insert_format_max_block_size = getClientConfiguration ( ) . getUInt64 ( " insert_format_max_block_size " ,
2024-03-23 01:42:22 +00:00
global_context - > getSettingsRef ( ) . max_insert_block_size ) ;
}
}
2023-11-23 17:13:12 +00:00
void ClientBase : : initTTYBuffer ( ProgressOption progress )
2022-10-01 21:19:36 +00:00
{
2022-11-12 02:55:26 +00:00
if ( tty_buf )
return ;
if ( progress = = ProgressOption : : OFF | | ( ! is_interactive & & progress = = ProgressOption : : DEFAULT ) )
2022-10-01 21:19:36 +00:00
{
2022-11-12 02:55:26 +00:00
need_render_progress = false ;
return ;
}
2022-10-01 21:19:36 +00:00
2022-11-12 02:55:26 +00:00
static constexpr auto tty_file_name = " /dev/tty " ;
2022-10-01 21:19:36 +00:00
2022-11-12 02:55:26 +00:00
/// Output all progress bar commands to terminal at once to avoid flicker.
/// This size is usually greater than the window size.
static constexpr size_t buf_size = 1024 ;
if ( is_interactive | | progress = = ProgressOption : : TTY )
{
std : : error_code ec ;
std : : filesystem : : file_status tty = std : : filesystem : : status ( tty_file_name , ec ) ;
2022-10-01 21:19:36 +00:00
2022-11-12 02:55:26 +00:00
if ( ! ec & & exists ( tty ) & & is_character_file ( tty )
& & ( tty . permissions ( ) & std : : filesystem : : perms : : others_write ) ! = std : : filesystem : : perms : : none )
{
try
2022-10-01 22:38:41 +00:00
{
2022-11-12 02:55:26 +00:00
tty_buf = std : : make_unique < WriteBufferFromFile > ( tty_file_name , buf_size ) ;
2022-10-02 14:08:33 +00:00
2022-11-12 02:55:26 +00:00
/// It is possible that the terminal file has writeable permissions
/// but we cannot write anything there. Check it with invisible character.
tty_buf - > write ( ' \0 ' ) ;
tty_buf - > next ( ) ;
2022-10-02 14:08:33 +00:00
2022-11-12 02:55:26 +00:00
return ;
}
catch ( const Exception & e )
{
if ( tty_buf )
tty_buf . reset ( ) ;
2022-10-02 14:08:33 +00:00
2022-11-12 02:55:26 +00:00
if ( e . code ( ) ! = ErrorCodes : : CANNOT_OPEN_FILE )
throw ;
2022-10-01 22:38:41 +00:00
2022-11-12 02:55:26 +00:00
/// It is normal if file exists, indicated as writeable but still cannot be opened.
/// Fallback to other options.
2022-10-01 22:38:41 +00:00
}
2022-10-01 21:19:36 +00:00
}
2022-11-12 02:55:26 +00:00
}
2022-11-03 07:00:42 +00:00
2022-11-12 02:55:26 +00:00
if ( stderr_is_a_tty | | progress = = ProgressOption : : ERR )
{
tty_buf = std : : make_unique < WriteBufferFromFileDescriptor > ( STDERR_FILENO , buf_size ) ;
2022-10-01 21:19:36 +00:00
}
2022-11-12 02:55:26 +00:00
else
need_render_progress = false ;
2022-10-01 21:19:36 +00:00
}
2022-05-01 15:24:41 +00:00
void ClientBase : : updateSuggest ( const ASTPtr & ast )
2021-12-26 16:10:25 +00:00
{
std : : vector < std : : string > new_words ;
2022-05-01 15:24:41 +00:00
if ( auto * create = ast - > as < ASTCreateQuery > ( ) )
2021-12-26 16:10:25 +00:00
{
2022-05-01 15:24:41 +00:00
if ( create - > database )
new_words . push_back ( create - > getDatabase ( ) ) ;
new_words . push_back ( create - > getTable ( ) ) ;
if ( create - > columns_list & & create - > columns_list - > columns )
2021-12-26 16:10:25 +00:00
{
2022-05-01 15:24:41 +00:00
for ( const auto & elem : create - > columns_list - > columns - > children )
{
if ( const auto * column = elem - > as < ASTColumnDeclaration > ( ) )
new_words . push_back ( column - > name ) ;
}
2021-12-26 16:10:25 +00:00
}
}
2022-05-01 15:24:41 +00:00
if ( const auto * create_function = ast - > as < ASTCreateFunctionQuery > ( ) )
{
new_words . push_back ( create_function - > getFunctionName ( ) ) ;
}
if ( ! new_words . empty ( ) )
suggest - > addWords ( std : : move ( new_words ) ) ;
2021-12-26 16:10:25 +00:00
}
2021-08-17 19:59:51 +00:00
2022-02-02 17:53:04 +00:00
bool ClientBase : : isSyncInsertWithData ( const ASTInsertQuery & insert_query , const ContextPtr & context )
{
if ( ! insert_query . data )
return false ;
auto settings = context - > getSettings ( ) ;
if ( insert_query . settings_ast )
settings . applyChanges ( insert_query . settings_ast - > as < ASTSetQuery > ( ) - > changes ) ;
return ! settings . async_insert ;
}
2021-08-17 19:59:51 +00:00
2021-08-19 11:07:47 +00:00
void ClientBase : : processTextAsSingleQuery ( const String & full_query )
2021-08-18 14:39:04 +00:00
{
/// Some parts of a query (result output and formatting) are executed
/// client-side. Thus we need to parse the query.
const char * begin = full_query . data ( ) ;
2024-03-26 09:53:08 +00:00
auto parsed_query = parseQuery ( begin , begin + full_query . size ( ) ,
global_context - > getSettingsRef ( ) ,
2024-06-25 14:23:37 +00:00
/*allow_multi_statements=*/ false ) ;
2021-08-18 14:39:04 +00:00
if ( ! parsed_query )
return ;
String query_to_execute ;
2021-12-26 16:10:25 +00:00
/// Query will be parsed before checking the result because error does not
/// always means a problem, i.e. if table already exists, and it is no a
/// huge problem if suggestion will be added even on error, since this is
/// just suggestion.
2022-05-01 15:24:41 +00:00
///
/// Do not update suggest, until suggestion will be ready
/// (this will avoid extra complexity)
if ( suggest )
updateSuggest ( parsed_query ) ;
2021-12-26 16:10:25 +00:00
2022-02-03 02:13:48 +00:00
/// An INSERT query may have the data that follows query text.
/// Send part of the query without data, because data will be sent separately.
/// But for asynchronous inserts we don't extract data, because it's needed
/// to be done on server side in that case (for coalescing the data from multiple inserts on server side).
2022-02-02 17:53:04 +00:00
const auto * insert = parsed_query - > as < ASTInsertQuery > ( ) ;
if ( insert & & isSyncInsertWithData ( * insert , global_context ) )
2021-08-18 14:39:04 +00:00
query_to_execute = full_query . substr ( 0 , insert - > data - full_query . data ( ) ) ;
else
query_to_execute = full_query ;
2021-09-24 08:29:01 +00:00
try
{
2021-12-16 10:10:49 +00:00
processParsedSingleQuery ( full_query , query_to_execute , parsed_query , echo_queries ) ;
2021-09-24 08:29:01 +00:00
}
catch ( Exception & e )
{
if ( ! is_interactive )
2021-09-26 21:22:04 +00:00
e . addMessage ( " (in query: {}) " , full_query ) ;
2021-10-07 20:26:58 +00:00
throw ;
2021-09-24 08:29:01 +00:00
}
2021-08-18 14:39:04 +00:00
if ( have_error )
2021-08-19 11:07:47 +00:00
processError ( full_query ) ;
2021-08-18 14:39:04 +00:00
}
2021-08-17 19:59:51 +00:00
void ClientBase : : processOrdinaryQuery ( const String & query_to_execute , ASTPtr parsed_query )
{
2022-07-07 22:16:01 +00:00
auto query = query_to_execute ;
2021-08-20 12:17:51 +00:00
2021-08-17 19:59:51 +00:00
/// Rewrite query only when we have query parameters.
/// Note that if query is rewritten, comments in query are lost.
/// But the user often wants to see comments in server logs, query log, processlist, etc.
2022-08-12 12:28:35 +00:00
/// For recent versions of the server query parameters will be transferred by network and applied on the server side.
if ( ! query_parameters . empty ( )
& & connection - > getServerRevision ( connection_parameters . timeouts ) < DBMS_MIN_PROTOCOL_VERSION_WITH_PARAMETERS )
2021-08-17 19:59:51 +00:00
{
/// Replace ASTQueryParameter with ASTLiteral for prepared statements.
ReplaceQueryParameterVisitor visitor ( query_parameters ) ;
visitor . visit ( parsed_query ) ;
2021-10-12 23:51:11 +00:00
/// Get new query after substitutions.
2023-08-16 20:35:45 +00:00
if ( visitor . getNumberOfReplacedParameters ( ) )
query = serializeAST ( * parsed_query ) ;
chassert ( ! query . empty ( ) ) ;
2021-08-17 19:59:51 +00:00
}
2022-12-02 14:16:28 +00:00
if ( allow_merge_tree_settings & & parsed_query - > as < ASTCreateQuery > ( ) )
{
/// Rewrite query if new settings were added.
if ( addMergeTreeSettings ( * parsed_query - > as < ASTCreateQuery > ( ) ) )
{
/// Replace query parameters because AST cannot be serialized otherwise.
if ( ! query_parameters . empty ( ) )
{
ReplaceQueryParameterVisitor visitor ( query_parameters ) ;
visitor . visit ( parsed_query ) ;
}
query = serializeAST ( * parsed_query ) ;
}
}
2023-06-16 08:10:06 +00:00
// Run some local checks to make sure queries into output file will work before sending to server.
2023-06-16 08:47:28 +00:00
if ( const auto * query_with_output = dynamic_cast < const ASTQueryWithOutput * > ( parsed_query . get ( ) ) )
2023-06-16 08:10:06 +00:00
{
2023-06-16 08:47:28 +00:00
String out_file ;
if ( query_with_output - > out_file )
2023-06-16 08:10:06 +00:00
{
2023-06-16 08:47:28 +00:00
const auto & out_file_node = query_with_output - > out_file - > as < ASTLiteral & > ( ) ;
out_file = out_file_node . value . safeGet < std : : string > ( ) ;
std : : string compression_method_string ;
if ( query_with_output - > compression )
2023-06-16 08:10:06 +00:00
{
2023-06-16 08:47:28 +00:00
const auto & compression_method_node = query_with_output - > compression - > as < ASTLiteral & > ( ) ;
compression_method_string = compression_method_node . value . safeGet < std : : string > ( ) ;
}
2023-06-16 08:10:06 +00:00
2023-06-16 08:47:28 +00:00
CompressionMethod compression_method = chooseCompressionMethod ( out_file , compression_method_string ) ;
UInt64 compression_level = 3 ;
2023-06-16 08:10:06 +00:00
2023-06-16 08:47:28 +00:00
if ( query_with_output - > is_outfile_append & & query_with_output - > is_outfile_truncate )
{
throw Exception (
ErrorCodes : : BAD_ARGUMENTS ,
" Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously. " ) ;
}
2023-06-16 08:10:06 +00:00
2023-06-16 08:47:28 +00:00
if ( query_with_output - > is_outfile_append & & compression_method ! = CompressionMethod : : None )
{
throw Exception (
ErrorCodes : : BAD_ARGUMENTS ,
" Cannot append to compressed file. Please use uncompressed file or remove APPEND keyword. " ) ;
}
2023-06-16 08:10:06 +00:00
2023-06-16 08:47:28 +00:00
if ( query_with_output - > compression_level )
{
const auto & compression_level_node = query_with_output - > compression_level - > as < ASTLiteral & > ( ) ;
bool res = compression_level_node . value . tryGet < UInt64 > ( compression_level ) ;
auto range = getCompressionLevelRange ( compression_method ) ;
2023-06-16 08:10:06 +00:00
2023-06-16 08:47:28 +00:00
if ( ! res | | compression_level < range . first | | compression_level > range . second )
2023-06-16 08:10:06 +00:00
throw Exception (
ErrorCodes : : BAD_ARGUMENTS ,
2023-06-16 08:47:28 +00:00
" Invalid compression level, must be positive integer in range {}-{} " ,
range . first ,
range . second ) ;
}
2023-06-16 08:10:06 +00:00
2023-06-16 08:47:28 +00:00
if ( fs : : exists ( out_file ) )
{
if ( ! query_with_output - > is_outfile_append & & ! query_with_output - > is_outfile_truncate )
2023-06-16 08:10:06 +00:00
{
2023-06-16 08:47:28 +00:00
throw Exception (
ErrorCodes : : FILE_ALREADY_EXISTS ,
" File {} exists, consider using APPEND or TRUNCATE. " ,
out_file ) ;
2023-06-16 08:10:06 +00:00
}
}
}
}
2023-03-15 06:06:55 +00:00
const auto & settings = global_context - > getSettingsRef ( ) ;
2023-03-30 14:00:44 +00:00
const Int32 signals_before_stop = settings . partial_result_on_first_cancel ? 2 : 1 ;
2023-03-15 13:05:38 +00:00
2021-08-17 19:59:51 +00:00
int retries_left = 10 ;
2021-09-04 18:19:01 +00:00
while ( retries_left )
2021-08-17 19:59:51 +00:00
{
try
{
2023-03-15 06:06:55 +00:00
QueryInterruptHandler : : start ( signals_before_stop ) ;
2022-04-08 05:48:20 +00:00
SCOPE_EXIT ( { QueryInterruptHandler : : stop ( ) ; } ) ;
2021-08-17 19:59:51 +00:00
connection - > sendQuery (
connection_parameters . timeouts ,
query ,
2022-08-12 12:28:35 +00:00
query_parameters ,
2021-08-17 19:59:51 +00:00
global_context - > getCurrentQueryId ( ) ,
query_processing_stage ,
& global_context - > getSettingsRef ( ) ,
& global_context - > getClientInfo ( ) ,
2022-05-06 15:04:03 +00:00
true ,
[ & ] ( const Progress & progress ) { onProgress ( progress ) ; } ) ;
2021-08-17 19:59:51 +00:00
2021-09-24 08:29:01 +00:00
if ( send_external_tables )
sendExternalTables ( parsed_query ) ;
2023-03-30 14:00:44 +00:00
receiveResult ( parsed_query , signals_before_stop , settings . partial_result_on_first_cancel ) ;
2021-08-17 19:59:51 +00:00
break ;
}
catch ( const Exception & e )
{
/// Retry when the server said "Client should retry" and no rows
/// has been received yet.
if ( processed_rows = = 0 & & e . code ( ) = = ErrorCodes : : DEADLOCK_AVOIDED & & - - retries_left )
{
2024-06-25 14:23:37 +00:00
error_stream < < " Got a transient error from the server, will "
2021-08-17 19:59:51 +00:00
< < " retry ( " < < retries_left < < " retries left) " ;
}
else
{
throw ;
}
}
}
2021-09-04 18:19:01 +00:00
assert ( retries_left > 0 ) ;
2021-08-17 19:59:51 +00:00
}
/// Receives and processes packets coming from server.
/// Also checks if query execution should be cancelled.
2023-03-30 14:00:44 +00:00
void ClientBase : : receiveResult ( ASTPtr parsed_query , Int32 signals_before_stop , bool partial_result_on_first_cancel )
2021-08-17 19:59:51 +00:00
{
// TODO: get the poll_interval from commandline.
const auto receive_timeout = connection_parameters . timeouts . receive_timeout ;
constexpr size_t default_poll_interval = 1000000 ; /// in microseconds
constexpr size_t min_poll_interval = 5000 ; /// in microseconds
const size_t poll_interval
= std : : max ( min_poll_interval , std : : min < size_t > ( receive_timeout . totalMicroseconds ( ) , default_poll_interval ) ) ;
2021-10-16 19:48:51 +00:00
bool break_on_timeout = connection - > getConnectionType ( ) ! = IServerConnection : : Type : : LOCAL ;
2022-04-02 15:12:51 +00:00
std : : exception_ptr local_format_error ;
2021-08-17 19:59:51 +00:00
while ( true )
{
Stopwatch receive_watch ( CLOCK_MONOTONIC_COARSE ) ;
while ( true )
{
/// Has the Ctrl+C been pressed and thus the query should be cancelled?
/// If this is the case, inform the server about it and receive the remaining packets
/// to avoid losing sync.
if ( ! cancelled )
{
2023-03-30 14:00:44 +00:00
if ( partial_result_on_first_cancel & & QueryInterruptHandler : : cancelled_status ( ) = = signals_before_stop - 1 )
2023-03-15 06:06:55 +00:00
{
connection - > sendCancel ( ) ;
2023-03-15 10:50:01 +00:00
/// First cancel reading request was sent. Next requests will only be with a full cancel
2023-03-30 14:00:44 +00:00
partial_result_on_first_cancel = false ;
2023-03-15 06:06:55 +00:00
}
else if ( QueryInterruptHandler : : cancelled ( ) )
2021-08-17 19:59:51 +00:00
{
2022-04-27 13:53:35 +00:00
cancelQuery ( ) ;
2021-08-17 19:59:51 +00:00
}
else
{
double elapsed = receive_watch . elapsedSeconds ( ) ;
2021-10-16 19:48:51 +00:00
if ( break_on_timeout & & elapsed > receive_timeout . totalSeconds ( ) )
2021-08-17 19:59:51 +00:00
{
2024-06-25 14:23:37 +00:00
output_stream < < " Timeout exceeded while receiving data from server. "
2021-08-17 19:59:51 +00:00
< < " Waited for " < < static_cast < size_t > ( elapsed ) < < " seconds, "
< < " timeout is " < < receive_timeout . totalSeconds ( ) < < " seconds. " < < std : : endl ;
2022-04-27 13:53:35 +00:00
cancelQuery ( ) ;
2021-08-17 19:59:51 +00:00
}
}
}
/// Poll for changes after a cancellation check, otherwise it never reached
/// because of progress updates from server.
2021-08-23 07:13:27 +00:00
2021-08-17 19:59:51 +00:00
if ( connection - > poll ( poll_interval ) )
break ;
}
2022-04-02 15:12:51 +00:00
try
{
if ( ! receiveAndProcessPacket ( parsed_query , cancelled ) )
break ;
}
catch ( const LocalFormatError & )
{
2023-08-01 14:32:10 +00:00
/// Remember the first exception.
if ( ! local_format_error )
local_format_error = std : : current_exception ( ) ;
2022-04-02 15:12:51 +00:00
connection - > sendCancel ( ) ;
}
2021-08-17 19:59:51 +00:00
}
2022-04-02 15:12:51 +00:00
if ( local_format_error )
std : : rethrow_exception ( local_format_error ) ;
2024-07-02 15:48:10 +00:00
if ( cancelled & & is_interactive & & ! cancelled_printed . exchange ( true ) )
2024-06-25 14:23:37 +00:00
output_stream < < " Query was cancelled. " < < std : : endl ;
2021-08-17 19:59:51 +00:00
}
/// Receive a part of the result, or progress info or an exception and process it.
/// Returns true if one should continue receiving packets.
/// Output of result is suppressed if query was cancelled.
2022-02-26 20:40:24 +00:00
bool ClientBase : : receiveAndProcessPacket ( ASTPtr parsed_query , bool cancelled_ )
2021-08-17 19:59:51 +00:00
{
Packet packet = connection - > receivePacket ( ) ;
switch ( packet . type )
{
case Protocol : : Server : : PartUUIDs :
return true ;
case Protocol : : Server : : Data :
2022-02-26 20:40:24 +00:00
if ( ! cancelled_ )
2021-08-17 19:59:51 +00:00
onData ( packet . block , parsed_query ) ;
return true ;
case Protocol : : Server : : Progress :
onProgress ( packet . progress ) ;
return true ;
case Protocol : : Server : : ProfileInfo :
onProfileInfo ( packet . profile_info ) ;
return true ;
case Protocol : : Server : : Totals :
2022-02-26 20:40:24 +00:00
if ( ! cancelled_ )
2021-08-17 19:59:51 +00:00
onTotals ( packet . block , parsed_query ) ;
return true ;
case Protocol : : Server : : Extremes :
2022-02-26 20:40:24 +00:00
if ( ! cancelled_ )
2021-08-17 19:59:51 +00:00
onExtremes ( packet . block , parsed_query ) ;
return true ;
case Protocol : : Server : : Exception :
onReceiveExceptionFromServer ( std : : move ( packet . exception ) ) ;
return false ;
case Protocol : : Server : : Log :
onLogData ( packet . block ) ;
return true ;
case Protocol : : Server : : EndOfStream :
onEndOfStream ( ) ;
return false ;
2021-08-30 11:04:59 +00:00
case Protocol : : Server : : ProfileEvents :
2021-09-14 11:06:00 +00:00
onProfileEvents ( packet . block ) ;
2021-08-30 11:04:59 +00:00
return true ;
2023-03-07 15:05:23 +00:00
case Protocol : : Server : : TimezoneUpdate :
2023-03-15 17:37:23 +00:00
onTimezoneUpdate ( packet . server_timezone ) ;
2023-03-07 15:05:23 +00:00
return true ;
2021-08-17 19:59:51 +00:00
default :
throw Exception (
ErrorCodes : : UNKNOWN_PACKET_FROM_SERVER , " Unknown packet {} from server {} " , packet . type , connection - > getDescription ( ) ) ;
}
}
2021-08-03 08:33:54 +00:00
void ClientBase : : onProgress ( const Progress & value )
{
if ( ! progress_indication . updateProgress ( value ) )
{
// Just a keep-alive update.
return ;
}
2021-10-11 16:11:50 +00:00
if ( output_format )
output_format - > onProgress ( value ) ;
2021-08-03 08:33:54 +00:00
2022-10-23 02:46:07 +00:00
if ( need_render_progress & & tty_buf )
2022-10-01 21:19:36 +00:00
progress_indication . writeProgress ( * tty_buf ) ;
2021-08-03 08:33:54 +00:00
}
2023-03-15 17:37:23 +00:00
void ClientBase : : onTimezoneUpdate ( const String & tz )
{
2023-05-10 23:10:34 +00:00
global_context - > setSetting ( " session_timezone " , tz ) ;
2023-03-15 17:37:23 +00:00
}
2021-08-03 08:33:54 +00:00
void ClientBase : : onEndOfStream ( )
{
2022-10-23 02:46:07 +00:00
if ( need_render_progress & & tty_buf )
2022-10-01 21:19:36 +00:00
progress_indication . clearProgressOutput ( * tty_buf ) ;
2021-08-03 08:33:54 +00:00
2021-10-11 16:11:50 +00:00
if ( output_format )
2022-12-28 20:01:41 +00:00
{
/// Do our best to estimate the start of the query so the output format matches the one reported by the server
bool is_running = false ;
output_format - > setStartTime (
clock_gettime_ns ( CLOCK_MONOTONIC ) - static_cast < UInt64 > ( progress_indication . elapsedSeconds ( ) * 1000000000 ) , is_running ) ;
2023-09-01 16:08:11 +00:00
try
{
output_format - > finalize ( ) ;
}
catch ( . . . )
{
/// Format should be reset to make it work for subsequent query
/// (otherwise it will throw again in resetOutput())
output_format . reset ( ) ;
throw ;
}
2022-12-28 20:01:41 +00:00
}
2021-08-03 08:33:54 +00:00
resetOutput ( ) ;
2024-06-06 05:29:06 +00:00
if ( is_interactive )
{
2024-07-02 15:48:10 +00:00
if ( cancelled & & ! cancelled_printed . exchange ( true ) )
2024-06-25 14:23:37 +00:00
output_stream < < " Query was cancelled. " < < std : : endl ;
2024-06-06 05:29:06 +00:00
else if ( ! written_first_block )
2024-06-25 14:23:37 +00:00
output_stream < < " Ok. " < < std : : endl ;
2024-06-06 05:29:06 +00:00
}
2021-08-03 08:33:54 +00:00
}
2021-09-14 11:06:00 +00:00
void ClientBase : : onProfileEvents ( Block & block )
{
2021-10-11 13:55:08 +00:00
const auto rows = block . rows ( ) ;
2021-10-12 18:03:54 +00:00
if ( rows = = 0 )
2021-09-14 11:06:00 +00:00
return ;
2021-09-14 13:24:57 +00:00
2022-03-01 07:54:23 +00:00
if ( getName ( ) = = " local " | | server_revision > = DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS )
2021-09-14 11:06:00 +00:00
{
2021-10-12 18:03:54 +00:00
const auto & array_thread_id = typeid_cast < const ColumnUInt64 & > ( * block . getByName ( " thread_id " ) . column ) . getData ( ) ;
const auto & names = typeid_cast < const ColumnString & > ( * block . getByName ( " name " ) . column ) ;
const auto & host_names = typeid_cast < const ColumnString & > ( * block . getByName ( " host_name " ) . column ) ;
2021-11-08 13:38:31 +00:00
const auto & array_values = typeid_cast < const ColumnInt64 & > ( * block . getByName ( " value " ) . column ) . getData ( ) ;
2021-10-12 18:03:54 +00:00
const auto * user_time_name = ProfileEvents : : getName ( ProfileEvents : : UserTimeMicroseconds ) ;
const auto * system_time_name = ProfileEvents : : getName ( ProfileEvents : : SystemTimeMicroseconds ) ;
2023-03-14 11:36:26 +00:00
HostToTimesMap thread_times ;
2021-10-12 18:03:54 +00:00
for ( size_t i = 0 ; i < rows ; + + i )
2021-09-14 13:24:57 +00:00
{
2021-10-12 18:03:54 +00:00
auto thread_id = array_thread_id [ i ] ;
auto host_name = host_names . getDataAt ( i ) . toString ( ) ;
2023-03-14 11:36:26 +00:00
/// In ProfileEvents packets thread id 0 specifies common profiling information
/// for all threads executing current query on specific host. So instead of summing per thread
/// consumption it's enough to look for data with thread id 0.
if ( thread_id ! = THREAD_GROUP_ID )
continue ;
2021-10-12 18:03:54 +00:00
auto event_name = names . getDataAt ( i ) ;
auto value = array_values [ i ] ;
2022-06-14 21:36:16 +00:00
/// Ignore negative time delta or memory usage just in case.
if ( value < 0 )
continue ;
2021-10-12 18:03:54 +00:00
if ( event_name = = user_time_name )
2023-03-14 11:36:26 +00:00
thread_times [ host_name ] . user_ms = value ;
2021-10-12 18:03:54 +00:00
else if ( event_name = = system_time_name )
2023-03-14 11:36:26 +00:00
thread_times [ host_name ] . system_ms = value ;
2021-10-12 18:03:54 +00:00
else if ( event_name = = MemoryTracker : : USAGE_EVENT_NAME )
2023-03-14 11:36:26 +00:00
thread_times [ host_name ] . memory_usage = value ;
2023-06-28 09:31:44 +00:00
else if ( event_name = = MemoryTracker : : PEAK_USAGE_EVENT_NAME )
thread_times [ host_name ] . peak_memory_usage = value ;
2021-09-14 13:24:57 +00:00
}
2022-07-15 22:27:26 +00:00
progress_indication . updateThreadEventData ( thread_times ) ;
2021-10-12 18:03:54 +00:00
2022-10-23 02:46:07 +00:00
if ( need_render_progress & & tty_buf )
2022-10-01 21:19:36 +00:00
progress_indication . writeProgress ( * tty_buf ) ;
2022-04-13 15:58:15 +00:00
2021-12-29 16:57:40 +00:00
if ( profile_events . print )
2021-09-14 13:24:57 +00:00
{
2021-12-29 16:57:40 +00:00
if ( profile_events . watch . elapsedMilliseconds ( ) > = profile_events . delay_ms )
{
2023-03-27 11:50:09 +00:00
/// We need to restart the watch each time we flushed these events
profile_events . watch . restart ( ) ;
2021-12-29 16:57:40 +00:00
initLogsOutputStream ( ) ;
2022-10-23 02:46:07 +00:00
if ( need_render_progress & & tty_buf )
2022-10-01 21:19:36 +00:00
progress_indication . clearProgressOutput ( * tty_buf ) ;
2021-12-29 16:57:40 +00:00
logs_out_stream - > writeProfileEvents ( block ) ;
logs_out_stream - > flush ( ) ;
2021-10-12 18:03:54 +00:00
2021-12-29 16:57:40 +00:00
profile_events . last_block = { } ;
}
else
{
incrementProfileEventsBlock ( profile_events . last_block , block ) ;
}
2021-09-17 16:47:54 +00:00
}
2021-09-14 11:06:00 +00:00
}
}
2021-08-30 11:04:59 +00:00
2021-07-12 09:30:16 +00:00
/// Flush all buffers.
2021-07-22 21:27:26 +00:00
void ClientBase : : resetOutput ( )
2021-07-12 09:30:16 +00:00
{
2023-06-19 09:56:02 +00:00
/// Order is important: format, compression, file
2022-09-21 16:53:39 +00:00
if ( output_format )
output_format - > finalize ( ) ;
2021-10-11 16:11:50 +00:00
output_format . reset ( ) ;
2023-06-19 09:56:02 +00:00
2021-07-12 09:30:16 +00:00
logs_out_stream . reset ( ) ;
2023-06-19 09:56:02 +00:00
if ( out_file_buf )
{
out_file_buf - > finalize ( ) ;
out_file_buf . reset ( ) ;
}
2021-07-12 09:30:16 +00:00
if ( pager_cmd )
{
pager_cmd - > in . close ( ) ;
pager_cmd - > wait ( ) ;
2023-09-30 16:43:20 +00:00
if ( SIG_ERR = = signal ( SIGPIPE , SIG_DFL ) )
2023-12-15 18:25:49 +00:00
throw ErrnoException ( ErrorCodes : : CANNOT_SET_SIGNAL_HANDLER , " Cannot set signal handler for SIGPIPE " ) ;
2023-09-30 16:43:20 +00:00
if ( SIG_ERR = = signal ( SIGINT , SIG_DFL ) )
2023-12-15 18:25:49 +00:00
throw ErrnoException ( ErrorCodes : : CANNOT_SET_SIGNAL_HANDLER , " Cannot set signal handler for SIGINT " ) ;
2023-09-30 16:43:20 +00:00
if ( SIG_ERR = = signal ( SIGQUIT , SIG_DFL ) )
2023-12-15 18:25:49 +00:00
throw ErrnoException ( ErrorCodes : : CANNOT_SET_SIGNAL_HANDLER , " Cannot set signal handler for SIGQUIT " ) ;
2023-09-30 16:43:20 +00:00
setupSignalHandler ( ) ;
2021-07-12 09:30:16 +00:00
}
pager_cmd = nullptr ;
if ( out_logs_buf )
{
2023-06-14 13:33:01 +00:00
out_logs_buf - > finalize ( ) ;
2021-07-12 09:30:16 +00:00
out_logs_buf . reset ( ) ;
}
std_out . next ( ) ;
}
2021-08-21 10:55:54 +00:00
2021-08-18 14:39:04 +00:00
/// Receive the block that serves as an example of the structure of table where data will be inserted.
bool ClientBase : : receiveSampleBlock ( Block & out , ColumnsDescription & columns_description , ASTPtr parsed_query )
{
while ( true )
{
Packet packet = connection - > receivePacket ( ) ;
switch ( packet . type )
{
case Protocol : : Server : : Data :
out = packet . block ;
return true ;
case Protocol : : Server : : Exception :
onReceiveExceptionFromServer ( std : : move ( packet . exception ) ) ;
return false ;
case Protocol : : Server : : Log :
onLogData ( packet . block ) ;
break ;
case Protocol : : Server : : TableColumns :
columns_description = ColumnsDescription : : parse ( packet . multistring_message [ 1 ] ) ;
return receiveSampleBlock ( out , columns_description , parsed_query ) ;
2023-03-07 15:05:23 +00:00
case Protocol : : Server : : TimezoneUpdate :
2023-03-15 17:37:23 +00:00
onTimezoneUpdate ( packet . server_timezone ) ;
2023-03-07 15:05:23 +00:00
break ;
2021-08-18 14:39:04 +00:00
default :
2023-01-23 21:13:58 +00:00
throw NetException ( ErrorCodes : : UNEXPECTED_PACKET_FROM_SERVER ,
2023-03-15 17:37:23 +00:00
" Unexpected packet from server (expected Data, Exception, Log or TimezoneUpdate, got {}) " ,
2023-01-23 21:13:58 +00:00
String ( Protocol : : Server : : toString ( packet . type ) ) ) ;
2021-08-18 14:39:04 +00:00
}
}
}
2022-09-08 16:37:18 +00:00
void ClientBase : : setInsertionTable ( const ASTInsertQuery & insert_query )
{
if ( ! global_context - > hasInsertionTable ( ) & & insert_query . table )
{
String table = insert_query . table - > as < ASTIdentifier & > ( ) . shortName ( ) ;
if ( ! table . empty ( ) )
{
String database = insert_query . database ? insert_query . database - > as < ASTIdentifier & > ( ) . shortName ( ) : " " ;
global_context - > setInsertionTable ( StorageID ( database , table ) ) ;
}
}
}
2023-05-17 03:43:35 +00:00
void ClientBase : : addMultiquery ( std : : string_view query , Arguments & common_arguments ) const
{
common_arguments . emplace_back ( " --multiquery " ) ;
common_arguments . emplace_back ( " -q " ) ;
common_arguments . emplace_back ( query ) ;
}
2023-11-02 18:05:02 +00:00
namespace
{
bool isStdinNotEmptyAndValid ( ReadBufferFromFileDescriptor & std_in )
{
try
{
return ! std_in . eof ( ) ;
}
catch ( const Exception & e )
{
if ( e . code ( ) = = ErrorCodes : : CANNOT_READ_FROM_FILE_DESCRIPTOR )
return false ;
throw ;
}
}
}
2023-05-17 03:43:35 +00:00
2021-08-18 14:39:04 +00:00
void ClientBase : : processInsertQuery ( const String & query_to_execute , ASTPtr parsed_query )
{
2021-10-12 23:51:11 +00:00
auto query = query_to_execute ;
2022-08-12 12:28:35 +00:00
if ( ! query_parameters . empty ( )
& & connection - > getServerRevision ( connection_parameters . timeouts ) < DBMS_MIN_PROTOCOL_VERSION_WITH_PARAMETERS )
2021-10-12 23:51:11 +00:00
{
/// Replace ASTQueryParameter with ASTLiteral for prepared statements.
ReplaceQueryParameterVisitor visitor ( query_parameters ) ;
visitor . visit ( parsed_query ) ;
/// Get new query after substitutions.
2023-08-16 20:35:45 +00:00
if ( visitor . getNumberOfReplacedParameters ( ) )
query = serializeAST ( * parsed_query ) ;
chassert ( ! query . empty ( ) ) ;
2021-10-12 23:51:11 +00:00
}
2021-08-18 14:39:04 +00:00
/// Process the query that requires transferring data blocks to the server.
2023-10-20 16:39:48 +00:00
const auto & parsed_insert_query = parsed_query - > as < ASTInsertQuery & > ( ) ;
2023-11-02 18:05:02 +00:00
if ( ( ! parsed_insert_query . data & & ! parsed_insert_query . infile ) & & ( is_interactive | | ( ! stdin_is_a_tty & & ! isStdinNotEmptyAndValid ( std_in ) ) ) )
2022-04-17 05:42:07 +00:00
{
const auto & settings = global_context - > getSettingsRef ( ) ;
if ( settings . throw_if_no_data_to_insert )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NO_DATA_TO_INSERT , " No data to insert " ) ;
2022-04-17 05:42:07 +00:00
else
return ;
}
2021-08-18 14:39:04 +00:00
2022-04-27 13:53:35 +00:00
QueryInterruptHandler : : start ( ) ;
SCOPE_EXIT ( { QueryInterruptHandler : : stop ( ) ; } ) ;
2021-08-18 14:39:04 +00:00
connection - > sendQuery (
connection_parameters . timeouts ,
2021-10-12 23:51:11 +00:00
query ,
2022-08-12 12:28:35 +00:00
query_parameters ,
2021-08-18 14:39:04 +00:00
global_context - > getCurrentQueryId ( ) ,
query_processing_stage ,
& global_context - > getSettingsRef ( ) ,
& global_context - > getClientInfo ( ) ,
2022-05-06 15:04:03 +00:00
true ,
[ & ] ( const Progress & progress ) { onProgress ( progress ) ; } ) ;
2021-08-18 14:39:04 +00:00
2021-09-24 08:29:01 +00:00
if ( send_external_tables )
sendExternalTables ( parsed_query ) ;
2021-08-18 14:39:04 +00:00
/// Receive description of table structure.
Block sample ;
ColumnsDescription columns_description ;
if ( receiveSampleBlock ( sample , columns_description , parsed_query ) )
{
/// If structure was received (thus, server has not thrown an exception),
/// send our data with that structure.
2022-09-08 16:37:18 +00:00
setInsertionTable ( parsed_insert_query ) ;
2022-08-23 04:17:36 +00:00
2021-08-18 14:39:04 +00:00
sendData ( sample , columns_description , parsed_query ) ;
receiveEndOfQuery ( ) ;
2022-08-23 04:17:36 +00:00
}
}
2021-08-18 14:39:04 +00:00
void ClientBase : : sendData ( Block & sample , const ColumnsDescription & columns_description , ASTPtr parsed_query )
{
2021-10-26 15:16:58 +00:00
/// Get columns description from variable or (if it was empty) create it from sample.
auto columns_description_for_query = columns_description . empty ( ) ? ColumnsDescription ( sample . getNamesAndTypesList ( ) ) : columns_description ;
if ( columns_description_for_query . empty ( ) )
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR ,
" Column description is empty and it can't be built from sample from table. "
" Cannot execute query. " ) ;
2021-10-26 15:16:58 +00:00
}
2021-08-18 14:39:04 +00:00
/// If INSERT data must be sent.
auto * parsed_insert_query = parsed_query - > as < ASTInsertQuery > ( ) ;
if ( ! parsed_insert_query )
return ;
2021-08-19 11:07:47 +00:00
2023-10-24 13:44:00 +00:00
bool have_data_in_stdin = ! is_interactive & & ! stdin_is_a_tty & & isStdinNotEmptyAndValid ( std_in ) ;
2022-05-06 15:04:03 +00:00
2022-10-25 01:43:29 +00:00
if ( need_render_progress )
2021-08-23 07:13:27 +00:00
{
/// Set total_bytes_to_read for current fd.
2022-05-25 14:49:40 +00:00
FileProgress file_progress ( 0 , std_in . getFileSize ( ) ) ;
2021-08-23 07:13:27 +00:00
progress_indication . updateProgress ( Progress ( file_progress ) ) ;
/// Set callback to be called on file progress.
2022-10-23 02:46:07 +00:00
if ( tty_buf )
progress_indication . setFileProgressCallback ( global_context , * tty_buf ) ;
2021-08-23 07:13:27 +00:00
}
2021-09-16 18:44:42 +00:00
/// If data fetched from file (maybe compressed file)
2021-08-18 21:01:17 +00:00
if ( parsed_insert_query - > infile )
{
2021-09-16 18:44:42 +00:00
/// Get name of this file (path to file)
2021-08-18 21:01:17 +00:00
const auto & in_file_node = parsed_insert_query - > infile - > as < ASTLiteral & > ( ) ;
const auto in_file = in_file_node . value . safeGet < std : : string > ( ) ;
2021-11-15 19:55:27 +00:00
2021-09-16 18:44:42 +00:00
std : : string compression_method ;
/// Compression method can be specified in query
if ( parsed_insert_query - > compression )
{
const auto & compression_method_node = parsed_insert_query - > compression - > as < ASTLiteral & > ( ) ;
compression_method = compression_method_node . value . safeGet < std : : string > ( ) ;
}
2022-01-07 05:16:41 +00:00
String current_format = parsed_insert_query - > format ;
if ( current_format . empty ( ) )
2024-01-22 22:55:50 +00:00
current_format = FormatFactory : : instance ( ) . getFormatFromFileName ( in_file ) ;
2022-01-07 05:16:41 +00:00
2021-10-13 15:43:52 +00:00
/// Create temporary storage file, to support globs and parallel reading
2023-09-20 16:25:47 +00:00
/// StorageFile doesn't support ephemeral/materialized/alias columns.
/// We should change ephemeral columns to ordinary and ignore materialized/alias columns.
ColumnsDescription columns_for_storage_file ;
for ( const auto & [ name , _ ] : columns_description_for_query . getInsertable ( ) )
{
ColumnDescription column = columns_description_for_query . get ( name ) ;
column . default_desc . kind = ColumnDefaultKind : : Default ;
columns_for_storage_file . add ( std : : move ( column ) ) ;
}
2023-09-19 18:14:42 +00:00
2021-10-13 15:43:52 +00:00
StorageFile : : CommonArguments args {
WithContext ( global_context ) ,
2021-10-26 15:16:58 +00:00
parsed_insert_query - > table_id ,
2022-01-07 05:16:41 +00:00
current_format ,
2021-10-26 15:16:58 +00:00
getFormatSettings ( global_context ) ,
2021-10-13 15:43:52 +00:00
compression_method ,
2023-09-19 18:14:42 +00:00
columns_for_storage_file ,
2021-10-13 15:43:52 +00:00
ConstraintsDescription { } ,
String { } ,
2023-05-07 12:18:52 +00:00
{ } ,
2023-07-31 12:04:27 +00:00
String { } ,
2021-10-13 15:43:52 +00:00
} ;
2022-04-19 20:47:29 +00:00
StoragePtr storage = std : : make_shared < StorageFile > ( in_file , global_context - > getUserFilesPath ( ) , args ) ;
2021-10-13 15:43:52 +00:00
storage - > startup ( ) ;
SelectQueryInfo query_info ;
2021-08-18 14:39:04 +00:00
2021-08-18 21:01:17 +00:00
try
{
2021-11-09 12:36:25 +00:00
auto metadata = storage - > getInMemoryMetadataPtr ( ) ;
2022-05-23 19:47:32 +00:00
QueryPlan plan ;
storage - > read (
plan ,
sample . getNames ( ) ,
storage - > getStorageSnapshot ( metadata , global_context ) ,
query_info ,
global_context ,
{ } ,
global_context - > getSettingsRef ( ) . max_block_size ,
getNumberOfPhysicalCPUCores ( ) ) ;
auto builder = plan . buildQueryPipeline (
QueryPlanOptimizationSettings : : fromContext ( global_context ) ,
BuildQueryPipelineSettings : : fromContext ( global_context ) ) ;
QueryPlanResourceHolder resources ;
auto pipe = QueryPipelineBuilder : : getPipe ( std : : move ( * builder ) , resources ) ;
2021-10-13 15:43:52 +00:00
sendDataFromPipe (
2022-05-23 19:47:32 +00:00
std : : move ( pipe ) ,
2022-04-14 11:30:52 +00:00
parsed_query ,
2023-05-07 04:16:30 +00:00
have_data_in_stdin ) ;
2021-08-18 21:01:17 +00:00
}
catch ( Exception & e )
{
e . addMessage ( " data for INSERT was parsed from file " ) ;
throw ;
}
2022-04-14 11:30:52 +00:00
2023-03-02 18:51:05 +00:00
if ( have_data_in_stdin & & ! cancelled )
2022-04-14 11:30:52 +00:00
sendDataFromStdin ( sample , columns_description_for_query , parsed_query ) ;
2021-08-18 21:01:17 +00:00
}
else if ( parsed_insert_query - > data )
2021-08-18 14:39:04 +00:00
{
/// Send data contained in the query.
ReadBufferFromMemory data_in ( parsed_insert_query - > data , parsed_insert_query - > end - parsed_insert_query - > data ) ;
try
{
2022-04-14 11:30:52 +00:00
sendDataFrom ( data_in , sample , columns_description_for_query , parsed_query , have_data_in_stdin ) ;
2023-03-02 18:51:05 +00:00
if ( have_data_in_stdin & & ! cancelled )
2022-04-14 11:30:52 +00:00
sendDataFromStdin ( sample , columns_description_for_query , parsed_query ) ;
2021-08-18 14:39:04 +00:00
}
catch ( Exception & e )
{
/// The following query will use data from input
// "INSERT INTO data FORMAT TSV\n " < data.csv
// And may be pretty hard to debug, so add information about data source to make it easier.
e . addMessage ( " data for INSERT was parsed from query " ) ;
throw ;
}
// Remember where the data ended. We use this info later to determine
// where the next query begins.
parsed_insert_query - > end = parsed_insert_query - > data + data_in . count ( ) ;
}
else if ( ! is_interactive )
{
2022-04-14 11:30:52 +00:00
sendDataFromStdin ( sample , columns_description_for_query , parsed_query ) ;
2021-08-18 14:39:04 +00:00
}
else
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NO_DATA_TO_INSERT , " No data to insert " ) ;
2021-08-18 14:39:04 +00:00
}
2022-04-14 11:30:52 +00:00
void ClientBase : : sendDataFrom ( ReadBuffer & buf , Block & sample , const ColumnsDescription & columns_description , ASTPtr parsed_query , bool have_more_data )
2021-08-18 14:39:04 +00:00
{
2024-03-23 01:42:22 +00:00
String current_format = " Values " ;
2021-08-18 14:39:04 +00:00
/// Data format can be specified in the INSERT query.
if ( const auto * insert = parsed_query - > as < ASTInsertQuery > ( ) )
{
if ( ! insert - > format . empty ( ) )
current_format = insert - > format ;
}
2021-10-11 16:11:50 +00:00
auto source = global_context - > getInputFormat ( current_format , buf , sample , insert_format_max_block_size ) ;
2021-08-18 14:39:04 +00:00
Pipe pipe ( source ) ;
if ( columns_description . hasDefaults ( ) )
{
pipe . addSimpleTransform ( [ & ] ( const Block & header )
{
return std : : make_shared < AddingDefaultsTransform > ( header , columns_description , * source , global_context ) ;
} ) ;
}
2022-04-14 11:30:52 +00:00
sendDataFromPipe ( std : : move ( pipe ) , parsed_query , have_more_data ) ;
2021-10-13 15:43:52 +00:00
}
2022-04-19 12:21:14 +00:00
void ClientBase : : sendDataFromPipe ( Pipe & & pipe , ASTPtr parsed_query , bool have_more_data )
2022-04-25 10:40:36 +00:00
try
2021-10-13 15:43:52 +00:00
{
2021-09-29 19:03:30 +00:00
QueryPipeline pipeline ( std : : move ( pipe ) ) ;
2021-08-18 14:39:04 +00:00
PullingAsyncPipelineExecutor executor ( pipeline ) ;
2022-11-21 12:58:48 +00:00
if ( need_render_progress )
{
2022-11-21 14:04:37 +00:00
pipeline . setProgressCallback ( [ this ] ( const Progress & progress ) { onProgress ( progress ) ; } ) ;
2022-11-21 12:58:48 +00:00
}
2021-08-18 14:39:04 +00:00
Block block ;
2022-09-05 15:42:49 +00:00
while ( executor . pull ( block ) )
2021-08-18 14:39:04 +00:00
{
2022-09-05 15:42:49 +00:00
if ( ! cancelled & & QueryInterruptHandler : : cancelled ( ) )
2022-04-27 13:53:35 +00:00
{
2022-09-05 15:42:49 +00:00
cancelQuery ( ) ;
executor . cancel ( ) ;
return ;
}
2022-04-27 13:53:35 +00:00
2022-09-05 15:42:49 +00:00
/// Check if server send Log packet
receiveLogsAndProfileEvents ( parsed_query ) ;
2021-08-18 14:39:04 +00:00
2022-09-05 15:42:49 +00:00
/// Check if server send Exception packet
auto packet_type = connection - > checkPacket ( 0 ) ;
if ( packet_type & & * packet_type = = Protocol : : Server : : Exception )
{
/**
2021-09-19 18:24:06 +00:00
* We ' re exiting with error , so it makes sense to kill the
* input stream without waiting for it to complete .
*/
2022-09-05 15:42:49 +00:00
executor . cancel ( ) ;
return ;
2021-08-18 14:39:04 +00:00
}
2022-09-05 15:42:49 +00:00
if ( block )
{
connection - > sendData ( block , /* name */ " " , /* scalar */ false ) ;
processed_rows + = block . rows ( ) ;
}
2022-08-23 04:17:36 +00:00
}
2022-09-05 15:42:49 +00:00
if ( ! have_more_data )
connection - > sendData ( { } , " " , false ) ;
2022-04-14 11:30:52 +00:00
}
2022-04-25 10:40:36 +00:00
catch ( . . . )
{
connection - > sendCancel ( ) ;
receiveEndOfQuery ( ) ;
throw ;
}
2022-04-14 11:30:52 +00:00
void ClientBase : : sendDataFromStdin ( Block & sample , const ColumnsDescription & columns_description , ASTPtr parsed_query )
{
/// Send data read from stdin.
try
{
sendDataFrom ( std_in , sample , columns_description , parsed_query ) ;
}
catch ( Exception & e )
{
e . addMessage ( " data for INSERT was parsed from stdin " ) ;
throw ;
}
2021-08-18 14:39:04 +00:00
}
/// Process Log packets, used when inserting data by blocks
2022-05-20 15:19:40 +00:00
void ClientBase : : receiveLogsAndProfileEvents ( ASTPtr parsed_query )
2021-08-18 14:39:04 +00:00
{
auto packet_type = connection - > checkPacket ( 0 ) ;
2023-04-19 14:39:52 +00:00
while ( packet_type & & ( * packet_type = = Protocol : : Server : : Log
| | * packet_type = = Protocol : : Server : : ProfileEvents
| | * packet_type = = Protocol : : Server : : TimezoneUpdate ) )
2021-08-18 14:39:04 +00:00
{
receiveAndProcessPacket ( parsed_query , false ) ;
2021-09-22 21:35:29 +00:00
packet_type = connection - > checkPacket ( 0 ) ;
2021-08-18 14:39:04 +00:00
}
}
/// Process Log packets, exit when receive Exception or EndOfStream
bool ClientBase : : receiveEndOfQuery ( )
{
while ( true )
{
Packet packet = connection - > receivePacket ( ) ;
switch ( packet . type )
{
case Protocol : : Server : : EndOfStream :
onEndOfStream ( ) ;
return true ;
case Protocol : : Server : : Exception :
onReceiveExceptionFromServer ( std : : move ( packet . exception ) ) ;
return false ;
case Protocol : : Server : : Log :
onLogData ( packet . block ) ;
break ;
2021-09-11 17:16:37 +00:00
case Protocol : : Server : : Progress :
onProgress ( packet . progress ) ;
2022-01-19 16:04:57 +00:00
break ;
2021-09-11 17:16:37 +00:00
2022-04-21 09:11:57 +00:00
case Protocol : : Server : : ProfileEvents :
onProfileEvents ( packet . block ) ;
break ;
2023-03-07 15:05:23 +00:00
case Protocol : : Server : : TimezoneUpdate :
2023-03-15 17:37:23 +00:00
onTimezoneUpdate ( packet . server_timezone ) ;
2023-03-07 15:05:23 +00:00
break ;
2021-08-18 14:39:04 +00:00
default :
2023-01-23 21:13:58 +00:00
throw NetException ( ErrorCodes : : UNEXPECTED_PACKET_FROM_SERVER ,
" Unexpected packet from server (expected Exception, EndOfStream, Log, Progress or ProfileEvents. Got {}) " ,
String ( Protocol : : Server : : toString ( packet . type ) ) ) ;
2021-08-18 14:39:04 +00:00
}
}
}
2022-04-27 13:53:35 +00:00
void ClientBase : : cancelQuery ( )
{
connection - > sendCancel ( ) ;
2022-10-23 02:46:07 +00:00
if ( need_render_progress & & tty_buf )
2022-10-01 21:19:36 +00:00
progress_indication . clearProgressOutput ( * tty_buf ) ;
2022-04-27 13:53:35 +00:00
if ( is_interactive )
2024-06-25 14:23:37 +00:00
output_stream < < " Cancelling query. " < < std : : endl ;
2022-04-27 13:53:35 +00:00
cancelled = true ;
}
2021-08-18 14:39:04 +00:00
2021-08-19 11:07:47 +00:00
void ClientBase : : processParsedSingleQuery ( const String & full_query , const String & query_to_execute ,
2021-08-18 21:01:17 +00:00
ASTPtr parsed_query , std : : optional < bool > echo_query_ , bool report_error )
2021-07-12 09:30:16 +00:00
{
2021-07-28 12:56:11 +00:00
resetOutput ( ) ;
have_error = false ;
2022-02-26 20:40:24 +00:00
cancelled = false ;
2024-06-06 05:29:06 +00:00
cancelled_printed = false ;
2021-09-09 13:25:25 +00:00
client_exception . reset ( ) ;
server_exception . reset ( ) ;
2021-07-28 12:56:11 +00:00
if ( echo_query_ & & * echo_query_ )
2021-07-12 09:30:16 +00:00
{
writeString ( full_query , std_out ) ;
writeChar ( ' \n ' , std_out ) ;
std_out . next ( ) ;
}
2021-08-21 10:55:54 +00:00
if ( is_interactive )
{
global_context - > setCurrentQueryId ( " " ) ;
// Generate a new query_id
for ( const auto & query_id_format : query_id_formats )
{
writeString ( query_id_format . first , std_out ) ;
Use fmt::runtime() for LOG_* for non constexpr
Here is oneliner:
$ gg 'LOG_\(DEBUG\|TRACE\|INFO\|TEST\|WARNING\|ERROR\|FATAL\)([^,]*, [a-zA-Z]' -- :*.cpp :*.h | cut -d: -f1 | sort -u | xargs -r sed -E -i 's#(LOG_[A-Z]*)\(([^,]*), ([A-Za-z][^,)]*)#\1(\2, fmt::runtime(\3)#'
Note, that I tried to do this with coccinelle (tool for semantic
patchin), but it cannot parse C++:
$ cat fmt.cocci
@@
expression log;
expression var;
@@
-LOG_DEBUG(log, var)
+LOG_DEBUG(log, fmt::runtime(var))
I've also tried to use some macros/templates magic to do this implicitly
in logger_useful.h, but I failed to do so, and apparently it is not
possible for now.
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
v2: manual fixes
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-02-01 09:10:27 +00:00
writeString ( fmt : : format ( fmt : : runtime ( query_id_format . second ) , fmt : : arg ( " query_id " , global_context - > getCurrentQueryId ( ) ) ) , std_out ) ;
2021-08-21 10:55:54 +00:00
writeChar ( ' \n ' , std_out ) ;
std_out . next ( ) ;
}
}
2021-07-12 09:30:16 +00:00
2022-04-05 12:46:18 +00:00
if ( const auto * set_query = parsed_query - > as < ASTSetQuery > ( ) )
{
const auto * logs_level_field = set_query - > changes . tryGet ( std : : string_view { " send_logs_level " } ) ;
if ( logs_level_field )
2023-10-24 20:46:44 +00:00
{
auto logs_level = logs_level_field - > safeGet < String > ( ) ;
/// Check that setting value is correct before updating logger level.
SettingFieldLogsLevelTraits : : fromString ( logs_level ) ;
updateLoggerLevel ( logs_level ) ;
}
2022-04-05 12:46:18 +00:00
}
2022-11-22 02:41:23 +00:00
if ( const auto * create_user_query = parsed_query - > as < ASTCreateUserQuery > ( ) )
{
2023-04-10 02:20:57 +00:00
if ( ! create_user_query - > attach & & create_user_query - > auth_data )
2022-11-22 02:41:23 +00:00
{
2023-04-10 02:20:57 +00:00
if ( const auto * auth_data = create_user_query - > auth_data - > as < ASTAuthenticationData > ( ) )
2023-04-10 16:40:49 +00:00
{
auto password = auth_data - > getPassword ( ) ;
if ( password )
global_context - > getAccessControl ( ) . checkPasswordComplexityRules ( * password ) ;
}
2022-11-22 02:41:23 +00:00
}
}
2021-07-12 09:30:16 +00:00
processed_rows = 0 ;
written_first_block = false ;
progress_indication . resetProgress ( ) ;
2021-10-12 18:03:54 +00:00
profile_events . watch . restart ( ) ;
2021-07-12 09:30:16 +00:00
2021-09-19 21:42:28 +00:00
{
/// Temporarily apply query settings to context.
std : : optional < Settings > old_settings ;
SCOPE_EXIT_SAFE ( {
if ( old_settings )
global_context - > setSettings ( * old_settings ) ;
} ) ;
auto apply_query_settings = [ & ] ( const IAST & settings_ast )
{
if ( ! old_settings )
old_settings . emplace ( global_context - > getSettingsRef ( ) ) ;
global_context - > applySettingsChanges ( settings_ast . as < ASTSetQuery > ( ) - > changes ) ;
2022-10-11 18:25:28 +00:00
global_context - > resetSettingsToDefaultValue ( settings_ast . as < ASTSetQuery > ( ) - > default_settings ) ;
2021-09-19 21:42:28 +00:00
} ;
const auto * insert = parsed_query - > as < ASTInsertQuery > ( ) ;
2023-02-03 09:26:00 +00:00
if ( const auto * select = parsed_query - > as < ASTSelectQuery > ( ) ; select & & select - > settings ( ) )
apply_query_settings ( * select - > settings ( ) ) ;
else if ( const auto * select_with_union = parsed_query - > as < ASTSelectWithUnionQuery > ( ) )
{
const ASTs & children = select_with_union - > list_of_selects - > children ;
if ( ! children . empty ( ) )
{
// On the client it is enough to apply settings only for the
// last SELECT, since the only thing that is important to apply
// on the client is format settings.
const auto * last_select = children . back ( ) - > as < ASTSelectQuery > ( ) ;
if ( last_select & & last_select - > settings ( ) )
{
apply_query_settings ( * last_select - > settings ( ) ) ;
}
}
}
else if ( const auto * query_with_output = parsed_query - > as < ASTQueryWithOutput > ( ) ; query_with_output & & query_with_output - > settings_ast )
apply_query_settings ( * query_with_output - > settings_ast ) ;
else if ( insert & & insert - > settings_ast )
2021-09-19 21:42:28 +00:00
apply_query_settings ( * insert - > settings_ast ) ;
2022-08-12 13:30:46 +00:00
if ( ! connection - > checkConnected ( connection_parameters . timeouts ) )
2021-09-19 21:42:28 +00:00
connect ( ) ;
ASTPtr input_function ;
if ( insert & & insert - > select )
insert - > tryFindInputFunction ( input_function ) ;
2023-10-20 16:39:48 +00:00
bool is_async_insert_with_inlined_data = global_context - > getSettingsRef ( ) . async_insert & & insert & & insert - > hasInlinedData ( ) ;
if ( is_async_insert_with_inlined_data )
{
2023-10-24 13:44:00 +00:00
bool have_data_in_stdin = ! is_interactive & & ! stdin_is_a_tty & & isStdinNotEmptyAndValid ( std_in ) ;
2023-10-20 16:39:48 +00:00
bool have_external_data = have_data_in_stdin | | insert - > infile ;
if ( have_external_data )
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED ,
" Processing async inserts with both inlined and external data (from stdin or infile) is not supported " ) ;
}
2022-02-02 17:53:04 +00:00
2021-09-19 21:42:28 +00:00
/// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately.
2024-04-08 03:20:09 +00:00
if ( insert & & ( ! insert - > select | | input_function ) & & ! is_async_insert_with_inlined_data )
2021-09-19 21:42:28 +00:00
{
if ( input_function & & insert - > format . empty ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_USAGE_OF_INPUT , " FORMAT must be specified for function input() " ) ;
2021-09-19 21:42:28 +00:00
processInsertQuery ( query_to_execute , parsed_query ) ;
}
else
processOrdinaryQuery ( query_to_execute , parsed_query ) ;
}
/// Do not change context (current DB, settings) in case of an exception.
if ( ! have_error )
{
if ( const auto * set_query = parsed_query - > as < ASTSetQuery > ( ) )
{
/// Save all changes in settings to avoid losing them if the connection is lost.
for ( const auto & change : set_query - > changes )
{
if ( change . name = = " profile " )
current_profile = change . value . safeGet < String > ( ) ;
else
global_context - > applySettingChange ( change ) ;
}
2022-10-11 18:25:28 +00:00
global_context - > resetSettingsToDefaultValue ( set_query - > default_settings ) ;
2022-11-09 04:28:02 +00:00
/// Query parameters inside SET queries should be also saved on the client side
/// to override their previous definitions set with --param_* arguments
/// and for substitutions to work inside INSERT ... VALUES queries
for ( const auto & [ name , value ] : set_query - > query_parameters )
query_parameters . insert_or_assign ( name , value ) ;
2024-02-26 01:11:51 +00:00
global_context - > addQueryParameters ( NameToNameMap { set_query - > query_parameters . begin ( ) , set_query - > query_parameters . end ( ) } ) ;
2021-09-19 21:42:28 +00:00
}
if ( const auto * use_query = parsed_query - > as < ASTUseQuery > ( ) )
{
2023-08-15 18:37:39 +00:00
const String & new_database = use_query - > getDatabase ( ) ;
2021-09-19 21:42:28 +00:00
/// If the client initiates the reconnection, it takes the settings from the config.
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " database " , new_database ) ;
2021-09-19 21:42:28 +00:00
/// If the connection initiates the reconnection, it uses its variable.
connection - > setDefaultDatabase ( new_database ) ;
}
}
2021-07-12 09:30:16 +00:00
2021-10-12 18:03:54 +00:00
/// Always print last block (if it was not printed already)
if ( profile_events . last_block )
{
initLogsOutputStream ( ) ;
2022-10-23 02:46:07 +00:00
if ( need_render_progress & & tty_buf )
2022-10-01 21:19:36 +00:00
progress_indication . clearProgressOutput ( * tty_buf ) ;
2021-10-12 18:03:54 +00:00
logs_out_stream - > writeProfileEvents ( profile_events . last_block ) ;
logs_out_stream - > flush ( ) ;
2022-04-11 14:42:45 +00:00
profile_events . last_block = { } ;
2021-10-12 18:03:54 +00:00
}
2021-07-12 09:30:16 +00:00
if ( is_interactive )
{
2024-06-25 14:23:37 +00:00
output_stream < < std : : endl ;
2023-10-13 07:39:57 +00:00
if ( ! server_exception | | processed_rows ! = 0 )
2024-06-25 14:23:37 +00:00
output_stream < < processed_rows < < " row " < < ( processed_rows = = 1 ? " " : " s " ) < < " in set. " ;
output_stream < < " Elapsed: " < < progress_indication . elapsedSeconds ( ) < < " sec. " ;
2021-07-12 09:30:16 +00:00
progress_indication . writeFinalProgress ( ) ;
2024-06-25 14:23:37 +00:00
output_stream < < std : : endl < < std : : endl ;
2021-07-12 09:30:16 +00:00
}
2024-06-25 14:23:37 +00:00
else if ( getClientConfiguration ( ) . getBool ( " print-time-to-stderr " , false ) )
2021-07-12 09:30:16 +00:00
{
2024-06-25 14:23:37 +00:00
error_stream < < progress_indication . elapsedSeconds ( ) < < " \n " ;
2021-07-12 09:30:16 +00:00
}
2021-07-12 12:25:17 +00:00
2024-06-25 14:23:37 +00:00
if ( ! is_interactive & & getClientConfiguration ( ) . getBool ( " print-num-processed-rows " , false ) )
2022-11-18 21:42:44 +00:00
{
2024-06-25 14:23:37 +00:00
output_stream < < " Processed rows: " < < processed_rows < < " \n " ;
2022-11-18 21:42:44 +00:00
}
2021-07-12 14:01:46 +00:00
if ( have_error & & report_error )
2021-08-19 11:07:47 +00:00
processError ( full_query ) ;
2021-07-12 09:30:16 +00:00
}
2021-08-21 10:55:54 +00:00
MultiQueryProcessingStage ClientBase : : analyzeMultiQueryText (
2021-08-20 14:10:59 +00:00
const char * & this_query_begin , const char * & this_query_end , const char * all_queries_end ,
String & query_to_execute , ASTPtr & parsed_query , const String & all_queries_text ,
2022-04-15 23:56:45 +00:00
std : : unique_ptr < Exception > & current_exception )
2021-07-12 09:30:16 +00:00
{
2022-02-26 20:40:24 +00:00
if ( ! is_interactive & & cancelled )
return MultiQueryProcessingStage : : QUERIES_END ;
2021-08-20 14:10:59 +00:00
if ( this_query_begin > = all_queries_end )
return MultiQueryProcessingStage : : QUERIES_END ;
// Remove leading empty newlines and other whitespace, because they
2024-04-10 19:39:02 +00:00
// are annoying to filter in the query log. This is mostly relevant for
2021-08-20 14:10:59 +00:00
// the tests.
while ( this_query_begin < all_queries_end & & isWhitespaceASCII ( * this_query_begin ) )
+ + this_query_begin ;
if ( this_query_begin > = all_queries_end )
return MultiQueryProcessingStage : : QUERIES_END ;
2022-10-07 10:46:45 +00:00
unsigned max_parser_depth = static_cast < unsigned > ( global_context - > getSettingsRef ( ) . max_parser_depth ) ;
2024-03-17 18:53:58 +00:00
unsigned max_parser_backtracks = static_cast < unsigned > ( global_context - > getSettingsRef ( ) . max_parser_backtracks ) ;
2022-10-07 10:46:45 +00:00
2021-08-20 14:10:59 +00:00
// If there are only comments left until the end of file, we just
// stop. The parser can't handle this situation because it always
// expects that there is some query that it can parse.
// We can get into this situation because the parser also doesn't
// skip the trailing comments after parsing a query. This is because
// they may as well be the leading comments for the next query,
// and it makes more sense to treat them as such.
2021-07-12 09:30:16 +00:00
{
2021-08-20 14:10:59 +00:00
Tokens tokens ( this_query_begin , all_queries_end ) ;
2024-03-17 18:53:58 +00:00
IParser : : Pos token_iterator ( tokens , max_parser_depth , max_parser_backtracks ) ;
2021-08-20 14:10:59 +00:00
if ( ! token_iterator . isValid ( ) )
return MultiQueryProcessingStage : : QUERIES_END ;
}
2021-07-12 09:30:16 +00:00
2021-08-20 14:10:59 +00:00
this_query_end = this_query_begin ;
try
{
2024-03-26 09:53:08 +00:00
parsed_query = parseQuery ( this_query_end , all_queries_end ,
global_context - > getSettingsRef ( ) ,
2024-06-25 14:23:37 +00:00
/*allow_multi_statements=*/ true ) ;
2021-08-20 14:10:59 +00:00
}
2024-04-10 19:39:02 +00:00
catch ( const Exception & e )
2021-08-20 14:10:59 +00:00
{
2022-04-15 23:56:45 +00:00
current_exception . reset ( e . clone ( ) ) ;
2021-08-20 14:10:59 +00:00
return MultiQueryProcessingStage : : PARSING_EXCEPTION ;
}
2021-07-12 09:30:16 +00:00
2021-08-20 14:10:59 +00:00
if ( ! parsed_query )
{
if ( ignore_error )
2021-07-12 09:30:16 +00:00
{
Tokens tokens ( this_query_begin , all_queries_end ) ;
2024-03-17 18:53:58 +00:00
IParser : : Pos token_iterator ( tokens , max_parser_depth , max_parser_backtracks ) ;
2021-08-20 14:10:59 +00:00
while ( token_iterator - > type ! = TokenType : : Semicolon & & token_iterator . isValid ( ) )
+ + token_iterator ;
this_query_begin = token_iterator - > end ;
2021-07-12 09:30:16 +00:00
2021-08-20 14:10:59 +00:00
return MultiQueryProcessingStage : : CONTINUE_PARSING ;
2021-07-12 14:51:09 +00:00
}
2021-07-12 09:30:16 +00:00
2021-08-20 14:10:59 +00:00
return MultiQueryProcessingStage : : PARSING_FAILED ;
}
2021-07-12 09:30:16 +00:00
2021-08-20 14:10:59 +00:00
// INSERT queries may have the inserted data in the query text
// that follow the query itself, e.g. "insert into t format CSV 1;2".
// They need special handling. First of all, here we find where the
2024-04-10 19:39:02 +00:00
// inserted data ends. In multi-query mode, it is delimited by a
2021-08-20 14:10:59 +00:00
// newline.
2024-04-10 19:39:02 +00:00
// The VALUES format needs even more handling - we also allow the
2021-08-20 14:10:59 +00:00
// data to be delimited by semicolon. This case is handled later by
// the format parser itself.
// We can't do multiline INSERTs with inline data, because most
// row input formats (e.g. TSV) can't tell when the input stops,
// unlike VALUES.
auto * insert_ast = parsed_query - > as < ASTInsertQuery > ( ) ;
2022-02-02 17:53:04 +00:00
const char * query_to_execute_end = this_query_end ;
2021-08-20 14:10:59 +00:00
if ( insert_ast & & insert_ast - > data )
{
this_query_end = find_first_symbols < ' \n ' > ( insert_ast - > data , all_queries_end ) ;
insert_ast - > end = this_query_end ;
2022-02-02 17:53:04 +00:00
query_to_execute_end = isSyncInsertWithData ( * insert_ast , global_context ) ? insert_ast - > data : this_query_end ;
2021-07-12 09:30:16 +00:00
}
2021-08-20 14:10:59 +00:00
2022-02-02 17:53:04 +00:00
query_to_execute = all_queries_text . substr ( this_query_begin - all_queries_text . data ( ) , query_to_execute_end - this_query_begin ) ;
2021-08-20 14:10:59 +00:00
// Try to include the trailing comment with test hints. It is just
// a guess for now, because we don't yet know where the query ends
// if it is an INSERT query with inline data. We will do it again
// after we have processed the query. But even this guess is
// beneficial so that we see proper trailing comments in "echo" and
// server log.
2024-03-17 18:53:58 +00:00
adjustQueryEnd ( this_query_end , all_queries_end , max_parser_depth , max_parser_backtracks ) ;
2021-08-20 14:10:59 +00:00
return MultiQueryProcessingStage : : EXECUTE_QUERY ;
2021-07-12 09:30:16 +00:00
}
2022-03-14 11:00:47 +00:00
bool ClientBase : : executeMultiQuery ( const String & all_queries_text )
{
bool echo_query = echo_queries ;
{
/// disable logs if expects errors
2022-04-05 13:35:59 +00:00
TestHint test_hint ( all_queries_text ) ;
2023-03-08 11:13:28 +00:00
if ( test_hint . hasClientErrors ( ) | | test_hint . hasServerErrors ( ) )
2022-03-14 11:00:47 +00:00
processTextAsSingleQuery ( " SET send_logs_level = 'fatal' " ) ;
}
2023-10-12 10:24:21 +00:00
/// Test tags are started with "--" so they are interpreted as comments anyway.
/// But if the echo is enabled we have to remove the test tags from `all_queries_text`
/// because we don't want test tags to be echoed.
2022-03-17 09:49:14 +00:00
size_t test_tags_length = getTestTagsLength ( all_queries_text ) ;
2022-03-14 11:00:47 +00:00
/// Several queries separated by ';'.
/// INSERT data is ended by the end of line, not ';'.
/// An exception is VALUES format where we also support semicolon in
/// addition to end of line.
const char * this_query_begin = all_queries_text . data ( ) + test_tags_length ;
const char * this_query_end ;
const char * all_queries_end = all_queries_text . data ( ) + all_queries_text . size ( ) ;
String full_query ; // full_query is the query + inline INSERT data + trailing comments (the latter is our best guess for now).
String query_to_execute ;
ASTPtr parsed_query ;
2022-04-15 23:56:45 +00:00
std : : unique_ptr < Exception > current_exception ;
2022-03-14 11:00:47 +00:00
while ( true )
{
auto stage = analyzeMultiQueryText ( this_query_begin , this_query_end , all_queries_end ,
query_to_execute , parsed_query , all_queries_text , current_exception ) ;
switch ( stage )
{
case MultiQueryProcessingStage : : QUERIES_END :
case MultiQueryProcessingStage : : PARSING_FAILED :
{
return true ;
}
case MultiQueryProcessingStage : : CONTINUE_PARSING :
{
continue ;
}
case MultiQueryProcessingStage : : PARSING_EXCEPTION :
{
this_query_end = find_first_symbols < ' \n ' > ( this_query_end , all_queries_end ) ;
// Try to find test hint for syntax error. We don't know where
// the query ends because we failed to parse it, so we consume
// the entire line.
2022-03-17 09:49:14 +00:00
TestHint hint ( String ( this_query_begin , this_query_end - this_query_begin ) ) ;
2023-03-08 11:13:28 +00:00
if ( hint . hasServerErrors ( ) )
2022-03-14 11:00:47 +00:00
{
// Syntax errors are considered as client errors
2023-03-03 12:40:16 +00:00
current_exception - > addMessage ( " \n Expected server error: {}. " , hint . serverErrors ( ) ) ;
2022-03-14 11:00:47 +00:00
current_exception - > rethrow ( ) ;
}
2023-03-09 10:16:29 +00:00
if ( ! hint . hasExpectedClientError ( current_exception - > code ( ) ) )
2022-03-14 11:00:47 +00:00
{
2023-03-08 11:13:28 +00:00
if ( hint . hasClientErrors ( ) )
2023-03-03 12:40:16 +00:00
current_exception - > addMessage ( " \n Expected client error: {}. " , hint . clientErrors ( ) ) ;
2022-03-14 11:00:47 +00:00
current_exception - > rethrow ( ) ;
}
/// It's expected syntax error, skip the line
this_query_begin = this_query_end ;
current_exception . reset ( ) ;
continue ;
}
case MultiQueryProcessingStage : : EXECUTE_QUERY :
{
full_query = all_queries_text . substr ( this_query_begin - all_queries_text . data ( ) , this_query_end - this_query_begin ) ;
if ( query_fuzzer_runs )
{
if ( ! processWithFuzzing ( full_query ) )
return false ;
this_query_begin = this_query_end ;
continue ;
}
// Now we know for sure where the query ends.
// Look for the hint in the text of query + insert data + trailing
// comments, e.g. insert into t format CSV 'a' -- { serverError 123 }.
// Use the updated query boundaries we just calculated.
2022-03-17 09:49:14 +00:00
TestHint test_hint ( full_query ) ;
2022-03-14 11:00:47 +00:00
// Echo all queries if asked; makes for a more readable reference file.
echo_query = test_hint . echoQueries ( ) . value_or ( echo_query ) ;
try
{
processParsedSingleQuery ( full_query , query_to_execute , parsed_query , echo_query , false ) ;
}
catch ( . . . )
{
// Surprisingly, this is a client error. A server error would
2024-04-22 01:36:34 +00:00
// have been reported without throwing (see onReceiveExceptionFromServer()).
2023-01-23 13:16:14 +00:00
client_exception = std : : make_unique < Exception > ( getCurrentExceptionMessageAndPattern ( print_stack_trace ) , getCurrentExceptionCode ( ) ) ;
2022-03-14 11:00:47 +00:00
have_error = true ;
}
// Check whether the error (or its absence) matches the test hints
// (or their absence).
bool error_matches_hint = true ;
if ( have_error )
{
2023-03-08 11:13:28 +00:00
if ( test_hint . hasServerErrors ( ) )
2022-03-14 11:00:47 +00:00
{
if ( ! server_exception )
{
error_matches_hint = false ;
fmt : : print ( stderr , " Expected server error code '{}' but got no server error (query: {}). \n " ,
2023-03-03 12:40:16 +00:00
test_hint . serverErrors ( ) , full_query ) ;
2022-03-14 11:00:47 +00:00
}
2023-03-09 10:16:29 +00:00
else if ( ! test_hint . hasExpectedServerError ( server_exception - > code ( ) ) )
2022-03-14 11:00:47 +00:00
{
error_matches_hint = false ;
fmt : : print ( stderr , " Expected server error code: {} but got: {} (query: {}). \n " ,
2023-03-03 12:40:16 +00:00
test_hint . serverErrors ( ) , server_exception - > code ( ) , full_query ) ;
2022-03-14 11:00:47 +00:00
}
}
2023-03-08 11:13:28 +00:00
if ( test_hint . hasClientErrors ( ) )
2022-03-14 11:00:47 +00:00
{
if ( ! client_exception )
{
error_matches_hint = false ;
fmt : : print ( stderr , " Expected client error code '{}' but got no client error (query: {}). \n " ,
2023-03-03 12:40:16 +00:00
test_hint . clientErrors ( ) , full_query ) ;
2022-03-14 11:00:47 +00:00
}
2023-03-09 10:16:29 +00:00
else if ( ! test_hint . hasExpectedClientError ( client_exception - > code ( ) ) )
2022-03-14 11:00:47 +00:00
{
error_matches_hint = false ;
fmt : : print ( stderr , " Expected client error code '{}' but got '{}' (query: {}). \n " ,
2023-03-03 12:40:16 +00:00
test_hint . clientErrors ( ) , client_exception - > code ( ) , full_query ) ;
2022-03-14 11:00:47 +00:00
}
}
2023-03-08 11:13:28 +00:00
if ( ! test_hint . hasClientErrors ( ) & & ! test_hint . hasServerErrors ( ) )
2022-03-14 11:00:47 +00:00
{
// No error was expected but it still occurred. This is the
2022-04-17 23:02:49 +00:00
// default case without test hint, doesn't need additional
2022-03-14 11:00:47 +00:00
// diagnostics.
error_matches_hint = false ;
}
}
else
{
2023-03-08 11:13:28 +00:00
if ( test_hint . hasClientErrors ( ) )
2022-03-14 11:00:47 +00:00
{
error_matches_hint = false ;
fmt : : print ( stderr ,
" The query succeeded but the client error '{}' was expected (query: {}). \n " ,
2023-03-03 12:40:16 +00:00
test_hint . clientErrors ( ) , full_query ) ;
2022-03-14 11:00:47 +00:00
}
2023-03-08 11:13:28 +00:00
if ( test_hint . hasServerErrors ( ) )
2022-03-14 11:00:47 +00:00
{
error_matches_hint = false ;
fmt : : print ( stderr ,
" The query succeeded but the server error '{}' was expected (query: {}). \n " ,
2023-03-03 12:40:16 +00:00
test_hint . serverErrors ( ) , full_query ) ;
2022-03-14 11:00:47 +00:00
}
}
// If the error is expected, force reconnect and ignore it.
if ( have_error & & error_matches_hint )
{
client_exception . reset ( ) ;
server_exception . reset ( ) ;
have_error = false ;
2022-08-12 13:30:46 +00:00
if ( ! connection - > checkConnected ( connection_parameters . timeouts ) )
2022-03-14 11:00:47 +00:00
connect ( ) ;
}
// For INSERTs with inline data: use the end of inline data as
// reported by the format parser (it is saved in sendData()).
// This allows us to handle queries like:
// insert into t values (1); select 1
// , where the inline data is delimited by semicolon and not by a
// newline.
auto * insert_ast = parsed_query - > as < ASTInsertQuery > ( ) ;
if ( insert_ast & & isSyncInsertWithData ( * insert_ast , global_context ) )
{
this_query_end = insert_ast - > end ;
2022-10-07 10:46:45 +00:00
adjustQueryEnd (
this_query_end , all_queries_end ,
2024-03-17 18:53:58 +00:00
static_cast < unsigned > ( global_context - > getSettingsRef ( ) . max_parser_depth ) ,
static_cast < unsigned > ( global_context - > getSettingsRef ( ) . max_parser_backtracks ) ) ;
2022-03-14 11:00:47 +00:00
}
// Report error.
if ( have_error )
processError ( full_query ) ;
// Stop processing queries if needed.
if ( have_error & & ! ignore_error )
return is_interactive ;
this_query_begin = this_query_end ;
break ;
}
}
}
}
2021-07-22 21:27:26 +00:00
bool ClientBase : : processQueryText ( const String & text )
2021-07-11 11:36:27 +00:00
{
2023-04-10 04:38:13 +00:00
auto trimmed_input = trim ( text , [ ] ( char c ) { return isWhitespaceASCII ( c ) | | c = = ' ; ' ; } ) ;
if ( exit_strings . end ( ) ! = exit_strings . find ( trimmed_input ) )
2021-07-11 23:17:14 +00:00
return false ;
2021-07-11 11:36:27 +00:00
2023-04-10 04:38:13 +00:00
if ( trimmed_input . starts_with ( " \\ i " ) )
{
size_t skip_prefix_size = std : : strlen ( " \\ i " ) ;
auto file_name = trim (
trimmed_input . substr ( skip_prefix_size , trimmed_input . size ( ) - skip_prefix_size ) ,
[ ] ( char c ) { return isWhitespaceASCII ( c ) ; } ) ;
return processMultiQueryFromFile ( file_name ) ;
}
2021-07-23 20:54:49 +00:00
if ( ! is_multiquery )
2021-07-11 23:17:14 +00:00
{
assert ( ! query_fuzzer_runs ) ;
2021-08-19 11:07:47 +00:00
processTextAsSingleQuery ( text ) ;
2021-07-11 20:35:29 +00:00
2021-07-11 23:17:14 +00:00
return true ;
}
2021-07-12 14:51:09 +00:00
if ( query_fuzzer_runs )
{
processWithFuzzing ( text ) ;
return true ;
}
2021-07-11 23:17:14 +00:00
2021-09-04 18:19:01 +00:00
return executeMultiQuery ( text ) ;
2021-07-11 11:36:27 +00:00
}
2021-07-11 20:35:29 +00:00
2021-11-18 18:07:35 +00:00
String ClientBase : : prompt ( ) const
{
2022-10-19 21:27:12 +00:00
return prompt_by_server_display_name ;
2021-11-18 18:07:35 +00:00
}
2022-02-02 15:19:33 +00:00
void ClientBase : : initQueryIdFormats ( )
{
if ( ! query_id_formats . empty ( ) )
return ;
/// Initialize query_id_formats if any
2024-06-25 14:23:37 +00:00
if ( getClientConfiguration ( ) . has ( " query_id_formats " ) )
2022-02-02 15:19:33 +00:00
{
Poco : : Util : : AbstractConfiguration : : Keys keys ;
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . keys ( " query_id_formats " , keys ) ;
2022-02-02 15:19:33 +00:00
for ( const auto & name : keys )
2024-06-25 14:23:37 +00:00
query_id_formats . emplace_back ( name + " : " , getClientConfiguration ( ) . getString ( " query_id_formats. " + name ) ) ;
2022-02-02 15:19:33 +00:00
}
if ( query_id_formats . empty ( ) )
query_id_formats . emplace_back ( " Query id: " , " {query_id} \n " ) ;
}
2022-07-07 22:16:01 +00:00
bool ClientBase : : addMergeTreeSettings ( ASTCreateQuery & ast_create )
{
if ( ast_create . attach
| | ! ast_create . storage
| | ! ast_create . storage - > isExtendedStorageDefinition ( )
| | ! ast_create . storage - > engine
| | ast_create . storage - > engine - > name . find ( " MergeTree " ) = = std : : string : : npos )
return false ;
auto all_changed = cmd_merge_tree_settings . allChanged ( ) ;
if ( all_changed . begin ( ) = = all_changed . end ( ) )
return false ;
if ( ! ast_create . storage - > settings )
{
auto settings_ast = std : : make_shared < ASTSetQuery > ( ) ;
settings_ast - > is_standalone = false ;
ast_create . storage - > set ( ast_create . storage - > settings , settings_ast ) ;
}
auto & storage_settings = * ast_create . storage - > settings ;
bool added_new_setting = false ;
for ( const auto & setting : all_changed )
{
if ( ! storage_settings . changes . tryGet ( setting . getName ( ) ) )
{
storage_settings . changes . emplace_back ( setting . getName ( ) , setting . getValue ( ) ) ;
added_new_setting = true ;
}
}
return added_new_setting ;
}
2021-08-18 21:01:17 +00:00
void ClientBase : : runInteractive ( )
2021-07-11 20:35:29 +00:00
{
2024-06-25 14:23:37 +00:00
if ( getClientConfiguration ( ) . has ( " query_id " ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " query_id could be specified only in non-interactive mode " ) ;
2024-06-25 14:23:37 +00:00
if ( getClientConfiguration ( ) . getBool ( " print-time-to-stderr " , false ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " time option could be specified only in non-interactive mode " ) ;
2021-07-11 20:35:29 +00:00
2022-02-02 15:19:33 +00:00
initQueryIdFormats ( ) ;
2021-07-11 20:35:29 +00:00
/// Initialize DateLUT here to avoid counting time spent here as query execution time.
2023-02-27 00:40:00 +00:00
const auto local_tz = DateLUT : : instance ( ) . getTimeZone ( ) ;
2021-07-11 20:35:29 +00:00
suggest . emplace ( ) ;
2021-09-11 11:34:22 +00:00
if ( load_suggestions )
{
/// Load suggestion data from the server.
if ( global_context - > getApplicationType ( ) = = Context : : ApplicationType : : CLIENT )
2024-06-25 14:23:37 +00:00
suggest - > load < Connection > ( global_context , connection_parameters , getClientConfiguration ( ) . getInt ( " suggestion_limit " ) , wait_for_suggestions_to_load ) ;
2021-09-11 11:34:22 +00:00
else if ( global_context - > getApplicationType ( ) = = Context : : ApplicationType : : LOCAL )
2024-06-25 14:23:37 +00:00
suggest - > load < LocalConnection > ( global_context , connection_parameters , getClientConfiguration ( ) . getInt ( " suggestion_limit " ) , wait_for_suggestions_to_load ) ;
2021-09-11 11:34:22 +00:00
}
2021-07-11 20:35:29 +00:00
if ( home_path . empty ( ) )
{
2022-08-21 18:24:17 +00:00
const char * home_path_cstr = getenv ( " HOME " ) ; // NOLINT(concurrency-mt-unsafe)
2021-07-11 20:35:29 +00:00
if ( home_path_cstr )
home_path = home_path_cstr ;
2021-07-12 14:01:46 +00:00
}
2021-07-11 20:35:29 +00:00
/// Load command history if present.
2024-06-25 14:23:37 +00:00
if ( getClientConfiguration ( ) . has ( " history_file " ) )
history_file = getClientConfiguration ( ) . getString ( " history_file " ) ;
2021-07-11 20:35:29 +00:00
else
{
2022-08-21 18:24:17 +00:00
auto * history_file_from_env = getenv ( " CLICKHOUSE_HISTORY_FILE " ) ; // NOLINT(concurrency-mt-unsafe)
2021-07-11 20:35:29 +00:00
if ( history_file_from_env )
history_file = history_file_from_env ;
else if ( ! home_path . empty ( ) )
history_file = home_path + " /.clickhouse-client-history " ;
}
if ( ! history_file . empty ( ) & & ! fs : : exists ( history_file ) )
{
/// Avoid TOCTOU issue.
try
{
FS : : createFile ( history_file ) ;
}
catch ( const ErrnoException & e )
{
if ( e . getErrno ( ) ! = EEXIST )
2023-06-28 09:56:11 +00:00
{
2024-06-25 14:23:37 +00:00
error_stream < < getCurrentExceptionMessage ( false ) < < ' \n ' ;
2023-06-28 09:56:11 +00:00
}
2021-07-11 20:35:29 +00:00
}
}
LineReader : : Patterns query_extenders = { " \\ " } ;
2022-04-11 12:46:24 +00:00
LineReader : : Patterns query_delimiters = { " ; " , " \\ G " , " \\ G; " } ;
2023-08-04 07:20:01 +00:00
char word_break_characters [ ] = " \t \v \f \a \b \r \n `~!@#$%^&*()-=+[{]} \\ |;:' \" ,<.>/? " ;
2021-07-11 20:35:29 +00:00
# if USE_REPLXX
replxx : : Replxx : : highlighter_callback_t highlight_callback { } ;
2024-06-25 14:23:37 +00:00
if ( getClientConfiguration ( ) . getBool ( " highlight " , true ) )
2021-07-11 20:35:29 +00:00
highlight_callback = highlight ;
2023-08-04 07:20:01 +00:00
ReplxxLineReader lr (
* suggest ,
history_file ,
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . has ( " multiline " ) ,
2023-08-04 07:20:01 +00:00
query_extenders ,
query_delimiters ,
word_break_characters ,
highlight_callback ) ;
2021-07-11 20:35:29 +00:00
# else
2023-08-04 07:20:01 +00:00
LineReader lr (
history_file ,
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . has ( " multiline " ) ,
2023-08-04 07:20:01 +00:00
query_extenders ,
query_delimiters ,
word_break_characters ) ;
2021-07-11 20:35:29 +00:00
# endif
2022-08-29 13:38:15 +00:00
static const std : : initializer_list < std : : pair < String , String > > backslash_aliases =
{
{ " \\ l " , " SHOW DATABASES " } ,
{ " \\ d " , " SHOW TABLES " } ,
{ " \\ c " , " USE " } ,
} ;
2022-08-30 16:58:10 +00:00
static const std : : initializer_list < String > repeat_last_input_aliases =
{
" . " , /// Vim shortcut
" / " /// Oracle SQL Plus shortcut
} ;
2022-08-29 13:38:15 +00:00
String last_input ;
2021-07-11 20:35:29 +00:00
do
{
2023-04-06 23:04:51 +00:00
String input ;
{
/// Enable bracketed-paste-mode so that we are able to paste multiline queries as a whole.
/// But keep it disabled outside of query input, because it breaks password input
/// (e.g. if we need to reconnect and show a password prompt).
/// (Alternatively, we could make the password input ignore the control sequences.)
lr . enableBracketedPaste ( ) ;
SCOPE_EXIT ( { lr . disableBracketedPaste ( ) ; } ) ;
input = lr . readLine ( prompt ( ) , " :-] " ) ;
}
2021-07-11 20:35:29 +00:00
if ( input . empty ( ) )
break ;
has_vertical_output_suffix = false ;
2022-04-11 12:46:24 +00:00
if ( input . ends_with ( " \\ G " ) | | input . ends_with ( " \\ G; " ) )
2021-07-11 20:35:29 +00:00
{
2022-04-12 03:01:20 +00:00
if ( input . ends_with ( " \\ G " ) )
input . resize ( input . size ( ) - 2 ) ;
2022-04-12 10:51:19 +00:00
else if ( input . ends_with ( " \\ G; " ) )
2022-04-12 03:01:20 +00:00
input . resize ( input . size ( ) - 3 ) ;
2022-04-12 10:51:19 +00:00
2021-07-11 20:35:29 +00:00
has_vertical_output_suffix = true ;
}
2022-08-29 13:38:15 +00:00
for ( const auto & [ alias , command ] : backslash_aliases )
2021-10-30 06:44:31 +00:00
{
2021-10-31 14:42:30 +00:00
auto it = std : : search ( input . begin ( ) , input . end ( ) , alias . begin ( ) , alias . end ( ) ) ;
if ( it ! = input . end ( ) & & std : : all_of ( input . begin ( ) , it , isWhitespaceASCII ) )
2021-10-30 06:44:31 +00:00
{
2021-10-31 14:42:30 +00:00
it + = alias . size ( ) ;
if ( it = = input . end ( ) | | isWhitespaceASCII ( * it ) )
{
String new_input = command ;
// append the rest of input to the command
// for parameters support, e.g. \c db_name -> USE db_name
new_input . append ( it , input . end ( ) ) ;
input = std : : move ( new_input ) ;
break ;
}
2021-10-30 06:44:31 +00:00
}
}
2022-08-29 13:38:15 +00:00
for ( const auto & alias : repeat_last_input_aliases )
{
if ( input = = alias )
{
input = last_input ;
break ;
}
}
2023-08-10 04:11:07 +00:00
if ( suggest & & suggest - > getLastError ( ) = = ErrorCodes : : USER_SESSION_LIMIT_EXCEEDED )
{
// If a separate connection loading suggestions failed to open a new session,
// use the main session to receive them.
2024-06-25 14:23:37 +00:00
suggest - > load ( * connection , connection_parameters . timeouts , getClientConfiguration ( ) . getInt ( " suggestion_limit " ) , global_context - > getClientInfo ( ) ) ;
2023-08-10 04:11:07 +00:00
}
2021-08-18 21:01:17 +00:00
try
{
if ( ! processQueryText ( input ) )
break ;
2022-08-29 13:38:15 +00:00
last_input = input ;
2021-08-18 21:01:17 +00:00
}
catch ( const Exception & e )
{
2024-05-06 02:37:11 +00:00
if ( e . code ( ) = = ErrorCodes : : USER_EXPIRED )
break ;
2024-06-14 13:47:37 +00:00
2021-08-18 21:01:17 +00:00
/// We don't need to handle the test hints in the interactive mode.
2024-06-25 14:23:37 +00:00
error_stream < < " Exception on client: " < < std : : endl < < getExceptionMessage ( e , print_stack_trace , true ) < < std : : endl < < std : : endl ;
2022-04-15 23:56:45 +00:00
client_exception . reset ( e . clone ( ) ) ;
2021-08-18 21:01:17 +00:00
}
if ( client_exception )
{
/// client_exception may have been set above or elsewhere.
/// Client-side exception during query execution can result in the loss of
/// sync in the connection protocol.
/// So we reconnect and allow to enter the next query.
2022-08-12 13:30:46 +00:00
if ( ! connection - > checkConnected ( connection_parameters . timeouts ) )
2021-08-18 21:01:17 +00:00
connect ( ) ;
}
2021-07-11 20:35:29 +00:00
}
while ( true ) ;
if ( isNewYearMode ( ) )
2024-06-25 14:23:37 +00:00
output_stream < < " Happy new year. " < < std : : endl ;
2021-07-11 20:35:29 +00:00
else if ( isChineseNewYearMode ( local_tz ) )
2024-06-25 14:23:37 +00:00
output_stream < < " Happy Chinese new year. 春节快乐! " < < std : : endl ;
2021-07-11 20:35:29 +00:00
else
2024-06-25 14:23:37 +00:00
output_stream < < " Bye. " < < std : : endl ;
2021-07-11 20:35:29 +00:00
}
2023-04-10 04:38:13 +00:00
bool ClientBase : : processMultiQueryFromFile ( const String & file_name )
{
String queries_from_file ;
ReadBufferFromFile in ( file_name ) ;
readStringUntilEOF ( queries_from_file , in ) ;
2024-06-25 14:23:37 +00:00
if ( ! getClientConfiguration ( ) . has ( " log_comment " ) )
2023-11-28 14:38:20 +00:00
{
Settings settings = global_context - > getSettings ( ) ;
/// NOTE: cannot use even weakly_canonical() since it fails for /dev/stdin due to resolving of "pipe:[X]"
settings . log_comment = fs : : absolute ( fs : : path ( file_name ) ) ;
global_context - > setSettings ( settings ) ;
}
2023-04-10 04:38:13 +00:00
return executeMultiQuery ( queries_from_file ) ;
}
2021-07-22 21:27:26 +00:00
void ClientBase : : runNonInteractive ( )
2021-07-18 14:42:41 +00:00
{
2022-02-02 15:19:33 +00:00
if ( delayed_interactive )
initQueryIdFormats ( ) ;
2021-07-18 14:42:41 +00:00
if ( ! queries_files . empty ( ) )
{
for ( const auto & queries_file : queries_files )
{
for ( const auto & interleave_file : interleave_queries_files )
2023-04-10 04:38:13 +00:00
if ( ! processMultiQueryFromFile ( interleave_file ) )
2021-07-18 14:42:41 +00:00
return ;
2023-04-10 04:38:13 +00:00
if ( ! processMultiQueryFromFile ( queries_file ) )
2021-07-18 14:42:41 +00:00
return ;
}
return ;
}
2023-05-30 16:17:03 +00:00
if ( ! queries . empty ( ) )
2021-07-18 14:42:41 +00:00
{
2023-05-30 16:17:03 +00:00
for ( const auto & query : queries )
{
2023-09-04 11:40:56 +00:00
if ( query_fuzzer_runs )
{
if ( ! processWithFuzzing ( query ) )
return ;
}
else
{
if ( ! processQueryText ( query ) )
return ;
}
2023-05-30 16:17:03 +00:00
}
2021-07-18 14:42:41 +00:00
}
else
{
/// If 'query' parameter is not set, read a query from stdin.
/// The query is read entirely into memory (streaming is disabled).
ReadBufferFromFileDescriptor in ( STDIN_FILENO ) ;
2023-05-30 16:17:03 +00:00
String text ;
2021-07-18 14:42:41 +00:00
readStringUntilEOF ( text , in ) ;
2023-09-04 11:40:56 +00:00
if ( query_fuzzer_runs )
processWithFuzzing ( text ) ;
else
processQueryText ( text ) ;
2021-07-18 14:42:41 +00:00
}
}
2024-03-18 01:16:52 +00:00
# if defined(FUZZING_MODE)
extern " C " int LLVMFuzzerRunDriver ( int * argc , char * * * argv , int ( * callback ) ( const uint8_t * data , size_t size ) ) ;
ClientBase * app ;
void ClientBase : : runLibFuzzer ( )
{
app = this ;
2024-03-20 21:12:57 +00:00
std : : vector < String > fuzzer_args_holder ;
if ( const char * fuzzer_args_env = getenv ( " FUZZER_ARGS " ) ) // NOLINT(concurrency-mt-unsafe)
boost : : split ( fuzzer_args_holder , fuzzer_args_env , isWhitespaceASCII , boost : : token_compress_on ) ;
std : : vector < char * > fuzzer_args ;
fuzzer_args . push_back ( argv0 ) ;
for ( auto & arg : fuzzer_args_holder )
fuzzer_args . emplace_back ( arg . data ( ) ) ;
2024-03-18 01:35:32 +00:00
int fuzzer_argc = fuzzer_args . size ( ) ;
char * * fuzzer_argv = fuzzer_args . data ( ) ;
2024-03-18 01:16:52 +00:00
LLVMFuzzerRunDriver ( & fuzzer_argc , & fuzzer_argv , [ ] ( const uint8_t * data , size_t size )
{
try
{
String query ( reinterpret_cast < const char * > ( data ) , size ) ;
app - > processQueryText ( query ) ;
}
catch ( . . . )
{
return - 1 ;
}
return 0 ;
} ) ;
}
2024-03-20 21:12:57 +00:00
# else
void ClientBase : : runLibFuzzer ( ) { }
2024-03-18 01:16:52 +00:00
# endif
2021-09-04 18:19:01 +00:00
void ClientBase : : clearTerminal ( )
2021-07-11 23:17:14 +00:00
{
/// Clear from cursor until end of screen.
/// It is needed if garbage is left in terminal.
/// Show cursor. It can be left hidden by invocation of previous programs.
/// A test for this feature: perl -e 'print "x"x100000'; echo -ne '\033[0;0H\033[?25l'; clickhouse-client
2024-06-25 14:23:37 +00:00
output_stream < < " \033 [0J " " \033 [?25h " ;
2021-07-11 23:17:14 +00:00
}
2021-09-04 18:19:01 +00:00
void ClientBase : : showClientVersion ( )
2021-07-11 23:17:14 +00:00
{
2024-06-25 14:23:37 +00:00
output_stream < < VERSION_NAME < < " " + getName ( ) + " version " < < VERSION_STRING < < VERSION_OFFICIAL < < " . " < < std : : endl ;
2021-07-11 23:17:14 +00:00
}
2023-02-04 12:00:16 +00:00
namespace
{
/// Define transparent hash to we can use
/// std::string_view with the containers
struct TransparentStringHash
{
using is_transparent = void ;
size_t operator ( ) ( std : : string_view txt ) const
{
return std : : hash < std : : string_view > { } ( txt ) ;
}
} ;
2023-04-20 17:07:44 +00:00
/*
* This functor is used to parse command line arguments and replace dashes with underscores ,
* allowing options to be specified using either dashes or underscores .
*/
class OptionsAliasParser
{
public :
explicit OptionsAliasParser ( const boost : : program_options : : options_description & options )
{
options_names . reserve ( options . options ( ) . size ( ) ) ;
for ( const auto & option : options . options ( ) )
options_names . insert ( option - > long_name ( ) ) ;
}
/*
* Parses arguments by replacing dashes with underscores , and matches the resulting name with known options
* Implements boost : : program_options : : ext_parser logic
*/
2024-05-10 04:58:21 +00:00
std : : pair < std : : string , std : : string > operator ( ) ( const std : : string & token ) const
2023-04-20 17:07:44 +00:00
{
2024-05-10 04:58:21 +00:00
if ( ! token . starts_with ( " -- " ) )
2023-04-20 17:07:44 +00:00
return { } ;
std : : string arg = token . substr ( 2 ) ;
// divide token by '=' to separate key and value if options style=long_allow_adjacent
auto pos_eq = arg . find ( ' = ' ) ;
std : : string key = arg . substr ( 0 , pos_eq ) ;
if ( options_names . contains ( key ) )
// option does not require any changes, because it is already correct
return { } ;
std : : replace ( key . begin ( ) , key . end ( ) , ' - ' , ' _ ' ) ;
if ( ! options_names . contains ( key ) )
// after replacing '-' with '_' argument is still unknown
return { } ;
std : : string value ;
if ( pos_eq ! = std : : string : : npos & & pos_eq < arg . size ( ) )
value = arg . substr ( pos_eq + 1 ) ;
return { key , value } ;
}
private :
std : : unordered_set < std : : string > options_names ;
} ;
2023-02-04 12:00:16 +00:00
}
2024-06-26 20:34:21 +00:00
/// Enable optimizations even in debug builds because otherwise options parsing becomes extremely slow affecting .sh tests
# if defined(__clang__)
# pragma clang optimize on
# endif
2021-10-14 13:34:05 +00:00
void ClientBase : : parseAndCheckOptions ( OptionsDescription & options_description , po : : variables_map & options , Arguments & arguments )
{
2022-03-02 16:33:21 +00:00
if ( allow_repeated_settings )
2024-04-25 17:46:20 +00:00
addProgramOptionsAsMultitokens ( cmd_settings , options_description . main_description . value ( ) ) ;
2022-03-02 16:33:21 +00:00
else
2024-04-25 17:46:20 +00:00
addProgramOptions ( cmd_settings , options_description . main_description . value ( ) ) ;
2022-07-07 22:16:01 +00:00
if ( allow_merge_tree_settings )
2022-07-08 14:47:55 +00:00
{
/// Add merge tree settings manually, because names of some settings
/// may clash. Query settings have higher priority and we just
2022-07-13 20:57:24 +00:00
/// skip ambiguous merge tree settings.
2022-07-08 14:47:55 +00:00
auto & main_options = options_description . main_description . value ( ) ;
2022-12-16 17:50:01 +00:00
2023-02-04 12:00:16 +00:00
std : : unordered_set < std : : string , TransparentStringHash , std : : equal_to < > > main_option_names ;
2022-12-16 17:50:01 +00:00
for ( const auto & option : main_options . options ( ) )
main_option_names . insert ( option - > long_name ( ) ) ;
2022-07-08 14:47:55 +00:00
for ( const auto & setting : cmd_merge_tree_settings . all ( ) )
{
2023-02-04 12:00:16 +00:00
const auto add_setting = [ & ] ( const std : : string_view name )
{
if ( auto it = main_option_names . find ( name ) ; it ! = main_option_names . end ( ) )
return ;
2022-07-08 14:47:55 +00:00
2023-02-04 12:00:16 +00:00
if ( allow_repeated_settings )
2024-03-19 16:04:29 +00:00
addProgramOptionAsMultitoken ( cmd_merge_tree_settings , main_options , name , setting ) ;
2023-02-04 12:00:16 +00:00
else
2024-03-19 16:04:29 +00:00
addProgramOption ( cmd_merge_tree_settings , main_options , name , setting ) ;
2023-02-04 12:00:16 +00:00
} ;
const auto & setting_name = setting . getName ( ) ;
add_setting ( setting_name ) ;
const auto & settings_to_aliases = MergeTreeSettings : : Traits : : settingsToAliases ( ) ;
if ( auto it = settings_to_aliases . find ( setting_name ) ; it ! = settings_to_aliases . end ( ) )
{
for ( const auto alias : it - > second )
{
add_setting ( alias ) ;
}
}
2022-07-08 14:47:55 +00:00
}
}
2022-07-07 22:16:01 +00:00
2021-10-14 13:34:05 +00:00
/// Parse main commandline options.
2023-04-20 17:07:44 +00:00
auto parser = po : : command_line_parser ( arguments )
. options ( options_description . main_description . value ( ) )
. extra_parser ( OptionsAliasParser ( options_description . main_description . value ( ) ) )
. allow_unregistered ( ) ;
2021-10-14 13:34:05 +00:00
po : : parsed_options parsed = parser . run ( ) ;
2021-10-15 15:29:27 +00:00
/// Check unrecognized options without positional options.
auto unrecognized_options = po : : collect_unrecognized ( parsed . options , po : : collect_unrecognized_mode : : exclude_positional ) ;
if ( ! unrecognized_options . empty ( ) )
2021-12-16 09:42:49 +00:00
{
auto hints = this - > getHints ( unrecognized_options [ 0 ] ) ;
if ( ! hints . empty ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : UNRECOGNIZED_ARGUMENTS , " Unrecognized option '{}'. Maybe you meant {} " ,
unrecognized_options [ 0 ] , toString ( hints ) ) ;
2021-12-16 12:31:06 +00:00
2021-10-15 15:29:27 +00:00
throw Exception ( ErrorCodes : : UNRECOGNIZED_ARGUMENTS , " Unrecognized option '{}' " , unrecognized_options [ 0 ] ) ;
2021-12-16 09:42:49 +00:00
}
2021-10-15 15:29:27 +00:00
2023-07-27 17:52:44 +00:00
/// Check positional options.
2024-04-28 02:39:44 +00:00
for ( const auto & op : parsed . options )
{
if ( ! op . unregistered & & op . string_key . empty ( ) & & ! op . original_tokens [ 0 ] . starts_with ( " -- " )
& & ! op . original_tokens [ 0 ] . empty ( ) & & ! op . value . empty ( ) )
{
/// Two special cases for better usability:
/// - if the option contains a whitespace, it might be a query: clickhouse "SELECT 1"
/// These are relevant for interactive usage - user-friendly, but questionable in general.
/// In case of ambiguity or for scripts, prefer using proper options.
const auto & token = op . original_tokens [ 0 ] ;
po : : variable_value value ( boost : : any ( op . value ) , false ) ;
const char * option ;
if ( token . contains ( ' ' ) )
option = " query " ;
else
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " Positional option `{}` is not supported. " , token ) ;
if ( ! options . emplace ( option , value ) . second )
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " Positional option `{}` is not supported. " , token ) ;
}
}
2023-07-27 17:52:44 +00:00
2021-10-14 13:34:05 +00:00
po : : store ( parsed , options ) ;
}
2021-09-11 11:49:42 +00:00
2021-07-22 21:27:26 +00:00
void ClientBase : : init ( int argc , char * * argv )
2021-07-11 11:36:27 +00:00
{
namespace po = boost : : program_options ;
/// Don't parse options with Poco library, we prefer neat boost::program_options.
stopOptionsProcessing ( ) ;
stdin_is_a_tty = isatty ( STDIN_FILENO ) ;
stdout_is_a_tty = isatty ( STDOUT_FILENO ) ;
2022-07-14 18:45:37 +00:00
stderr_is_a_tty = isatty ( STDERR_FILENO ) ;
2021-07-29 12:48:07 +00:00
terminal_width = getTerminalWidth ( ) ;
2021-07-11 11:36:27 +00:00
2021-07-29 12:48:07 +00:00
std : : vector < Arguments > external_tables_arguments ;
2023-09-04 14:48:38 +00:00
Arguments common_arguments = { " " } ; /// 0th argument is ignored.
2022-02-10 09:43:08 +00:00
std : : vector < Arguments > hosts_and_ports_arguments ;
2021-07-11 11:36:27 +00:00
2024-03-20 21:12:57 +00:00
if ( argc )
argv0 = argv [ 0 ] ;
2022-02-10 09:43:08 +00:00
readArguments ( argc , argv , common_arguments , external_tables_arguments , hosts_and_ports_arguments ) ;
2021-07-11 11:36:27 +00:00
2023-04-20 17:07:44 +00:00
/// Support for Unicode dashes
2023-04-22 21:46:06 +00:00
/// Interpret Unicode dashes as default double-hyphen
2023-04-20 17:07:44 +00:00
for ( auto & arg : common_arguments )
2023-04-22 14:21:42 +00:00
{
2023-04-22 21:46:06 +00:00
// replace em-dash(U+2014)
2023-04-22 14:21:42 +00:00
boost : : replace_all ( arg , " — " , " -- " ) ;
2023-04-22 21:46:06 +00:00
// replace en-dash(U+2013)
boost : : replace_all ( arg , " – " , " -- " ) ;
// replace mathematical minus(U+2212)
boost : : replace_all ( arg , " − " , " -- " ) ;
2023-04-22 14:21:42 +00:00
}
2023-04-20 17:07:44 +00:00
2021-07-29 12:48:07 +00:00
OptionsDescription options_description ;
2021-08-20 21:19:06 +00:00
options_description . main_description . emplace ( createOptionsDescription ( " Main options " , terminal_width ) ) ;
/// Common options for clickhouse-client and clickhouse-local.
options_description . main_description - > add_options ( )
2024-04-26 11:19:22 +00:00
( " help " , " print usage summary, combine with --verbose to display all options " )
2024-04-25 16:51:17 +00:00
( " verbose " , " print query and other debugging info " )
2021-08-20 21:19:06 +00:00
( " version,V " , " print version information and exit " )
( " version-clean " , " print version in machine-readable format and exit " )
2021-09-19 18:24:06 +00:00
( " config-file,C " , po : : value < std : : string > ( ) , " config-file path " )
2023-05-17 03:43:35 +00:00
2023-09-04 14:48:38 +00:00
( " query,q " , po : : value < std : : vector < std : : string > > ( ) - > multitoken ( ) , R " (query; can be specified multiple times (--query " SELECT 1 " --query " SELECT 2 " ...)) " )
2023-09-04 11:40:56 +00:00
( " queries-file " , po : : value < std : : vector < std : : string > > ( ) - > multitoken ( ) , " file path with queries to execute; multiple files can be specified (--queries-file file1 file2...) " )
2023-05-23 00:27:17 +00:00
( " multiquery,n " , " If specified, multiple queries separated by semicolons can be listed after --query. For convenience, it is also possible to omit --query and pass the queries directly after --multiquery. " )
2023-05-17 03:43:35 +00:00
( " multiline,m " , " If specified, allow multiline queries (do not send the query on Enter) " )
2021-08-20 21:19:06 +00:00
( " database,d " , po : : value < std : : string > ( ) , " database " )
2023-05-17 03:43:35 +00:00
( " query_kind " , po : : value < std : : string > ( ) - > default_value ( " initial_query " ) , " One of initial_query/secondary_query/no_query " )
( " query_id " , po : : value < std : : string > ( ) , " query_id " )
2023-05-17 04:03:52 +00:00
2021-08-20 21:19:06 +00:00
( " history_file " , po : : value < std : : string > ( ) , " path to history file " )
( " stage " , po : : 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 " )
2022-11-12 02:55:26 +00:00
( " progress " , po : : value < ProgressOption > ( ) - > implicit_value ( ProgressOption : : TTY , " tty " ) - > default_value ( ProgressOption : : DEFAULT , " default " ) , " Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off " )
2021-08-20 21:19:06 +00:00
( " disable_suggestion,A " , " Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client. " )
2024-04-08 18:54:28 +00:00
( " wait_for_suggestions_to_load " , " Load suggestion data synchonously. " )
2021-08-20 21:19:06 +00:00
( " time,t " , " print query execution time to stderr in non-interactive mode (for benchmarks) " )
2021-10-30 12:28:50 +00:00
2021-08-20 21:19:06 +00:00
( " echo " , " in batch mode, print query before execution " )
2021-10-30 12:28:50 +00:00
( " log-level " , po : : value < std : : string > ( ) , " log level " )
( " server_logs_file " , po : : value < std : : string > ( ) , " put server logs into specified file " )
2023-09-04 11:40:56 +00:00
( " suggestion_limit " , po : : value < int > ( ) - > default_value ( 10000 ) , " Suggestion limit for how many databases, tables and columns to fetch. " )
2021-09-11 11:34:22 +00:00
2024-03-23 00:43:28 +00:00
( " format,f " , po : : value < std : : string > ( ) , " default output format (and input format for clickhouse-local) " )
( " output-format " , po : : value < std : : string > ( ) , " default output format (this option has preference over --format) " )
2024-03-22 20:06:36 +00:00
2021-08-20 21:19:06 +00:00
( " vertical,E " , " vertical output format, same as --format=Vertical or FORMAT Vertical or \\ G at end of command " )
( " highlight " , po : : value < bool > ( ) - > default_value ( true ) , " enable or disable basic syntax highlight in interactive command line " )
( " ignore-error " , " do not stop processing in multiquery mode " )
( " stacktrace " , " print stack traces of exceptions " )
2021-10-12 20:17:15 +00:00
( " hardware-utilization " , " print hardware utilization information in progress bar " )
2021-10-12 18:03:54 +00:00
( " print-profile-events " , po : : value ( & profile_events . print ) - > zero_tokens ( ) , " Printing ProfileEvents packets " )
( " profile-events-delay-ms " , po : : value < UInt64 > ( ) - > default_value ( profile_events . delay_ms ) , " Delay between printing `ProfileEvents` packets (-1 - print only totals, 0 - print every single packet) " )
2022-11-23 07:40:22 +00:00
( " processed-rows " , " print the number of locally processed rows " )
2021-10-29 12:04:08 +00:00
( " interactive " , " Process queries-file or --query query and start interactive mode " )
2021-11-16 19:21:03 +00:00
( " pager " , po : : value < std : : string > ( ) , " Pipe all output into this command (less or similar) " )
2023-11-27 18:21:49 +00:00
( " max_memory_usage_in_client " , po : : value < std : : string > ( ) , " Set memory limit in client/local server " )
2024-03-20 21:12:57 +00:00
( " fuzzer-args " , po : : value < std : : string > ( ) , " Command line arguments for the LLVM's libFuzzer driver. Only relevant if the application is compiled with libFuzzer. " )
2024-07-09 21:59:53 +00:00
2024-07-12 17:18:16 +00:00
( " client_logs_file " , po : : value < std : : string > ( ) , " Path to a file for writing client logs. Currently we only have fatal logs (when the client crashes) " )
2021-08-20 21:19:06 +00:00
;
2021-10-14 13:34:05 +00:00
addOptions ( options_description ) ;
2021-12-16 09:42:49 +00:00
2024-04-26 11:19:22 +00:00
OptionsDescription options_description_non_verbose = options_description ;
2021-12-20 15:09:20 +00:00
auto getter = [ ] ( const auto & op )
{
2021-12-16 09:42:49 +00:00
String op_long_name = op - > long_name ( ) ;
return " -- " + String ( op_long_name ) ;
} ;
2021-12-17 03:17:18 +00:00
if ( options_description . main_description )
{
const auto & main_options = options_description . main_description - > options ( ) ;
std : : transform ( main_options . begin ( ) , main_options . end ( ) , std : : back_inserter ( cmd_options ) , getter ) ;
}
if ( options_description . external_description )
{
const auto & external_options = options_description . external_description - > options ( ) ;
std : : transform ( external_options . begin ( ) , external_options . end ( ) , std : : back_inserter ( cmd_options ) , getter ) ;
}
2021-12-16 09:42:49 +00:00
2023-09-04 14:48:38 +00:00
po : : variables_map options ;
2021-10-14 13:34:05 +00:00
parseAndCheckOptions ( options_description , options , common_arguments ) ;
2021-07-11 11:36:27 +00:00
po : : notify ( options ) ;
if ( options . count ( " version " ) | | options . count ( " V " ) )
{
showClientVersion ( ) ;
2022-08-21 18:24:17 +00:00
exit ( 0 ) ; // NOLINT(concurrency-mt-unsafe)
2021-07-11 11:36:27 +00:00
}
if ( options . count ( " version-clean " ) )
{
2024-06-25 14:23:37 +00:00
output_stream < < VERSION_STRING ;
2022-08-21 18:24:17 +00:00
exit ( 0 ) ; // NOLINT(concurrency-mt-unsafe)
2021-07-11 11:36:27 +00:00
}
2024-04-25 17:46:20 +00:00
if ( options . count ( " verbose " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " verbose " , true ) ;
2024-04-25 17:46:20 +00:00
2021-07-11 11:36:27 +00:00
/// Output of help message.
if ( options . count ( " help " )
| | ( options . count ( " host " ) & & options [ " host " ] . as < std : : string > ( ) = = " elp " ) ) /// If user writes -help instead of --help.
{
2024-06-25 14:23:37 +00:00
if ( getClientConfiguration ( ) . getBool ( " verbose " , false ) )
2024-04-26 11:19:22 +00:00
printHelpMessage ( options_description , true ) ;
2024-04-25 17:46:20 +00:00
else
2024-04-26 11:19:22 +00:00
printHelpMessage ( options_description_non_verbose , false ) ;
2022-08-21 18:24:17 +00:00
exit ( 0 ) ; // NOLINT(concurrency-mt-unsafe)
2021-07-11 11:36:27 +00:00
}
2021-08-20 21:19:06 +00:00
/// Common options for clickhouse-client and clickhouse-local.
2024-06-25 14:23:37 +00:00
/// Output execution time to stderr in batch mode.
2021-08-20 21:19:06 +00:00
if ( options . count ( " time " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " print-time-to-stderr " , true ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " query " ) )
2023-05-30 16:17:03 +00:00
queries = options [ " query " ] . as < std : : vector < std : : string > > ( ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " query_id " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " query_id " , options [ " query_id " ] . as < std : : string > ( ) ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " database " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " database " , options [ " database " ] . as < std : : string > ( ) ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " config-file " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " config-file " , options [ " config-file " ] . as < std : : string > ( ) ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " queries-file " ) )
queries_files = options [ " queries-file " ] . as < std : : vector < std : : string > > ( ) ;
if ( options . count ( " multiline " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " multiline " , true ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " multiquery " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " multiquery " , true ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " ignore-error " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " ignore-error " , true ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " format " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " format " , options [ " format " ] . as < std : : string > ( ) ) ;
2024-03-23 00:43:28 +00:00
if ( options . count ( " output-format " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " output-format " , options [ " output-format " ] . as < std : : string > ( ) ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " vertical " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " vertical " , true ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " stacktrace " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " stacktrace " , true ) ;
2021-10-12 18:03:54 +00:00
if ( options . count ( " print-profile-events " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " print-profile-events " , true ) ;
2021-10-12 18:03:54 +00:00
if ( options . count ( " profile-events-delay-ms " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setUInt64 ( " profile-events-delay-ms " , options [ " profile-events-delay-ms " ] . as < UInt64 > ( ) ) ;
/// Whether to print the number of processed rows at
2022-11-23 07:40:22 +00:00
if ( options . count ( " processed-rows " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " print-num-processed-rows " , true ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " progress " ) )
2022-11-03 03:16:55 +00:00
{
switch ( options [ " progress " ] . as < ProgressOption > ( ) )
{
2022-11-12 02:55:26 +00:00
case DEFAULT :
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " progress " , " default " ) ;
2022-11-12 02:55:26 +00:00
break ;
2022-11-03 03:16:55 +00:00
case OFF :
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " progress " , " off " ) ;
2022-11-03 03:16:55 +00:00
break ;
case TTY :
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " progress " , " tty " ) ;
2022-11-03 03:16:55 +00:00
break ;
case ERR :
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " progress " , " err " ) ;
2022-11-03 03:16:55 +00:00
break ;
}
}
2021-08-20 21:19:06 +00:00
if ( options . count ( " echo " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " echo " , true ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " disable_suggestion " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " disable_suggestion " , true ) ;
2024-04-08 18:54:28 +00:00
if ( options . count ( " wait_for_suggestions_to_load " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " wait_for_suggestions_to_load " , true ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " suggestion_limit " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setInt ( " suggestion_limit " , options [ " suggestion_limit " ] . as < int > ( ) ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " highlight " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " highlight " , options [ " highlight " ] . as < bool > ( ) ) ;
2021-08-20 21:19:06 +00:00
if ( options . count ( " history_file " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " history_file " , options [ " history_file " ] . as < std : : string > ( ) ) ;
2021-11-03 09:17:27 +00:00
if ( options . count ( " interactive " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setBool ( " interactive " , true ) ;
2021-11-16 19:21:03 +00:00
if ( options . count ( " pager " ) )
2024-06-25 14:23:37 +00:00
getClientConfiguration ( ) . setString ( " pager " , options [ " pager " ] . as < std : : string > ( ) ) ;
2021-11-03 09:17:27 +00:00
2021-07-11 11:36:27 +00:00
if ( options . count ( " log-level " ) )
Poco : : Logger : : root ( ) . setLevel ( options [ " log-level " ] . as < std : : string > ( ) ) ;
2021-10-30 12:28:50 +00:00
if ( options . count ( " server_logs_file " ) )
server_logs_file = options [ " server_logs_file " ] . as < std : : string > ( ) ;
2021-07-11 11:36:27 +00:00
2021-10-04 18:00:31 +00:00
query_processing_stage = QueryProcessingStage : : fromString ( options [ " stage " ] . as < std : : string > ( ) ) ;
2022-05-17 11:48:06 +00:00
query_kind = parseQueryKind ( options [ " query_kind " ] . as < std : : string > ( ) ) ;
2021-10-12 18:03:54 +00:00
profile_events . print = options . count ( " print-profile-events " ) ;
profile_events . delay_ms = options [ " profile-events-delay-ms " ] . as < UInt64 > ( ) ;
2021-10-04 18:00:31 +00:00
2022-02-10 09:43:08 +00:00
processOptions ( options_description , options , external_tables_arguments , hosts_and_ports_arguments ) ;
2023-04-20 17:07:44 +00:00
{
std : : unordered_set < std : : string > alias_names ;
alias_names . reserve ( options_description . main_description - > options ( ) . size ( ) ) ;
for ( const auto & option : options_description . main_description - > options ( ) )
alias_names . insert ( option - > long_name ( ) ) ;
2024-06-25 14:23:37 +00:00
argsToConfig ( common_arguments , getClientConfiguration ( ) , 100 , & alias_names ) ;
2023-04-20 17:07:44 +00:00
}
2021-07-28 12:56:11 +00:00
clearPasswordFromCommandLine ( argc , argv ) ;
2021-12-31 07:58:44 +00:00
/// Limit on total memory usage
2024-06-25 14:23:37 +00:00
std : : string max_client_memory_usage = getClientConfiguration ( ) . getString ( " max_memory_usage_in_client " , " 0 " /*default value*/ ) ;
2023-11-27 18:21:49 +00:00
if ( max_client_memory_usage ! = " 0 " )
2021-12-31 07:58:44 +00:00
{
2023-11-28 13:29:44 +00:00
UInt64 max_client_memory_usage_int = parseWithSizeSuffix < UInt64 > ( max_client_memory_usage . c_str ( ) , max_client_memory_usage . length ( ) ) ;
2023-11-28 13:41:55 +00:00
2023-11-28 13:29:44 +00:00
total_memory_tracker . setHardLimit ( max_client_memory_usage_int ) ;
total_memory_tracker . setDescription ( " (total) " ) ;
total_memory_tracker . setMetric ( CurrentMetrics : : MemoryTracking ) ;
2021-12-31 07:58:44 +00:00
}
2023-12-04 09:00:06 +00:00
2024-03-08 20:37:30 +00:00
/// Print stacktrace in case of crash
2024-03-11 22:40:19 +00:00
HandledSignals : : instance ( ) . setupTerminateHandler ( ) ;
2024-03-08 20:37:30 +00:00
HandledSignals : : instance ( ) . setupCommonDeadlySignalHandlers ( ) ;
2024-03-11 22:40:19 +00:00
/// We don't setup signal handlers for SIGINT, SIGQUIT, SIGTERM because we don't
/// have an option for client to shutdown gracefully.
2024-03-08 20:37:30 +00:00
2024-07-09 21:59:53 +00:00
fatal_channel_ptr = new Poco : : SplitterChannel ;
fatal_console_channel_ptr = new Poco : : ConsoleChannel ;
fatal_channel_ptr - > addChannel ( fatal_console_channel_ptr ) ;
2024-07-12 17:18:16 +00:00
if ( options . count ( " client_logs_file " ) )
2024-07-09 21:59:53 +00:00
{
2024-07-12 17:18:16 +00:00
fatal_file_channel_ptr = new Poco : : SimpleFileChannel ( options [ " client_logs_file " ] . as < std : : string > ( ) ) ;
2024-07-09 21:59:53 +00:00
fatal_channel_ptr - > addChannel ( fatal_file_channel_ptr ) ;
}
2024-03-11 18:26:23 +00:00
fatal_log = createLogger ( " ClientBase " , fatal_channel_ptr . get ( ) , Poco : : Message : : PRIO_FATAL ) ;
2024-03-08 20:37:30 +00:00
signal_listener = std : : make_unique < SignalListener > ( nullptr , fatal_log ) ;
signal_listener_thread . start ( * signal_listener ) ;
2021-07-11 11:36:27 +00:00
}
}