ClickHouse/src/Client/ClientBase.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

330 lines
12 KiB
C++
Raw Normal View History

2021-07-11 11:36:27 +00:00
#pragma once
2023-08-22 03:52:57 +00:00
#include <string_view>
#include "Common/NamePrompter.h"
2022-07-07 22:16:01 +00:00
#include <Parsers/ASTCreateQuery.h>
2021-09-29 19:17:26 +00:00
#include <Common/ProgressIndication.h>
#include <Common/InterruptListener.h>
#include <Common/ShellCommand.h>
#include <Common/Stopwatch.h>
#include <Common/DNSResolver.h>
2021-09-29 19:17:26 +00:00
#include <Core/ExternalTable.h>
2021-07-11 11:36:27 +00:00
#include <Poco/Util/Application.h>
#include <Interpreters/Context.h>
#include <Client/Suggest.h>
2021-07-11 23:17:14 +00:00
#include <Client/QueryFuzzer.h>
2021-09-29 19:17:26 +00:00
#include <boost/program_options.hpp>
#include <Storages/StorageFile.h>
#include <Storages/SelectQueryInfo.h>
2022-07-07 22:16:01 +00:00
#include <Storages/MergeTree/MergeTreeSettings.h>
2021-07-11 11:36:27 +00:00
2022-10-05 00:29:52 +00:00
2021-07-29 12:48:07 +00:00
namespace po = boost::program_options;
namespace DB
{
2021-08-20 14:10:59 +00:00
2023-08-22 03:52:57 +00:00
static constexpr std::string_view DEFAULT_CLIENT_NAME = "client";
2023-03-10 02:23:57 +00:00
static const NameSet exit_strings
{
"exit", "quit", "logout", "учше", "йгше", "дщпщге",
"exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж",
"q", "й", "\\q", "\\Q", "\\й", "\\Й", ":q", "Жй"
};
2021-07-29 12:48:07 +00:00
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
2021-07-11 11:36:27 +00:00
2021-08-21 10:55:54 +00:00
enum MultiQueryProcessingStage
{
QUERIES_END,
PARSING_EXCEPTION,
CONTINUE_PARSING,
EXECUTE_QUERY,
PARSING_FAILED,
};
enum ProgressOption
{
2022-11-12 02:55:26 +00:00
DEFAULT,
OFF,
TTY,
ERR,
};
2022-11-12 02:55:26 +00:00
ProgressOption toProgressOption(std::string progress);
std::istream& operator>> (std::istream & in, ProgressOption & progress);
2021-10-08 14:03:54 +00:00
class InternalTextLogs;
2022-10-05 00:29:52 +00:00
class WriteBufferFromFileDescriptor;
2021-10-08 14:03:54 +00:00
2023-09-10 03:48:12 +00:00
class ClientBase : public Poco::Util::Application, public IHints<2>
2021-07-11 11:36:27 +00:00
{
2021-07-11 23:17:14 +00:00
2021-07-11 11:36:27 +00:00
public:
2021-07-22 21:27:26 +00:00
using Arguments = std::vector<String>;
2021-07-11 11:36:27 +00:00
2021-10-08 14:03:54 +00:00
ClientBase();
~ClientBase() override;
2021-07-11 11:36:27 +00:00
void init(int argc, char ** argv);
2021-07-23 20:54:49 +00:00
std::vector<String> getAllRegisteredNames() const override { return cmd_options; }
2021-07-11 11:36:27 +00:00
protected:
void runInteractive();
2021-08-19 11:07:47 +00:00
void runNonInteractive();
2021-08-20 14:10:59 +00:00
virtual bool processWithFuzzing(const String &)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Query processing with fuzzing is not implemented");
2021-08-20 14:10:59 +00:00
}
2021-08-19 11:07:47 +00:00
2021-09-04 18:19:01 +00:00
virtual void connect() = 0;
2021-08-21 10:55:54 +00:00
virtual void processError(const String & query) const = 0;
2021-09-19 21:42:28 +00:00
virtual String getName() const = 0;
2021-08-17 19:59:51 +00:00
void processOrdinaryQuery(const String & query_to_execute, ASTPtr parsed_query);
2021-08-18 14:39:04 +00:00
void processInsertQuery(const String & query_to_execute, ASTPtr parsed_query);
2021-08-20 14:10:59 +00:00
2021-08-21 10:55:54 +00:00
void processTextAsSingleQuery(const String & full_query);
2021-08-20 14:10:59 +00:00
void processParsedSingleQuery(const String & full_query, const String & query_to_execute,
ASTPtr parsed_query, std::optional<bool> echo_query_ = {}, bool report_error = false);
2021-08-18 14:39:04 +00:00
static void adjustQueryEnd(const char *& this_query_end, const char * all_queries_end, uint32_t max_parser_depth);
2021-08-21 10:55:54 +00:00
ASTPtr parseQuery(const char *& pos, const char * end, bool allow_multi_statements) const;
2021-10-03 09:11:59 +00:00
static void setupSignalHandler();
2021-08-20 14:10:59 +00:00
2022-03-14 11:00:47 +00:00
bool executeMultiQuery(const String & all_queries_text);
2021-08-20 14:10:59 +00:00
MultiQueryProcessingStage analyzeMultiQueryText(
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);
static void clearTerminal();
2021-09-04 18:19:01 +00:00
void showClientVersion();
2021-07-28 12:56:11 +00:00
using ProgramOptionsDescription = boost::program_options::options_description;
using CommandLineOptions = boost::program_options::variables_map;
2021-07-12 09:30:16 +00:00
2021-07-28 12:56:11 +00:00
struct OptionsDescription
{
std::optional<ProgramOptionsDescription> main_description;
std::optional<ProgramOptionsDescription> external_description;
std::optional<ProgramOptionsDescription> hosts_and_ports_description;
2021-07-28 12:56:11 +00:00
};
2022-04-05 13:38:44 +00:00
virtual void updateLoggerLevel(const String &) {}
2021-07-28 12:56:11 +00:00
virtual void printHelpMessage(const OptionsDescription & options_description) = 0;
virtual void addOptions(OptionsDescription & options_description) = 0;
2021-07-28 12:56:11 +00:00
virtual void processOptions(const OptionsDescription & options_description,
const CommandLineOptions & options,
const std::vector<Arguments> & external_tables_arguments,
const std::vector<Arguments> & hosts_and_ports_arguments) = 0;
2021-07-28 12:56:11 +00:00
virtual void processConfig() = 0;
2021-07-12 14:51:09 +00:00
2021-08-19 11:07:47 +00:00
bool processQueryText(const String & text);
2022-03-01 09:22:12 +00:00
virtual void readArguments(
int argc,
char ** argv,
Arguments & common_arguments,
std::vector<Arguments> & external_tables_arguments,
std::vector<Arguments> & hosts_and_ports_arguments) = 0;
2022-09-08 16:37:18 +00:00
void setInsertionTable(const ASTInsertQuery & insert_query);
void addMultiquery(std::string_view query, Arguments & common_arguments) const;
2021-08-19 11:07:47 +00:00
private:
void receiveResult(ASTPtr parsed_query, Int32 signals_before_stop, bool partial_result_on_first_cancel);
bool receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled_);
Send profile events for INSERT queries (previously only SELECT was supported) Reproducer: echo "1" | clickhouse-client --query="insert into function null('foo String') format TSV" --print-profile-events --profile-events-delay-ms=-1 However, clickhouse-local is differnt, it does sent the periodically, but only if query was long enough, i.e.: # yes | head -n100000 | clickhouse-local --query="insert into function null('foo String') format TSV" --print-profile-events --profile-events-delay-ms=-1 [s1.ch] 2022.05.20 15:20:27 [ 0 ] ContextLock: 10 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] DiskReadElapsedMicroseconds: 29 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] IOBufferAllocBytes: 200000 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] IOBufferAllocs: 1 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] InsertQuery: 1 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] InsertedBytes: 1000000 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] InsertedRows: 100000 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] MemoryTrackerUsage: 1521975 (gauge) [s1.ch] 2022.05.20 15:20:27 [ 0 ] OSCPUVirtualTimeMicroseconds: 102148 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] OSReadChars: 135700 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] OSWriteChars: 8 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] Query: 1 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] RWLockAcquiredReadLocks: 2 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] ReadBufferFromFileDescriptorRead: 5 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] ReadBufferFromFileDescriptorReadBytes: 134464 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] RealTimeMicroseconds: 293747 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] SoftPageFaults: 382 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] TableFunctionExecute: 1 (increment) [s1.ch] 2022.05.20 15:20:27 [ 0 ] UserTimeMicroseconds: 102148 (increment) v2: Proper support ProfileEvents in INSERTs (with protocol change) v3: Receive profile events on INSERT queries Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-05-20 15:19:40 +00:00
void receiveLogsAndProfileEvents(ASTPtr parsed_query);
2021-08-19 11:07:47 +00:00
bool receiveSampleBlock(Block & out, ColumnsDescription & columns_description, ASTPtr parsed_query);
bool receiveEndOfQuery();
void cancelQuery();
2021-08-19 11:07:47 +00:00
void onProgress(const Progress & value);
void onTimezoneUpdate(const String & tz);
2021-08-19 11:07:47 +00:00
void onData(Block & block, ASTPtr parsed_query);
void onLogData(Block & block);
void onTotals(Block & block, ASTPtr parsed_query);
void onExtremes(Block & block, ASTPtr parsed_query);
void onReceiveExceptionFromServer(std::unique_ptr<Exception> && e);
2021-10-15 20:18:20 +00:00
void onProfileInfo(const ProfileInfo & profile_info);
2021-08-19 11:07:47 +00:00
void onEndOfStream();
2021-09-14 11:06:00 +00:00
void onProfileEvents(Block & block);
2021-08-19 11:07:47 +00:00
void sendData(Block & sample, const ColumnsDescription & columns_description, ASTPtr parsed_query);
void sendDataFrom(ReadBuffer & buf, Block & sample,
const ColumnsDescription & columns_description, ASTPtr parsed_query, bool have_more_data = false);
void sendDataFromPipe(Pipe && pipe, ASTPtr parsed_query, bool have_more_data = false);
void sendDataFromStdin(Block & sample, const ColumnsDescription & columns_description, ASTPtr parsed_query);
2021-08-21 10:55:54 +00:00
void sendExternalTables(ASTPtr parsed_query);
2021-08-19 11:07:47 +00:00
2022-05-09 19:13:02 +00:00
void initOutputFormat(const Block & block, ASTPtr parsed_query);
2021-08-19 11:07:47 +00:00
void initLogsOutputStream();
String prompt() const;
2021-07-12 12:25:17 +00:00
2021-08-21 10:55:54 +00:00
void resetOutput();
void parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments);
2021-07-22 21:27:26 +00:00
void updateSuggest(const ASTPtr & ast);
2022-02-02 15:19:33 +00:00
void initQueryIdFormats();
2022-07-07 22:16:01 +00:00
bool addMergeTreeSettings(ASTCreateQuery & ast_create);
2022-02-02 15:19:33 +00:00
2021-07-22 21:27:26 +00:00
protected:
static bool isSyncInsertWithData(const ASTInsertQuery & insert_query, const ContextPtr & context);
2022-07-04 13:40:23 +00:00
bool processMultiQueryFromFile(const String & file_name);
/// Adjust some settings after command line options and config had been processed.
void adjustSettings();
2023-11-23 17:13:12 +00:00
void initTTYBuffer(ProgressOption progress);
2022-11-12 02:55:26 +00:00
Fix possible heap-use-after-free in local if history file cannot be created ASAN report: Code: 586. DB::ErrnoException: Cannot create file: /src/.clickhouse_history, errno: 2, strerror: No such file or directory. (CANNOT_CREATE_FILE) ================================================================= ==1==ERROR: AddressSanitizer: heap-use-after-free on address 0x6240000208f0 at pc 0x000030d22ade bp 0x7ffff2ff3f70 sp 0x7ffff2ff3f68 READ of size 8 at 0x6240000208f0 thread T2 #0 0x30d22add in DB::ProcessList::insert() build_docker/../src/Interpreters/ProcessList.cpp:89:36 #1 0x31411018 in DB::executeQueryImpl() build_docker/../src/Interpreters/executeQuery.cpp:516:60 #2 0x3140e1ab in DB::executeQuery() build_docker/../src/Interpreters/executeQuery.cpp:1083:30 #3 0x3364391e in DB::LocalConnection::sendQuery() build_docker/../src/Client/LocalConnection.cpp:119:21 #4 0x3367bab0 in DB::Suggest::fetch() build_docker/../src/Client/Suggest.cpp:141:16 #5 0x336820eb in void DB::Suggest::load<DB::LocalConnection>()::'lambda'()::operator()() const build_docker/../src/Client/Suggest.cpp:118:17 0x6240000208f0 is located 2032 bytes inside of 7056-byte region [0x624000020100,0x624000021c90) freed by thread T0 here: #0 0xe381ef2 in operator delete(void*, unsigned long) (/wrk/clickhouse-asan+0xe381ef2) (BuildId: 6ea6d1a5d2d5a164f60f0fd8230936305bc8d9d0) #1 0x335509fe in DB::ClientBase::~ClientBase() build_docker/../src/Client/ClientBase.cpp:293:25 #2 0x1f809bd5 in mainEntryClickHouseLocal(int, char**) build_docker/../programs/local/LocalServer.cpp:804:5 #3 0xe3856ad in main build_docker/../programs/main.cpp:482:12 #4 0x7ffff7dc0082 in __libc_start_main /build/glibc-SzIz7B/glibc-2.31/csu/../csu/libc-start.c:308:16 previously allocated by thread T0 here: #0 0xe38128d in operator new(unsigned long) (/wrk/clickhouse-asan+0xe38128d) (BuildId: 6ea6d1a5d2d5a164f60f0fd8230936305bc8d9d0) #1 0x2f34a7f3 in std::__1::__unique_if<DB::ContextSharedPart>::__unique_single std::__1::make_unique[abi:v15003]<DB::ContextSharedPart>() build_docker/../contrib/libcxx/include/__memory/unique_ptr.h:714:28 #2 0x2f34a7f3 in DB::Context::createShared() build_docker/../src/Interpreters/Context.cpp:603:32 #3 0x1f7f901d in DB::LocalServer::processConfig() build_docker/../programs/local/LocalServer.cpp:535:22 #4 0x1f7f4d92 in DB::LocalServer::main() build_docker/../programs/local/LocalServer.cpp:419:5 #5 0x3af24ffe in Poco::Util::Application::run() build_docker/../contrib/poco/Util/src/Application.cpp:334:8 #6 0x1f809bca in mainEntryClickHouseLocal(int, char**) build_docker/../programs/local/LocalServer.cpp:803:20 #7 0xe3856ad in main build_docker/../programs/main.cpp:482:12 #8 0x7ffff7dc0082 in __libc_start_main /build/glibc-SzIz7B/glibc-2.31/csu/../csu/libc-start.c:308:16 Thread T2 created by T0 here: #0 0xe32fedc in pthread_create (/wrk/clickhouse-asan+0xe32fedc) (BuildId: 6ea6d1a5d2d5a164f60f0fd8230936305bc8d9d0) #1 0x336806df in std::__1::__libcpp_thread_create[abi:v15003](unsigned long*, void* (*)(void*), void*) build_docker/../contrib/libcxx/include/__threading_support:376:10 #2 0x336806df in std::__1::thread::thread<void DB::Suggest::load<DB::LocalConnection>()::'lambda'(), void>() build_docker/../contrib/libcxx/include/thread:311:16 #3 0x3367ff5b in void DB::Suggest::load<DB::LocalConnection>(std::__1::shared_ptr<DB::Context const>, DB::ConnectionParameters const&, int) build_docker/../src/Client/Suggest.cpp:110:22 #4 0x3357fee9 in DB::ClientBase::runInteractive() build_docker/../src/Client/ClientBase.cpp:2066:22 #5 0x1f7f5264 in DB::LocalServer::main() build_docker/../programs/local/LocalServer.cpp #6 0x3af24ffe in Poco::Util::Application::run() build_docker/../contrib/poco/Util/src/Application.cpp:334:8 #7 0x1f809bca in mainEntryClickHouseLocal(int, char**) build_docker/../programs/local/LocalServer.cpp:803:20 #8 0xe3856ad in main build_docker/../programs/main.cpp:482:12 #9 0x7ffff7dc0082 in __libc_start_main /build/glibc-SzIz7B/glibc-2.31/csu/../csu/libc-start.c:308:16 Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-11-20 20:40:43 +00:00
/// Should be one of the first, to be destroyed the last,
/// since other members can use them.
SharedContextHolder shared_context;
ContextMutablePtr global_context;
2021-07-22 21:27:26 +00:00
bool is_interactive = false; /// Use either interactive line editing interface or batch mode.
bool is_multiquery = false;
2021-10-29 12:04:08 +00:00
bool delayed_interactive = false;
2021-07-22 21:27:26 +00:00
bool echo_queries = false; /// Print queries before execution in batch mode.
bool ignore_error = false; /// In case of errors, don't print error message, continue to next query. Only applicable for non-interactive mode.
bool print_time_to_stderr = false; /// Output execution time to stderr in batch mode.
std::optional<Suggest> suggest;
2021-09-11 11:34:22 +00:00
bool load_suggestions = false;
2021-07-22 21:27:26 +00:00
2023-09-04 11:40:56 +00:00
std::vector<String> queries; /// Queries passed via '--query'
2021-07-22 21:27:26 +00:00
std::vector<String> queries_files; /// If not empty, queries will be read from these files
std::vector<String> interleave_queries_files; /// If not empty, run queries from these files before processing every file from 'queries_files'.
std::vector<String> cmd_options;
2021-07-22 21:27:26 +00:00
bool stdin_is_a_tty = false; /// stdin is a terminal.
bool stdout_is_a_tty = false; /// stdout is a terminal.
bool stderr_is_a_tty = false; /// stderr is a terminal.
2021-07-22 21:27:26 +00:00
uint64_t terminal_width = 0;
String pager;
2021-09-04 18:19:01 +00:00
String format; /// Query results output format.
bool select_into_file = false; /// If writing result INTO OUTFILE. It affects progress rendering.
bool select_into_file_and_stdout = false; /// If writing result INTO OUTFILE AND STDOUT. It affects progress rendering.
2021-09-04 18:19:01 +00:00
bool is_default_format = true; /// false, if format is set in the config or command line.
size_t format_max_block_size = 0; /// Max block size for console output.
String insert_format; /// Format of INSERT data that is read from stdin in batch mode.
size_t insert_format_max_block_size = 0; /// Max block size when reading INSERT data.
size_t max_client_network_bandwidth = 0; /// The maximum speed of data exchange over the network for the client in bytes per second.
bool has_vertical_output_suffix = false; /// Is \G present at the end of the query string?
/// We will format query_id in interactive mode in various ways, the default is just to print Query id: ...
std::vector<std::pair<String, String>> query_id_formats;
2021-07-22 21:27:26 +00:00
/// Settings specified via command line args
Settings cmd_settings;
2022-07-07 22:16:01 +00:00
MergeTreeSettings cmd_merge_tree_settings;
2021-07-22 21:27:26 +00:00
/// thread status should be destructed before shared context because it relies on process list.
std::optional<ThreadStatus> thread_status;
ServerConnectionPtr connection;
ConnectionParameters connection_parameters;
2021-07-22 21:27:26 +00:00
/// Buffer that reads from stdin in batch mode.
ReadBufferFromFileDescriptor std_in{STDIN_FILENO};
/// Console output.
WriteBufferFromFileDescriptor std_out{STDOUT_FILENO};
std::unique_ptr<ShellCommand> pager_cmd;
2021-07-22 21:27:26 +00:00
/// The user can specify to redirect query output to a file.
std::unique_ptr<WriteBuffer> out_file_buf;
2021-10-11 16:11:50 +00:00
std::shared_ptr<IOutputFormat> output_format;
2021-07-22 21:27:26 +00:00
/// The user could specify special file for server logs (stderr by default)
std::unique_ptr<WriteBuffer> out_logs_buf;
String server_logs_file;
2021-10-08 14:03:54 +00:00
std::unique_ptr<InternalTextLogs> logs_out_stream;
2021-07-22 21:27:26 +00:00
2022-10-01 21:19:36 +00:00
/// /dev/tty if accessible or std::cerr - for progress bar.
/// We prefer to output progress bar directly to tty to allow user to redirect stdout and stderr and still get the progress indication.
2022-10-05 00:29:52 +00:00
std::unique_ptr<WriteBufferFromFileDescriptor> tty_buf;
2022-10-01 21:19:36 +00:00
2021-09-04 18:19:01 +00:00
String home_path;
String history_file; /// Path to a file containing command history.
2021-08-17 19:59:51 +00:00
2021-09-04 18:19:01 +00:00
String current_profile;
2021-08-17 19:59:51 +00:00
2021-09-04 18:19:01 +00:00
UInt64 server_revision = 0;
String server_version;
String prompt_by_server_display_name;
String server_display_name;
2021-08-17 19:59:51 +00:00
2021-09-04 18:19:01 +00:00
ProgressIndication progress_indication;
bool need_render_progress = true;
2022-02-15 12:11:13 +00:00
bool need_render_profile_events = true;
2021-09-04 18:19:01 +00:00
bool written_first_block = false;
size_t processed_rows = 0; /// How many rows have been read or written.
bool print_num_processed_rows = false; /// Whether to print the number of processed rows at
2021-08-17 19:59:51 +00:00
2021-10-16 11:28:57 +00:00
bool print_stack_trace = false;
2021-08-17 19:59:51 +00:00
/// The last exception that was received from the server. Is used for the
/// return code in batch mode.
std::unique_ptr<Exception> server_exception;
/// Likewise, the last exception that occurred on the client.
std::unique_ptr<Exception> client_exception;
2021-09-04 18:19:01 +00:00
/// If the last query resulted in exception. `server_exception` or
/// `client_exception` must be set.
bool have_error = false;
2021-08-17 19:59:51 +00:00
2021-09-04 18:19:01 +00:00
std::list<ExternalTable> external_tables; /// External tables info.
2021-09-24 08:29:01 +00:00
bool send_external_tables = false;
2021-09-04 18:19:01 +00:00
NameToNameMap query_parameters; /// Dictionary with query parameters for prepared statements.
2021-08-18 14:39:04 +00:00
2021-09-04 18:19:01 +00:00
QueryFuzzer fuzzer;
int query_fuzzer_runs = 0;
2022-08-11 02:34:10 +00:00
int create_query_fuzzer_runs = 0;
2021-08-19 11:07:47 +00:00
struct
{
bool print = false;
/// UINT64_MAX -- print only last
UInt64 delay_ms = 0;
Stopwatch watch;
/// For printing only last (delay_ms == 0).
Block last_block;
} profile_events;
2021-09-04 18:19:01 +00:00
QueryProcessingStage::Enum query_processing_stage;
ClientInfo::QueryKind query_kind;
bool fake_drop = false;
2022-02-17 17:28:46 +00:00
struct HostAndPort
{
String host;
2022-03-10 19:41:03 +00:00
std::optional<UInt16> port;
};
std::vector<HostAndPort> hosts_and_ports{};
2022-03-02 16:33:21 +00:00
bool allow_repeated_settings = false;
2022-07-07 22:16:01 +00:00
bool allow_merge_tree_settings = false;
bool cancelled = false;
/// Does log_comment has specified by user?
bool has_log_comment = false;
2021-07-11 11:36:27 +00:00
};
}