ClickHouse/src/Client/ClientBase.cpp

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

3251 lines
119 KiB
C++
Raw Normal View History

2021-07-22 21:27:26 +00:00
#include <Client/ClientBase.h>
#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
#include <base/argsToConfig.h>
#include <base/safeExit.h>
#include <Core/Block.h>
2024-03-19 16:04:29 +00:00
#include <Core/BaseSettingsProgramOptions.h>
#include <Core/Protocol.h>
#include <Common/DateLUT.h>
#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>
#include <Common/filesystemHelpers.h>
2021-08-18 14:39:04 +00:00
#include <Common/NetException.h>
#include <Common/SignalHandlers.h>
#include <Common/tryGetFileNameByFileDescriptor.h>
#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>
#include <Parsers/ASTCreateFunctionQuery.h>
2022-11-22 02:41:23 +00:00
#include <Parsers/Access/ASTCreateUserQuery.h>
#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>
#include <Parsers/ASTColumnDeclaration.h>
2022-07-07 22:16:01 +00:00
#include <Parsers/ASTFunction.h>
#include <Parsers/PRQL/ParserPRQLQuery.h>
#include <Parsers/Kusto/ParserKQLStatement.h>
#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>
#include <QueryPipeline/QueryPipeline.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
2021-08-17 19:59:51 +00:00
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#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>
#include <IO/CompressionMethod.h>
#include <IO/ForkWriteBuffer.h>
#include <Access/AccessControl.h>
#include <Storages/ColumnsDescription.h>
#include <boost/algorithm/string/case_conv.hpp>
#include <boost/algorithm/string/replace.hpp>
2024-03-20 21:12:57 +00:00
#include <boost/algorithm/string/split.hpp>
#include <iostream>
#include <filesystem>
#include <limits>
#include <map>
#include <memory>
#include <unordered_map>
2023-11-13 09:09:23 +00:00
#include <Common/config_version.h>
#include "config.h"
2021-08-17 19:59:51 +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
namespace CurrentMetrics
{
extern const Metric MemoryTracking;
}
2021-07-11 11:36:27 +00:00
namespace DB
{
namespace ErrorCodes
{
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;
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;
extern const int FILE_ALREADY_EXISTS;
2023-08-10 04:11:07 +00:00
extern const int USER_SESSION_LIMIT_EXCEEDED;
extern const int NOT_IMPLEMENTED;
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
}
}
namespace ProfileEvents
{
extern const Event UserTimeMicroseconds;
extern const Event SystemTimeMicroseconds;
2021-07-31 12:34:29 +00:00
}
namespace
{
constexpr UInt64 THREAD_GROUP_ID = 0;
}
2021-07-31 12:34:29 +00:00
namespace DB
{
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);
}
std::istream& operator>> (std::istream & in, ProgressOption & progress)
{
std::string token;
in >> token;
2022-11-12 02:55:26 +00:00
progress = toProgressOption(token);
return in;
}
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);
}
static void incrementProfileEventsBlock(Block & dst, const Block & src)
{
if (!dst)
{
dst = src.cloneEmpty();
}
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();
Fix filtering out snapshots from profile events This fixes POSITION_OUT_OF_BOUND error: $ clickhouse-client --print-profile-events --profile-events-delay-ms=-1 -n -q 'select sleep(1); select 1' 0 [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] ContextLock: 9 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] FunctionExecute: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] MemoryTrackerUsage: 2132102 (gauge) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkReceiveElapsedMicroseconds: 139 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkSendBytes: 4850 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkSendElapsedMicroseconds: 1844 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] Query: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] RWLockAcquiredReadLocks: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectQuery: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectedBytes: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectedRows: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSCPUVirtualTimeMicroseconds: 1842 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSReadChars: 426 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSWriteChars: 322 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] RealTimeMicroseconds: 1002689 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] SleepFunctionCalls: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] SleepFunctionMicroseconds: 1000000 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] UserTimeMicroseconds: 1843 (increment) 1 Error on processing query: Code: 11. DB::Exception: Position out of bound in Block::erase(), max position = 5. (POSITION_OUT_OF_BOUND) (version 22.4.1.1) (query: select 1) Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-04-12 16:51:39 +00:00
const auto & src_array_thread_id = typeid_cast<const ColumnUInt64 &>(*src.getByName("thread_id").column).getData();
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
{
return std::tie(name, host_name)
< std::tie(rhs.name, rhs.host_name);
}
};
std::map<Id, UInt64> rows_by_name;
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
auto thread_id = src_array_thread_id[src_row];
if (thread_id != THREAD_GROUP_ID)
continue;
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;
Fix filtering out snapshots from profile events This fixes POSITION_OUT_OF_BOUND error: $ clickhouse-client --print-profile-events --profile-events-delay-ms=-1 -n -q 'select sleep(1); select 1' 0 [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] ContextLock: 9 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] FunctionExecute: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] MemoryTrackerUsage: 2132102 (gauge) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkReceiveElapsedMicroseconds: 139 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkSendBytes: 4850 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkSendElapsedMicroseconds: 1844 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] Query: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] RWLockAcquiredReadLocks: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectQuery: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectedBytes: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectedRows: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSCPUVirtualTimeMicroseconds: 1842 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSReadChars: 426 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSWriteChars: 322 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] RealTimeMicroseconds: 1002689 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] SleepFunctionCalls: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] SleepFunctionMicroseconds: 1000000 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] UserTimeMicroseconds: 1843 (increment) 1 Error on processing query: Code: 11. DB::Exception: Position out of bound in Block::erase(), max position = 5. (POSITION_OUT_OF_BOUND) (version 22.4.1.1) (query: select 1) Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-04-12 16:51:39 +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]))
{
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)
{
Fix filtering out snapshots from profile events This fixes POSITION_OUT_OF_BOUND error: $ clickhouse-client --print-profile-events --profile-events-delay-ms=-1 -n -q 'select sleep(1); select 1' 0 [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] ContextLock: 9 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] FunctionExecute: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] MemoryTrackerUsage: 2132102 (gauge) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkReceiveElapsedMicroseconds: 139 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkSendBytes: 4850 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkSendElapsedMicroseconds: 1844 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] Query: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] RWLockAcquiredReadLocks: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectQuery: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectedBytes: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectedRows: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSCPUVirtualTimeMicroseconds: 1842 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSReadChars: 426 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSWriteChars: 322 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] RealTimeMicroseconds: 1002689 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] SleepFunctionCalls: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] SleepFunctionMicroseconds: 1000000 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] UserTimeMicroseconds: 1843 (increment) 1 Error on processing query: Code: 11. DB::Exception: Position out of bound in Block::erase(), max position = 5. (POSITION_OUT_OF_BOUND) (version 22.4.1.1) (query: select 1) Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-04-12 16:51:39 +00:00
for (size_t col = 0; col < src.columns(); ++col)
{
Fix filtering out snapshots from profile events This fixes POSITION_OUT_OF_BOUND error: $ clickhouse-client --print-profile-events --profile-events-delay-ms=-1 -n -q 'select sleep(1); select 1' 0 [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] ContextLock: 9 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] FunctionExecute: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] MemoryTrackerUsage: 2132102 (gauge) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkReceiveElapsedMicroseconds: 139 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkSendBytes: 4850 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] NetworkSendElapsedMicroseconds: 1844 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] Query: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] RWLockAcquiredReadLocks: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectQuery: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectedBytes: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:48 [ 0 ] SelectedRows: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSCPUVirtualTimeMicroseconds: 1842 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSReadChars: 426 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] OSWriteChars: 322 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] RealTimeMicroseconds: 1002689 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] SleepFunctionCalls: 1 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] SleepFunctionMicroseconds: 1000000 (increment) [p1.azat.localdomain] 2022.04.12 19:31:49 [ 0 ] UserTimeMicroseconds: 1843 (increment) 1 Error on processing query: Code: 11. DB::Exception: Position out of bound in Block::erase(), max position = 5. (POSITION_OUT_OF_BOUND) (version 22.4.1.1) (query: select 1) Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-04-12 16:51:39 +00:00
mutable_columns[col]->insert((*src.getByPosition(col).column)[pos]);
}
}
dst.setColumns(std::move(mutable_columns));
}
2021-10-02 08:10:34 +00:00
std::atomic<Int32> exit_after_signals = 0;
2021-10-02 08:10:34 +00:00
class QueryInterruptHandler : private boost::noncopyable
{
public:
/// 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); }
/// 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.
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
{
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
}
/// To cancel the query on local format error.
class LocalFormatError : public DB::Exception
{
public:
using Exception::Exception;
};
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-02 08:10:34 +00:00
void ClientBase::setupSignalHandler()
{
QueryInterruptHandler::stop();
2021-10-02 08:10:34 +00:00
struct sigaction new_act;
memset(&new_act, 0, sizeof(new_act));
2021-10-02 13:05:19 +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
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
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
{
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;
const Dialect & dialect = settings.dialect;
if (dialect == Dialect::kusto)
parser = std::make_unique<ParserKQLStatement>(end, settings.allow_settings_after_format_in_insert);
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);
else
parser = std::make_unique<ParserQuery>(end, settings.allow_settings_after_format_in_insert);
2021-07-12 09:30:16 +00:00
if (is_interactive || ignore_error)
{
String message;
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);
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);
if (!res)
{
2024-06-25 14:23:37 +00:00
error_stream << std::endl << message << std::endl << std::endl;
return nullptr;
2021-07-12 09:30:16 +00:00
}
}
else
{
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);
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;
}
/// 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())
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));
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;
/// 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
try
{
2023-10-21 01:14:22 +00:00
output_format->write(materializeBlock(block));
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());
}
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)
{
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-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);
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);
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);
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)
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");
/// 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");
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
select_into_file = false;
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)
{
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 &>();
compression_level_node.value.tryGet<UInt64>(compression_level);
2021-09-16 18:44:42 +00:00
}
2023-04-18 07:14:35 +00:00
auto flags = O_WRONLY | O_EXCL;
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;
else if (file_exists && query_with_output->is_outfile_truncate)
flags |= O_TRUNC;
2023-04-18 07:14:35 +00:00
else
flags |= O_CREAT;
out_file_buf = wrapWriteBufferWithCompressionMethod(
2023-04-18 07:14:35 +00:00
std::make_unique<WriteBufferFromFile>(out_file, DBMS_DEFAULT_BUFFER_SIZE, flags),
compression_method,
static_cast<int>(compression_level)
);
if (query_with_output->is_into_outfile_with_stdout)
{
select_into_file_and_stdout = true;
out_file_buf = std::make_unique<ForkWriteBuffer>(std::vector<WriteBufferPtr>{std::move(out_file_buf),
std::make_shared<WriteBufferFromFileDescriptor>(STDOUT_FILENO)});
}
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)
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();
}
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";
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;
if (!out_file_buf && default_output_compression_method != CompressionMethod::None)
out_file_buf = wrapWriteBufferWithCompressionMethod(out_buf, default_output_compression_method, 3, 0);
/// It is not clear how to write progress and logs
/// intermixed with data with parallel formatting.
/// It may increase code complexity significantly.
if (!extras_into_stdout || select_only_into_file)
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
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
}
}
catch (...)
{
2023-01-23 13:16:14 +00:00
throw LocalFormatError(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode());
}
2021-08-17 19:59:51 +00:00
void ClientBase::initLogsOutputStream()
{
if (!logs_out_stream)
{
WriteBuffer * wb = out_logs_buf.get();
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();
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;
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();
}
}
logs_out_stream = std::make_unique<InternalTextLogs>(*wb, color_logs);
2021-08-17 19:59:51 +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;
}
/// 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)
{
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;
}
}
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);
}
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";
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
}
void ClientBase::updateSuggest(const ASTPtr & ast)
{
std::vector<std::string> new_words;
if (auto * create = ast->as<ASTCreateQuery>())
{
if (create->database)
new_words.push_back(create->getDatabase());
new_words.push_back(create->getTable());
if (create->columns_list && create->columns_list->columns)
{
for (const auto & elem : create->columns_list->columns->children)
{
if (const auto * column = elem->as<ASTColumnDeclaration>())
new_words.push_back(column->name);
}
}
}
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-08-17 19:59:51 +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();
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;
/// 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.
///
/// Do not update suggest, until suggestion will be ready
/// (this will avoid extra complexity)
if (suggest)
updateSuggest(parsed_query);
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).
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
{
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-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.
/// 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
}
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);
}
}
// 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:47:28 +00:00
String out_file;
if (query_with_output->out_file)
{
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: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:47:28 +00:00
CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string);
UInt64 compression_level = 3;
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: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: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:47:28 +00:00
if (!res || compression_level < range.first || compression_level > range.second)
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: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:47:28 +00:00
throw Exception(
ErrorCodes::FILE_ALREADY_EXISTS,
"File {} exists, consider using APPEND or TRUNCATE.",
out_file);
}
}
}
}
const auto & settings = global_context->getSettingsRef();
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
{
QueryInterruptHandler::start(signals_before_stop);
SCOPE_EXIT({ QueryInterruptHandler::stop(); });
2021-08-17 19:59:51 +00:00
connection->sendQuery(
connection_parameters.timeouts,
query,
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);
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.
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;
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)
{
if (partial_result_on_first_cancel && QueryInterruptHandler::cancelled_status() == signals_before_stop - 1)
{
connection->sendCancel();
/// First cancel reading request was sent. Next requests will only be with a full cancel
partial_result_on_first_cancel = false;
}
else if (QueryInterruptHandler::cancelled())
2021-08-17 19:59:51 +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;
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;
}
try
{
if (!receiveAndProcessPacket(parsed_query, cancelled))
break;
}
catch (const LocalFormatError &)
{
/// Remember the first exception.
if (!local_format_error)
local_format_error = std::current_exception();
connection->sendCancel();
}
2021-08-17 19:59: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.
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:
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:
if (!cancelled_)
2021-08-17 19:59:51 +00:00
onTotals(packet.block, parsed_query);
return true;
case Protocol::Server::Extremes:
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;
case Protocol::Server::TimezoneUpdate:
onTimezoneUpdate(packet.server_timezone);
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
}
void ClientBase::onTimezoneUpdate(const String & tz)
{
2023-05-10 23:10:34 +00:00
global_context->setSetting("session_timezone", tz);
}
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)
{
/// 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);
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;
}
}
2021-08-03 08:33:54 +00:00
resetOutput();
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;
else if (!written_first_block)
2024-06-25 14:23:37 +00:00
output_stream << "Ok." << std::endl;
}
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();
if (rows == 0)
2021-09-14 11:06:00 +00:00
return;
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
{
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);
const auto & array_values = typeid_cast<const ColumnInt64 &>(*block.getByName("value").column).getData();
const auto * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds);
const auto * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds);
HostToTimesMap thread_times;
for (size_t i = 0; i < rows; ++i)
{
auto thread_id = array_thread_id[i];
auto host_name = host_names.getDataAt(i).toString();
/// 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;
auto event_name = names.getDataAt(i);
auto value = array_values[i];
/// Ignore negative time delta or memory usage just in case.
if (value < 0)
continue;
if (event_name == user_time_name)
thread_times[host_name].user_ms = value;
else if (event_name == system_time_name)
thread_times[host_name].system_ms = value;
else if (event_name == MemoryTracker::USAGE_EVENT_NAME)
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;
}
2022-07-15 22:27:26 +00:00
progress_indication.updateThreadEventData(thread_times);
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-12-29 16:57:40 +00:00
if (profile_events.print)
{
2021-12-29 16:57:40 +00:00
if (profile_events.watch.elapsedMilliseconds() >= profile_events.delay_ms)
{
/// 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-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
{
/// 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();
2021-07-12 09:30:16 +00:00
logs_out_stream.reset();
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();
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");
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");
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");
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);
case Protocol::Server::TimezoneUpdate:
onTimezoneUpdate(packet.server_timezone);
break;
2021-08-18 14:39:04 +00:00
default:
throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER,
"Unexpected packet from server (expected Data, Exception, Log or TimezoneUpdate, got {})",
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));
}
}
}
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;
}
}
}
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;
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.
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)
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
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,
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);
2021-08-18 14:39:04 +00:00
sendData(sample, columns_description, parsed_query);
receiveEndOfQuery();
}
}
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())
{
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
bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && isStdinNotEmptyAndValid(std_in);
2022-05-06 15:04:03 +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)
if (parsed_insert_query->infile)
{
2021-09-16 18:44:42 +00:00
/// Get name of this file (path to file)
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())
current_format = FormatFactory::instance().getFormatFromFileName(in_file);
2022-01-07 05:16:41 +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));
}
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),
compression_method,
columns_for_storage_file,
ConstraintsDescription{},
String{},
{},
2023-07-31 12:04:27 +00:00
String{},
};
StoragePtr storage = std::make_shared<StorageFile>(in_file, global_context->getUserFilesPath(), args);
storage->startup();
SelectQueryInfo query_info;
2021-08-18 14:39:04 +00:00
try
{
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);
sendDataFromPipe(
2022-05-23 19:47:32 +00:00
std::move(pipe),
parsed_query,
2023-05-07 04:16:30 +00:00
have_data_in_stdin);
}
catch (Exception & e)
{
e.addMessage("data for INSERT was parsed from file");
throw;
}
2023-03-02 18:51:05 +00:00
if (have_data_in_stdin && !cancelled)
sendDataFromStdin(sample, columns_description_for_query, parsed_query);
}
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
{
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)
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)
{
sendDataFromStdin(sample, columns_description_for_query, parsed_query);
2021-08-18 14:39:04 +00:00
}
else
throw Exception(ErrorCodes::NO_DATA_TO_INSERT, "No data to insert");
2021-08-18 14:39:04 +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);
});
}
sendDataFromPipe(std::move(pipe), parsed_query, have_more_data);
}
2022-04-19 12:21:14 +00:00
void ClientBase::sendDataFromPipe(Pipe&& pipe, ASTPtr parsed_query, bool have_more_data)
try
{
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-09-05 15:42:49 +00:00
cancelQuery();
executor.cancel();
return;
}
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-09-05 15:42:49 +00:00
if (!have_more_data)
connection->sendData({}, "", false);
}
catch (...)
{
connection->sendCancel();
receiveEndOfQuery();
throw;
}
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
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 ClientBase::receiveLogsAndProfileEvents(ASTPtr parsed_query)
2021-08-18 14:39:04 +00:00
{
auto packet_type = connection->checkPacket(0);
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);
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);
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;
case Protocol::Server::TimezoneUpdate:
onTimezoneUpdate(packet.server_timezone);
break;
2021-08-18 14:39:04 +00:00
default:
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
}
}
}
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);
if (is_interactive)
2024-06-25 14:23:37 +00:00
output_stream << "Cancelling query." << std::endl;
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,
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;
cancelled = false;
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);
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)
{
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>())
{
if (!create_user_query->attach && create_user_query->auth_data)
2022-11-22 02:41:23 +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();
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);
global_context->resetSettingsToDefaultValue(settings_ast.as<ASTSetQuery>()->default_settings);
2021-09-19 21:42:28 +00:00
};
const auto * insert = parsed_query->as<ASTInsertQuery>();
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);
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);
bool is_async_insert_with_inlined_data = global_context->getSettingsRef().async_insert && insert && insert->hasInlinedData();
if (is_async_insert_with_inlined_data)
{
bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && isStdinNotEmptyAndValid(std_in);
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");
}
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())
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);
}
global_context->resetSettingsToDefaultValue(set_query->default_settings);
/// 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);
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
/// 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);
logs_out_stream->writeProfileEvents(profile_events.last_block);
logs_out_stream->flush();
profile_events.last_block = {};
}
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))
{
2024-06-25 14:23:37 +00:00
output_stream << "Processed rows: " << processed_rows << "\n";
}
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
{
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
// 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;
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);
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
{
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
}
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
// inserted data ends. In multi-query mode, it is delimited by a
2021-08-20 14:10:59 +00:00
// newline.
// 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>();
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;
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
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'");
}
/// 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
current_exception->addMessage("\nExpected 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())
current_exception->addMessage("\nExpected 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",
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",
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",
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",
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",
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",
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;
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;
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
{
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
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 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
}
String ClientBase::prompt() const
{
return prompt_by_server_display_name;
}
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;
}
void ClientBase::runInteractive()
{
2024-06-25 14:23:37 +00:00
if (getClientConfiguration().has("query_id"))
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))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "time option could be specified only in non-interactive mode");
2022-02-02 15:19:33 +00:00
initQueryIdFormats();
/// 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();
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
}
if (home_path.empty())
{
const char * home_path_cstr = getenv("HOME"); // NOLINT(concurrency-mt-unsafe)
if (home_path_cstr)
home_path = home_path_cstr;
2021-07-12 14:01:46 +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");
else
{
auto * history_file_from_env = getenv("CLICKHOUSE_HISTORY_FILE"); // NOLINT(concurrency-mt-unsafe)
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
}
}
}
LineReader::Patterns query_extenders = {"\\"};
2022-04-11 12:46:24 +00:00
LineReader::Patterns query_delimiters = {";", "\\G", "\\G;"};
char word_break_characters[] = " \t\v\f\a\b\r\n`~!@#$%^&*()-=+[{]}\\|;:'\",<.>/?";
#if USE_REPLXX
replxx::Replxx::highlighter_callback_t highlight_callback{};
2024-06-25 14:23:37 +00:00
if (getClientConfiguration().getBool("highlight", true))
highlight_callback = highlight;
ReplxxLineReader lr(
*suggest,
history_file,
2024-06-25 14:23:37 +00:00
getClientConfiguration().has("multiline"),
query_extenders,
query_delimiters,
word_break_characters,
highlight_callback);
#else
LineReader lr(
history_file,
2024-06-25 14:23:37 +00:00
getClientConfiguration().has("multiline"),
query_extenders,
query_delimiters,
word_break_characters);
#endif
static const std::initializer_list<std::pair<String, String>> backslash_aliases =
{
{ "\\l", "SHOW DATABASES" },
{ "\\d", "SHOW TABLES" },
{ "\\c", "USE" },
};
static const std::initializer_list<String> repeat_last_input_aliases =
{
".", /// Vim shortcut
"/" /// Oracle SQL Plus shortcut
};
String last_input;
do
{
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(), ":-] ");
}
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;"))
{
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
has_vertical_output_suffix = true;
}
for (const auto & [alias, command] : backslash_aliases)
{
auto it = std::search(input.begin(), input.end(), alias.begin(), alias.end());
if (it != input.end() && std::all_of(input.begin(), it, isWhitespaceASCII))
{
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;
}
}
}
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
}
try
{
if (!processQueryText(input))
break;
last_input = input;
}
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
/// 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());
}
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.
if (!connection->checkConnected(connection_parameters.timeouts))
connect();
}
}
while (true);
if (isNewYearMode())
2024-06-25 14:23:37 +00:00
output_stream << "Happy new year." << std::endl;
else if (isChineseNewYearMode(local_tz))
2024-06-25 14:23:37 +00:00
output_stream << "Happy Chinese new year. 春节快乐!" << std::endl;
else
2024-06-25 14:23:37 +00:00
output_stream << "Bye." << std::endl;
}
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);
}
return executeMultiQuery(queries_from_file);
}
2021-07-22 21:27:26 +00:00
void ClientBase::runNonInteractive()
{
2022-02-02 15:19:33 +00:00
if (delayed_interactive)
initQueryIdFormats();
if (!queries_files.empty())
{
for (const auto & queries_file : queries_files)
{
for (const auto & interleave_file : interleave_queries_files)
if (!processMultiQueryFromFile(interleave_file))
return;
if (!processMultiQueryFromFile(queries_file))
return;
}
return;
}
if (!queries.empty())
{
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;
}
}
}
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);
String text;
readStringUntilEOF(text, in);
2023-09-04 11:40:56 +00:00
if (query_fuzzer_runs)
processWithFuzzing(text);
else
processQueryText(text);
}
}
#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();
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() {}
#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
}
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);
}
};
/*
* 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
{
2024-05-10 04:58:21 +00:00
if (!token.starts_with("--"))
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;
};
}
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
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)
{
/// Add merge tree settings manually, because names of some settings
/// may clash. Query settings have higher priority and we just
/// skip ambiguous merge tree settings.
auto & main_options = options_description.main_description.value();
2022-12-16 17:50:01 +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());
for (const auto & setting : cmd_merge_tree_settings.all())
{
const auto add_setting = [&](const std::string_view name)
{
if (auto it = main_option_names.find(name); it != main_option_names.end())
return;
if (allow_repeated_settings)
2024-03-19 16:04:29 +00:00
addProgramOptionAsMultitoken(cmd_merge_tree_settings, main_options, name, setting);
else
2024-03-19 16:04:29 +00:00
addProgramOption(cmd_merge_tree_settings, main_options, name, setting);
};
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-07 22:16:01 +00:00
/// Parse main commandline options.
auto parser = po::command_line_parser(arguments)
.options(options_description.main_description.value())
.extra_parser(OptionsAliasParser(options_description.main_description.value()))
.allow_unregistered();
po::parsed_options parsed = parser.run();
/// Check unrecognized options without positional options.
auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::exclude_positional);
if (!unrecognized_options.empty())
{
auto hints = this->getHints(unrecognized_options[0]);
if (!hints.empty())
throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'. Maybe you meant {}",
unrecognized_options[0], toString(hints));
2021-12-16 12:31:06 +00:00
throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]);
}
2023-07-27 17:52:44 +00:00
/// Check positional options.
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
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);
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.
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];
readArguments(argc, argv, common_arguments, external_tables_arguments, hosts_and_ports_arguments);
2021-07-11 11:36:27 +00:00
/// Support for Unicode dashes
2023-04-22 21:46:06 +00:00
/// Interpret Unicode dashes as default double-hyphen
for (auto & arg : common_arguments)
{
2023-04-22 21:46:06 +00:00
// replace em-dash(U+2014)
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, "", "--");
}
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()
("help", "print usage summary, combine with --verbose to display all options")
("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-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.")
("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")
("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.")
("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")
("hardware-utilization", "print hardware utilization information in progress bar")
("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")
("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
;
addOptions(options_description);
OptionsDescription options_description_non_verbose = options_description;
2021-12-20 15:09:20 +00:00
auto getter = [](const auto & op)
{
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);
}
2023-09-04 14:48:38 +00:00
po::variables_map options;
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();
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;
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))
printHelpMessage(options_description, true);
2024-04-25 17:46:20 +00:00
else
printHelpMessage(options_description_non_verbose, false);
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"))
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);
if (options.count("print-profile-events"))
2024-06-25 14:23:37 +00:00
getClientConfiguration().setBool("print-profile-events", true);
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"))
{
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;
case OFF:
2024-06-25 14:23:37 +00:00
getClientConfiguration().setString("progress", "off");
break;
case TTY:
2024-06-25 14:23:37 +00:00
getClientConfiguration().setString("progress", "tty");
break;
case ERR:
2024-06-25 14:23:37 +00:00
getClientConfiguration().setString("progress", "err");
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);
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);
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
query_processing_stage = QueryProcessingStage::fromString(options["stage"].as<std::string>());
query_kind = parseQueryKind(options["query_kind"].as<std::string>());
profile_events.print = options.count("print-profile-events");
profile_events.delay_ms = options["profile-events-delay-ms"].as<UInt64>();
processOptions(options_description, options, external_tables_arguments, hosts_and_ports_arguments);
{
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);
}
2021-07-28 12:56:11 +00:00
clearPasswordFromCommandLine(argc, argv);
/// 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")
{
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);
}
/// Print stacktrace in case of crash
HandledSignals::instance().setupTerminateHandler();
HandledSignals::instance().setupCommonDeadlySignalHandlers();
/// We don't setup signal handlers for SIGINT, SIGQUIT, SIGTERM because we don't
/// have an option for client to shutdown gracefully.
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);
signal_listener = std::make_unique<SignalListener>(nullptr, fatal_log);
signal_listener_thread.start(*signal_listener);
2021-07-11 11:36:27 +00:00
}
}