2021-08-31 13:50:56 +00:00
|
|
|
#include <algorithm>
|
2023-09-26 15:50:19 +00:00
|
|
|
#include <exception>
|
2021-09-02 14:27:19 +00:00
|
|
|
#include <memory>
|
2021-08-31 13:50:56 +00:00
|
|
|
#include <mutex>
|
2024-10-22 12:26:06 +00:00
|
|
|
#include <optional>
|
2021-10-31 15:11:46 +00:00
|
|
|
#include <string_view>
|
2024-07-22 23:14:19 +00:00
|
|
|
#include <vector>
|
|
|
|
#include <Access/AccessControl.h>
|
|
|
|
#include <Access/Credentials.h>
|
2018-12-28 18:15:26 +00:00
|
|
|
#include <Compression/CompressedReadBuffer.h>
|
|
|
|
#include <Compression/CompressedWriteBuffer.h>
|
2024-07-22 23:14:19 +00:00
|
|
|
#include <Compression/CompressionFactory.h>
|
|
|
|
#include <Core/ExternalTable.h>
|
|
|
|
#include <Core/ServerSettings.h>
|
|
|
|
#include <Formats/NativeReader.h>
|
|
|
|
#include <Formats/NativeWriter.h>
|
2020-12-02 21:05:51 +00:00
|
|
|
#include <IO/LimitReadBuffer.h>
|
2024-07-22 23:14:19 +00:00
|
|
|
#include <IO/Progress.h>
|
|
|
|
#include <IO/ReadBufferFromPocoSocket.h>
|
2018-09-20 20:51:21 +00:00
|
|
|
#include <IO/ReadHelpers.h>
|
2024-07-22 23:14:19 +00:00
|
|
|
#include <IO/WriteBufferFromPocoSocket.h>
|
2018-09-20 20:51:21 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2024-09-26 08:27:37 +00:00
|
|
|
#include <IO/WriteBuffer.h>
|
2024-07-22 23:14:19 +00:00
|
|
|
#include <Interpreters/AsynchronousInsertQueue.h>
|
2018-08-14 20:29:42 +00:00
|
|
|
#include <Interpreters/InternalTextLogsQueue.h>
|
2020-11-09 15:07:38 +00:00
|
|
|
#include <Interpreters/OpenTelemetrySpanLog.h>
|
2021-08-01 14:12:34 +00:00
|
|
|
#include <Interpreters/Session.h>
|
2024-07-22 23:14:19 +00:00
|
|
|
#include <Interpreters/Squashing.h>
|
|
|
|
#include <Interpreters/TablesStatus.h>
|
|
|
|
#include <Interpreters/executeQuery.h>
|
|
|
|
#include <Parsers/ASTInsertQuery.h>
|
2021-10-22 07:15:34 +00:00
|
|
|
#include <Server/TCPServer.h>
|
2020-11-20 17:23:53 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
|
2024-02-09 17:55:21 +00:00
|
|
|
#include <Storages/ObjectStorage/StorageObjectStorageCluster.h>
|
2024-07-22 23:14:19 +00:00
|
|
|
#include <Storages/StorageReplicatedMergeTree.h>
|
|
|
|
#include <Poco/Net/NetException.h>
|
|
|
|
#include <Poco/Net/SocketAddress.h>
|
|
|
|
#include <Poco/Util/LayeredConfiguration.h>
|
2024-10-03 14:23:57 +00:00
|
|
|
#include "Common/StackTrace.h"
|
2024-09-26 08:27:37 +00:00
|
|
|
#include <Common/Exception.h>
|
2021-10-07 08:26:08 +00:00
|
|
|
#include <Common/CurrentMetrics.h>
|
2024-07-22 23:14:19 +00:00
|
|
|
#include <Common/CurrentThread.h>
|
|
|
|
#include <Common/NetException.h>
|
|
|
|
#include <Common/OpenSSLHelpers.h>
|
|
|
|
#include <Common/Stopwatch.h>
|
|
|
|
#include <Common/logger_useful.h>
|
|
|
|
#include <Common/scope_guard_safe.h>
|
|
|
|
#include <Common/setThreadName.h>
|
2023-03-03 13:52:54 +00:00
|
|
|
#include <Common/thread_local_rng.h>
|
2024-10-22 12:26:06 +00:00
|
|
|
#include "Core/Settings.h"
|
|
|
|
#include "base/defines.h"
|
|
|
|
#include "base/scope_guard.h"
|
2018-07-09 16:31:24 +00:00
|
|
|
|
2020-05-20 19:01:36 +00:00
|
|
|
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
2021-09-02 11:29:20 +00:00
|
|
|
#include <Processors/Executors/PushingPipelineExecutor.h>
|
2021-09-10 14:52:24 +00:00
|
|
|
#include <Processors/Executors/PushingAsyncPipelineExecutor.h>
|
2021-09-15 19:35:48 +00:00
|
|
|
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
2021-09-03 17:29:36 +00:00
|
|
|
#include <Processors/Sinks/SinkToStorage.h>
|
2019-03-26 18:28:37 +00:00
|
|
|
|
2023-03-15 00:09:29 +00:00
|
|
|
#if USE_SSL
|
|
|
|
# include <Poco/Net/SecureStreamSocket.h>
|
|
|
|
# include <Poco/Net/SecureStreamSocketImpl.h>
|
|
|
|
#endif
|
|
|
|
|
2023-11-13 09:09:23 +00:00
|
|
|
#include <Core/Protocol.h>
|
|
|
|
#include <Storages/MergeTree/RequestResponse.h>
|
2012-03-09 15:46:52 +00:00
|
|
|
#include "TCPHandler.h"
|
|
|
|
|
2023-11-13 09:09:23 +00:00
|
|
|
#include <Common/config_version.h>
|
2020-04-16 12:31:57 +00:00
|
|
|
|
2024-07-22 23:14:19 +00:00
|
|
|
#include <fmt/format.h>
|
|
|
|
|
2021-10-31 15:11:46 +00:00
|
|
|
using namespace std::literals;
|
2022-08-12 12:28:35 +00:00
|
|
|
using namespace DB;
|
2021-10-31 15:11:46 +00:00
|
|
|
|
2024-09-18 12:20:53 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
namespace Setting
|
|
|
|
{
|
|
|
|
extern const SettingsBool allow_experimental_analyzer;
|
|
|
|
extern const SettingsBool allow_experimental_codecs;
|
|
|
|
extern const SettingsBool allow_experimental_query_deduplication;
|
|
|
|
extern const SettingsBool allow_suspicious_codecs;
|
|
|
|
extern const SettingsBool async_insert;
|
|
|
|
extern const SettingsUInt64 async_insert_max_data_size;
|
|
|
|
extern const SettingsBool calculate_text_stack_trace;
|
|
|
|
extern const SettingsBool deduplicate_blocks_in_dependent_materialized_views;
|
|
|
|
extern const SettingsBool enable_zstd_qat_codec;
|
|
|
|
extern const SettingsUInt64 idle_connection_timeout;
|
|
|
|
extern const SettingsBool input_format_defaults_for_omitted_fields;
|
|
|
|
extern const SettingsUInt64 interactive_delay;
|
|
|
|
extern const SettingsBool low_cardinality_allow_in_native_format;
|
|
|
|
extern const SettingsString network_compression_method;
|
|
|
|
extern const SettingsInt64 network_zstd_compression_level;
|
|
|
|
extern const SettingsBool partial_result_on_first_cancel;
|
|
|
|
extern const SettingsUInt64 poll_interval;
|
|
|
|
extern const SettingsSeconds receive_timeout;
|
|
|
|
extern const SettingsLogsLevel send_logs_level;
|
|
|
|
extern const SettingsString send_logs_source_regexp;
|
|
|
|
extern const SettingsSeconds send_timeout;
|
|
|
|
extern const SettingsTimezone session_timezone;
|
|
|
|
extern const SettingsMilliseconds sleep_after_receiving_query_ms;
|
|
|
|
extern const SettingsMilliseconds sleep_in_send_data_ms;
|
|
|
|
extern const SettingsMilliseconds sleep_in_send_tables_status_ms;
|
|
|
|
extern const SettingsBool throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert;
|
|
|
|
extern const SettingsUInt64 unknown_packet_in_send_data;
|
|
|
|
extern const SettingsBool wait_for_async_insert;
|
|
|
|
extern const SettingsSeconds wait_for_async_insert_timeout;
|
2024-09-27 06:10:50 +00:00
|
|
|
extern const SettingsBool use_concurrency_control;
|
2024-09-18 12:20:53 +00:00
|
|
|
}
|
2024-10-16 19:13:26 +00:00
|
|
|
|
|
|
|
namespace ServerSetting
|
|
|
|
{
|
|
|
|
extern const ServerSettingsBool validate_tcp_client_information;
|
|
|
|
}
|
2024-09-18 12:20:53 +00:00
|
|
|
}
|
2021-10-31 15:11:46 +00:00
|
|
|
|
2021-10-07 08:26:08 +00:00
|
|
|
namespace CurrentMetrics
|
|
|
|
{
|
|
|
|
extern const Metric QueryThread;
|
2023-02-12 09:04:05 +00:00
|
|
|
extern const Metric ReadTaskRequestsSent;
|
|
|
|
extern const Metric MergeTreeReadTaskRequestsSent;
|
|
|
|
extern const Metric MergeTreeAllRangesAnnouncementsSent;
|
|
|
|
}
|
|
|
|
|
|
|
|
namespace ProfileEvents
|
|
|
|
{
|
|
|
|
extern const Event ReadTaskRequestsSent;
|
|
|
|
extern const Event MergeTreeReadTaskRequestsSent;
|
|
|
|
extern const Event MergeTreeAllRangesAnnouncementsSent;
|
|
|
|
extern const Event ReadTaskRequestsSentElapsedMicroseconds;
|
|
|
|
extern const Event MergeTreeReadTaskRequestsSentElapsedMicroseconds;
|
|
|
|
extern const Event MergeTreeAllRangesAnnouncementsSentElapsedMicroseconds;
|
2021-10-07 08:26:08 +00:00
|
|
|
}
|
2018-02-16 20:53:47 +00:00
|
|
|
|
2023-08-22 03:52:57 +00:00
|
|
|
namespace DB::ErrorCodes
|
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
extern const int ABORTED;
|
2023-08-22 03:52:57 +00:00
|
|
|
extern const int ATTEMPT_TO_READ_AFTER_EOF;
|
2024-10-22 12:26:06 +00:00
|
|
|
extern const int AUTHENTICATION_FAILED;
|
2023-08-22 03:52:57 +00:00
|
|
|
extern const int CLIENT_HAS_CONNECTED_TO_WRONG_PORT;
|
2024-10-22 12:26:06 +00:00
|
|
|
extern const int CLIENT_INFO_DOES_NOT_MATCH;
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int NETWORK_ERROR;
|
2023-08-22 03:52:57 +00:00
|
|
|
extern const int SOCKET_TIMEOUT;
|
2024-10-22 12:26:06 +00:00
|
|
|
extern const int SUPPORT_IS_DISABLED;
|
|
|
|
extern const int TIMEOUT_EXCEEDED;
|
2023-08-22 03:52:57 +00:00
|
|
|
extern const int UNEXPECTED_PACKET_FROM_CLIENT;
|
2024-10-22 12:26:06 +00:00
|
|
|
extern const int UNKNOWN_EXCEPTION;
|
|
|
|
extern const int UNKNOWN_PACKET_FROM_CLIENT;
|
2023-08-22 03:52:57 +00:00
|
|
|
extern const int UNKNOWN_PROTOCOL;
|
2023-09-26 15:50:19 +00:00
|
|
|
extern const int UNSUPPORTED_METHOD;
|
2024-04-26 15:03:40 +00:00
|
|
|
extern const int USER_EXPIRED;
|
2024-10-24 16:50:22 +00:00
|
|
|
|
|
|
|
// We have to distinguish the case when query is killed by `KILL QUERY` statement
|
|
|
|
// and when it is killed by `Protocol::Client::Cancel` packet.
|
|
|
|
|
|
|
|
// When query is illed by `KILL QUERY` statement we have to end the execution
|
|
|
|
// and send the exception to the actual client which initiated the TCP connection.
|
|
|
|
|
|
|
|
// When query is killed by `Protocol::Client::Cancel` packet we just stop execution,
|
|
|
|
// there is no need to send the exception which has been caused by the cancel packet.
|
|
|
|
extern const int QUERY_WAS_CANCELLED_BY_CLIENT;
|
2023-08-22 03:52:57 +00:00
|
|
|
}
|
|
|
|
|
2022-08-12 12:28:35 +00:00
|
|
|
namespace
|
|
|
|
{
|
2023-08-30 06:14:39 +00:00
|
|
|
// This function corrects the wrong client_name from the old client.
|
2023-08-29 23:50:41 +00:00
|
|
|
// Old clients 28.7 and some intermediate versions of 28.7 were sending different ClientInfo.client_name
|
|
|
|
// "ClickHouse client" was sent with the hello message.
|
|
|
|
// "ClickHouse" or "ClickHouse " was sent with the query message.
|
2023-08-30 06:14:39 +00:00
|
|
|
void correctQueryClientInfo(const ClientInfo & session_client_info, ClientInfo & client_info)
|
2023-08-29 23:50:41 +00:00
|
|
|
{
|
|
|
|
if (client_info.getVersionNumber() <= VersionNumber(23, 8, 1) &&
|
2023-08-30 06:14:39 +00:00
|
|
|
session_client_info.client_name == "ClickHouse client" &&
|
2023-08-29 23:50:41 +00:00
|
|
|
(client_info.client_name == "ClickHouse" || client_info.client_name == "ClickHouse "))
|
|
|
|
{
|
|
|
|
client_info.client_name = "ClickHouse client";
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-08-22 03:52:57 +00:00
|
|
|
void validateClientInfo(const ClientInfo & session_client_info, const ClientInfo & client_info)
|
|
|
|
{
|
|
|
|
// Secondary query may contain different client_info.
|
|
|
|
// In the case of select from distributed table or 'select * from remote' from non-tcp handler. Server sends the initial client_info data.
|
|
|
|
//
|
|
|
|
// Example 1: curl -q -s --max-time 60 -sS "http://127.0.0.1:8123/?" -d "SELECT 1 FROM remote('127.0.0.1', system.one)"
|
|
|
|
// HTTP handler initiates TCP connection with remote 127.0.0.1 (session on remote 127.0.0.1 use TCP interface)
|
|
|
|
// HTTP handler sends client_info with HTTP interface and HTTP data by TCP protocol in Protocol::Client::Query message.
|
|
|
|
//
|
|
|
|
// Example 2: select * from <distributed_table> --host shard_1 // distributed table has 2 shards: shard_1, shard_2
|
|
|
|
// shard_1 receives a message with 'ClickHouse client' client_name
|
|
|
|
// shard_1 initiates TCP connection with shard_2 with 'ClickHouse server' client_name.
|
|
|
|
// shard_1 sends 'ClickHouse client' client_name in Protocol::Client::Query message to shard_2.
|
|
|
|
if (client_info.query_kind == ClientInfo::QueryKind::SECONDARY_QUERY)
|
|
|
|
return;
|
|
|
|
|
|
|
|
if (session_client_info.interface != client_info.interface)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
DB::ErrorCodes::CLIENT_INFO_DOES_NOT_MATCH,
|
|
|
|
"Client info's interface does not match: {} not equal to {}",
|
|
|
|
toString(session_client_info.interface),
|
|
|
|
toString(client_info.interface));
|
|
|
|
}
|
|
|
|
|
|
|
|
if (session_client_info.interface == ClientInfo::Interface::TCP)
|
|
|
|
{
|
|
|
|
if (session_client_info.client_name != client_info.client_name)
|
|
|
|
throw Exception(
|
|
|
|
DB::ErrorCodes::CLIENT_INFO_DOES_NOT_MATCH,
|
|
|
|
"Client info's client_name does not match: {} not equal to {}",
|
|
|
|
session_client_info.client_name,
|
|
|
|
client_info.client_name);
|
|
|
|
|
|
|
|
// TCP handler got patch version 0 always for backward compatibility.
|
|
|
|
if (!session_client_info.clientVersionEquals(client_info, false))
|
|
|
|
throw Exception(
|
|
|
|
DB::ErrorCodes::CLIENT_INFO_DOES_NOT_MATCH,
|
|
|
|
"Client info's version does not match: {} not equal to {}",
|
|
|
|
session_client_info.getVersionStr(),
|
|
|
|
client_info.getVersionStr());
|
|
|
|
|
|
|
|
// os_user, quota_key, client_trace_context can be different.
|
|
|
|
}
|
|
|
|
}
|
2024-10-22 12:26:06 +00:00
|
|
|
struct TurnOffBoolSettingTemporary
|
|
|
|
{
|
|
|
|
bool & setting;
|
|
|
|
bool prev_val;
|
|
|
|
|
|
|
|
explicit TurnOffBoolSettingTemporary(bool & setting_)
|
|
|
|
: setting(setting_)
|
|
|
|
, prev_val(setting_)
|
|
|
|
{
|
|
|
|
if (prev_val)
|
|
|
|
setting = false;
|
|
|
|
}
|
|
|
|
|
|
|
|
~TurnOffBoolSettingTemporary()
|
|
|
|
{
|
|
|
|
if (prev_val)
|
|
|
|
setting = true;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2022-08-12 12:28:35 +00:00
|
|
|
}
|
|
|
|
|
2012-03-09 15:46:52 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2024-02-02 14:24:44 +00:00
|
|
|
TCPHandler::TCPHandler(
|
|
|
|
IServer & server_,
|
|
|
|
TCPServer & tcp_server_,
|
|
|
|
const Poco::Net::StreamSocket & socket_,
|
|
|
|
bool parse_proxy_protocol_,
|
|
|
|
std::string server_display_name_,
|
|
|
|
std::string host_name_,
|
|
|
|
const ProfileEvents::Event & read_event_,
|
|
|
|
const ProfileEvents::Event & write_event_)
|
2021-01-28 04:04:07 +00:00
|
|
|
: Poco::Net::TCPServerConnection(socket_)
|
|
|
|
, server(server_)
|
2021-10-22 07:15:34 +00:00
|
|
|
, tcp_server(tcp_server_)
|
2021-01-28 04:04:07 +00:00
|
|
|
, parse_proxy_protocol(parse_proxy_protocol_)
|
2024-01-23 17:04:50 +00:00
|
|
|
, log(getLogger("TCPHandler"))
|
2024-01-03 16:47:15 +00:00
|
|
|
, read_event(read_event_)
|
|
|
|
, write_event(write_event_)
|
2021-01-28 04:04:07 +00:00
|
|
|
, server_display_name(std::move(server_display_name_))
|
2024-02-02 14:24:44 +00:00
|
|
|
, host_name(std::move(host_name_))
|
2021-01-28 04:04:07 +00:00
|
|
|
{
|
|
|
|
}
|
2021-03-05 14:57:16 +00:00
|
|
|
|
2024-02-02 14:24:44 +00:00
|
|
|
TCPHandler::TCPHandler(
|
|
|
|
IServer & server_,
|
|
|
|
TCPServer & tcp_server_,
|
|
|
|
const Poco::Net::StreamSocket & socket_,
|
|
|
|
TCPProtocolStackData & stack_data,
|
|
|
|
std::string server_display_name_,
|
|
|
|
std::string host_name_,
|
|
|
|
const ProfileEvents::Event & read_event_,
|
|
|
|
const ProfileEvents::Event & write_event_)
|
|
|
|
: Poco::Net::TCPServerConnection(socket_)
|
2022-09-10 20:21:37 +00:00
|
|
|
, server(server_)
|
|
|
|
, tcp_server(tcp_server_)
|
2024-01-23 17:04:50 +00:00
|
|
|
, log(getLogger("TCPHandler"))
|
2022-09-10 20:21:37 +00:00
|
|
|
, forwarded_for(stack_data.forwarded_for)
|
2022-09-18 07:11:52 +00:00
|
|
|
, certificate(stack_data.certificate)
|
2024-01-03 16:47:15 +00:00
|
|
|
, read_event(read_event_)
|
|
|
|
, write_event(write_event_)
|
2022-09-19 02:01:09 +00:00
|
|
|
, default_database(stack_data.default_database)
|
2022-09-10 20:21:37 +00:00
|
|
|
, server_display_name(std::move(server_display_name_))
|
2024-02-02 14:24:44 +00:00
|
|
|
, host_name(std::move(host_name_))
|
2022-09-10 20:21:37 +00:00
|
|
|
{
|
2022-12-11 23:39:16 +00:00
|
|
|
if (!forwarded_for.empty())
|
|
|
|
LOG_TRACE(log, "Forwarded client address: {}", forwarded_for);
|
2022-09-10 20:21:37 +00:00
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2024-09-26 08:27:37 +00:00
|
|
|
TCPHandler::~TCPHandler() = default;
|
2012-03-09 15:46:52 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2012-03-09 15:46:52 +00:00
|
|
|
void TCPHandler::runImpl()
|
|
|
|
{
|
2018-08-31 00:59:48 +00:00
|
|
|
setThreadName("TCPHandler");
|
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
extractConnectionSettingsFromContext(server.context());
|
2019-07-30 22:22:45 +00:00
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
socket().setReceiveTimeout(receive_timeout);
|
|
|
|
socket().setSendTimeout(send_timeout);
|
2014-04-06 06:53:45 +00:00
|
|
|
socket().setNoDelay(true);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-05-14 15:37:20 +00:00
|
|
|
in = std::make_shared<ReadBufferFromPocoSocketChunked>(socket(), read_event);
|
2024-09-26 08:27:37 +00:00
|
|
|
out = std::make_shared<AutoCanceledWriteBuffer<WriteBufferFromPocoSocketChunked>>(socket(), write_event);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-12-02 21:05:51 +00:00
|
|
|
/// Support for PROXY protocol
|
|
|
|
if (parse_proxy_protocol && !receiveProxyHeader())
|
|
|
|
return;
|
|
|
|
|
2016-01-06 20:37:50 +00:00
|
|
|
if (in->eof())
|
|
|
|
{
|
2020-12-11 10:51:48 +00:00
|
|
|
LOG_INFO(log, "Client has not sent any data.");
|
2016-01-06 20:37:50 +00:00
|
|
|
return;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-07-30 22:22:45 +00:00
|
|
|
/// User will be authenticated here. It will also set settings from user profile into connection_context.
|
2013-08-10 09:04:45 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
receiveHello();
|
2023-07-30 22:09:03 +00:00
|
|
|
|
2024-10-28 16:43:43 +00:00
|
|
|
if (!default_database.empty())
|
|
|
|
DatabaseCatalog::instance().assertDatabaseExists(default_database);
|
|
|
|
|
2023-07-30 22:09:03 +00:00
|
|
|
/// In interserver mode queries are executed without a session context.
|
|
|
|
if (!is_interserver_mode)
|
|
|
|
session->makeSessionContext();
|
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
sendHello();
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2022-08-03 20:36:52 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM)
|
2022-08-03 19:44:08 +00:00
|
|
|
receiveAddendum();
|
2021-09-21 18:07:52 +00:00
|
|
|
|
2024-05-20 16:25:19 +00:00
|
|
|
{
|
2024-06-26 03:19:16 +00:00
|
|
|
/// Server side of chunked protocol negotiation.
|
2024-06-26 03:43:28 +00:00
|
|
|
/// Server advertises its protocol capabilities (separate for send and receive channels) by sending
|
2024-06-26 03:19:16 +00:00
|
|
|
/// in its 'Hello' response one of four types - chunked, notchunked, chunked_optional, notchunked_optional.
|
|
|
|
/// Not optional types are strict meaning that server only supports this type, optional means that
|
|
|
|
/// server prefer this type but capable to work in opposite.
|
|
|
|
/// Client selects which type it is going to communicate based on the settings from config or arguments,
|
|
|
|
/// and sends either "chunked" or "notchunked" protocol request in addendum section of handshake.
|
|
|
|
/// Client can detect if server's protocol capabilities are not compatible with client's settings (for example
|
2024-06-26 03:43:28 +00:00
|
|
|
/// server strictly requires chunked protocol but client's settings only allows notchunked protocol) - in such case
|
|
|
|
/// client should interrupt this connection. However if client continues with incompatible protocol type request, server
|
2024-06-26 03:19:16 +00:00
|
|
|
/// will send appropriate exception and disconnect client.
|
|
|
|
|
2024-05-23 22:01:32 +00:00
|
|
|
auto is_chunked = [](const String & chunked_srv_str, const String & chunked_cl_str, const String & direction)
|
|
|
|
{
|
|
|
|
bool chunked_srv = chunked_srv_str.starts_with("chunked");
|
|
|
|
bool optional_srv = chunked_srv_str.ends_with("_optional");
|
|
|
|
bool chunked_cl = chunked_cl_str.starts_with("chunked");
|
|
|
|
|
|
|
|
if (optional_srv)
|
|
|
|
return chunked_cl;
|
2024-06-26 03:19:16 +00:00
|
|
|
|
2024-05-23 22:01:32 +00:00
|
|
|
if (chunked_cl != chunked_srv)
|
|
|
|
throw NetException(
|
|
|
|
ErrorCodes::NETWORK_ERROR,
|
|
|
|
"Incompatible protocol: {} is {}, client requested {}",
|
|
|
|
direction,
|
|
|
|
chunked_srv ? "chunked" : "notchunked",
|
|
|
|
chunked_cl ? "chunked" : "notchunked");
|
|
|
|
|
|
|
|
return chunked_srv;
|
|
|
|
};
|
|
|
|
|
2024-07-10 00:20:11 +00:00
|
|
|
bool out_chunked = is_chunked(server.config().getString("proto_caps.send", "notchunked"), proto_recv_chunked_cl, "send");
|
|
|
|
bool in_chunked = is_chunked(server.config().getString("proto_caps.recv", "notchunked"), proto_send_chunked_cl, "recv");
|
2024-05-23 22:01:32 +00:00
|
|
|
|
|
|
|
if (out_chunked)
|
|
|
|
out->enableChunked();
|
|
|
|
if (in_chunked)
|
|
|
|
in->enableChunked();
|
2024-05-20 16:25:19 +00:00
|
|
|
}
|
|
|
|
|
2023-07-30 22:09:03 +00:00
|
|
|
if (!is_interserver_mode)
|
2021-09-21 18:07:52 +00:00
|
|
|
{
|
|
|
|
/// If session created, then settings in session context has been updated.
|
|
|
|
/// So it's better to update the connection settings for flexibility.
|
|
|
|
extractConnectionSettingsFromContext(session->sessionContext());
|
|
|
|
|
|
|
|
/// When connecting, the default database could be specified.
|
|
|
|
if (!default_database.empty())
|
|
|
|
session->sessionContext()->setCurrentDatabase(default_database);
|
|
|
|
}
|
2013-08-10 09:04:45 +00:00
|
|
|
}
|
2017-03-09 04:26:17 +00:00
|
|
|
catch (const Exception & e) /// Typical for an incorrect username, password, or address.
|
2013-08-10 09:04:45 +00:00
|
|
|
{
|
2014-03-02 19:52:55 +00:00
|
|
|
if (e.code() == ErrorCodes::CLIENT_HAS_CONNECTED_TO_WRONG_PORT)
|
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "Client has connected to wrong port.");
|
|
|
|
return;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2016-01-06 20:37:50 +00:00
|
|
|
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
|
|
|
|
{
|
2020-12-11 10:51:48 +00:00
|
|
|
LOG_INFO(log, "Client has gone away.");
|
2016-01-06 20:37:50 +00:00
|
|
|
return;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2013-08-10 09:04:45 +00:00
|
|
|
try
|
|
|
|
{
|
2017-04-17 16:02:48 +00:00
|
|
|
/// We try to send error information to the client.
|
2021-08-01 14:12:34 +00:00
|
|
|
sendException(e, send_exception_with_stack_trace);
|
2013-08-10 09:04:45 +00:00
|
|
|
}
|
2024-09-26 08:27:37 +00:00
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2013-08-10 09:04:45 +00:00
|
|
|
throw;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-10-22 07:15:34 +00:00
|
|
|
while (tcp_server.isOpen())
|
2012-03-09 15:46:52 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
/// We don't really have session in interserver mode, new one is created for each query. It's better to reset it now.
|
|
|
|
if (is_interserver_mode)
|
|
|
|
session.reset();
|
|
|
|
|
2019-07-04 22:57:26 +00:00
|
|
|
/// We are waiting for a packet from the client. Thus, every `poll_interval` seconds check whether we need to shut down.
|
2019-07-04 22:23:45 +00:00
|
|
|
{
|
|
|
|
Stopwatch idle_time;
|
2021-03-03 20:40:43 +00:00
|
|
|
UInt64 timeout_ms = std::min(poll_interval, idle_connection_timeout) * 1000000;
|
2024-10-31 11:22:26 +00:00
|
|
|
|
2024-05-14 15:37:20 +00:00
|
|
|
while (tcp_server.isOpen() && !server.isCancelled() && !in->poll(timeout_ms))
|
2019-07-04 22:23:45 +00:00
|
|
|
{
|
2021-03-03 20:40:43 +00:00
|
|
|
if (idle_time.elapsedSeconds() > idle_connection_timeout)
|
2019-07-04 23:03:20 +00:00
|
|
|
{
|
|
|
|
LOG_TRACE(log, "Closing idle connection");
|
2019-07-04 22:23:45 +00:00
|
|
|
return;
|
2019-07-04 23:03:20 +00:00
|
|
|
}
|
2019-07-04 22:23:45 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
/// If we need to shut down, or client disconnects.
|
|
|
|
if (!tcp_server.isOpen() || server.isCancelled() || in->eof())
|
|
|
|
{
|
|
|
|
LOG_TEST(log, "Closing connection (open: {}, cancelled: {}, eof: {})", tcp_server.isOpen(), server.isCancelled(), in->eof());
|
|
|
|
return;
|
|
|
|
}
|
2022-08-05 11:33:27 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-08-25 12:20:13 +00:00
|
|
|
OpenTelemetry::TracingContextHolderPtr thread_trace_context;
|
2024-10-22 12:26:06 +00:00
|
|
|
/// Initialized later. It has to be destroyed after query_state is destroyed.
|
|
|
|
std::optional<CurrentThread::QueryScope> query_scope;
|
|
|
|
/// QueryState should be cleared before QueryScope, since otherwise
|
|
|
|
/// the MemoryTracker will be wrong for possible deallocations.
|
|
|
|
/// (i.e. deallocations from the Aggregator with two-level aggregation)
|
|
|
|
/// Also it resets socket's timeouts.
|
|
|
|
std::optional<QueryState> query_state;
|
2018-09-07 23:22:02 +00:00
|
|
|
|
2017-03-09 04:26:17 +00:00
|
|
|
/** An exception during the execution of request (it must be sent over the network to the client).
|
2017-04-17 16:02:48 +00:00
|
|
|
* The client will be able to accept it, if it did not happen while sending another packet and the client has not disconnected yet.
|
|
|
|
*/
|
2022-04-15 23:56:45 +00:00
|
|
|
std::unique_ptr<DB::Exception> exception;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2012-05-08 05:42:05 +00:00
|
|
|
try
|
2013-02-01 19:02:04 +00:00
|
|
|
{
|
2017-04-17 16:02:48 +00:00
|
|
|
/** If Query - process it. If Ping or Cancel - go back to the beginning.
|
2024-10-31 11:22:26 +00:00
|
|
|
* There may come settings for a separate query that modify `query_context`.
|
|
|
|
* It's possible to receive part uuids packet before the query, so then receivePacket has to be called twice.
|
|
|
|
*/
|
2024-10-22 12:26:06 +00:00
|
|
|
if (!receivePacketsExpectQuery(query_state))
|
2013-09-14 07:58:42 +00:00
|
|
|
continue;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-11-20 17:23:53 +00:00
|
|
|
/** If part_uuids got received in previous packet, trying to read again.
|
2024-10-31 11:22:26 +00:00
|
|
|
*/
|
2024-10-22 12:26:06 +00:00
|
|
|
if (part_uuids_to_ignore.has_value() && !receivePacketsExpectQuery(query_state))
|
2020-11-20 17:23:53 +00:00
|
|
|
continue;
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
chassert(query_state.has_value());
|
|
|
|
|
2022-07-07 09:44:19 +00:00
|
|
|
/// Set up tracing context for this query on current thread
|
2022-08-25 12:20:13 +00:00
|
|
|
thread_trace_context = std::make_unique<OpenTelemetry::TracingContextHolder>("TCPHandler",
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->query_context->getClientInfo().client_trace_context,
|
|
|
|
query_state->query_context->getSettingsRef(),
|
|
|
|
query_state->query_context->getOpenTelemetrySpanLog());
|
2024-05-09 00:07:04 +00:00
|
|
|
thread_trace_context->root_span.kind = OpenTelemetry::SpanKind::SERVER;
|
2022-07-07 09:42:22 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
query_scope.emplace(query_state->query_context, /* fatal_error_callback */ [this, &query_state]
|
2023-03-03 00:22:04 +00:00
|
|
|
{
|
2024-10-31 11:22:26 +00:00
|
|
|
std::lock_guard lock(callback_mutex);
|
2024-10-22 12:26:06 +00:00
|
|
|
sendLogs(query_state.value());
|
2023-03-03 00:22:04 +00:00
|
|
|
});
|
2018-06-01 19:39:32 +00:00
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
/// If query received, then settings in query_context has been updated.
|
|
|
|
/// So it's better to update the connection settings for flexibility.
|
2024-10-22 12:26:06 +00:00
|
|
|
extractConnectionSettingsFromContext(query_state->query_context);
|
2021-08-01 14:12:34 +00:00
|
|
|
|
2024-07-30 18:48:34 +00:00
|
|
|
/// Sync timeouts on client and server during current query to avoid dangling queries on server.
|
|
|
|
/// It should be reset at the end of query.
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->timeout_setter = std::make_unique<TimeoutSetter>(socket(), send_timeout, receive_timeout);
|
|
|
|
|
|
|
|
SCOPE_EXIT(logQueryDuration(query_state.value()));
|
2018-08-24 07:30:53 +00:00
|
|
|
|
2018-06-06 20:57:07 +00:00
|
|
|
/// Should we send internal logs to client?
|
2024-10-22 12:26:06 +00:00
|
|
|
const auto client_logs_level = query_state->query_context->getSettingsRef()[Setting::send_logs_level];
|
2020-09-17 12:15:05 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_LOGS
|
2019-08-09 13:02:01 +00:00
|
|
|
&& client_logs_level != LogsLevel::none)
|
2018-06-06 20:57:07 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->logs_queue = std::make_shared<InternalTextLogsQueue>();
|
|
|
|
query_state->logs_queue->max_priority = Poco::Logger::parseLevel(client_logs_level.toString());
|
|
|
|
query_state->logs_queue->setSourceRegexp(query_state->query_context->getSettingsRef()[Setting::send_logs_source_regexp]);
|
|
|
|
CurrentThread::attachInternalTextLogsQueue(query_state->logs_queue, client_logs_level);
|
2018-06-06 20:57:07 +00:00
|
|
|
}
|
2021-11-08 13:43:34 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS)
|
2021-09-20 21:52:01 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->profile_queue = std::make_shared<InternalProfileEventsQueue>(std::numeric_limits<int>::max());
|
|
|
|
CurrentThread::attachInternalProfileEventsQueue(query_state->profile_queue);
|
2018-06-06 20:57:07 +00:00
|
|
|
}
|
|
|
|
|
2024-04-26 17:56:24 +00:00
|
|
|
if (!is_interserver_mode)
|
2024-04-30 12:52:43 +00:00
|
|
|
session->checkIfUserIsStillValid();
|
2024-04-26 15:03:40 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->query_context->setExternalTablesInitializer([this, &query_state] (ContextPtr context)
|
2018-08-24 00:07:25 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
if (context != query_state->query_context)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected context in external tables initializer");
|
2018-06-01 15:32:27 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
std::lock_guard lock(callback_mutex);
|
|
|
|
|
2018-06-01 15:32:27 +00:00
|
|
|
/// Get blocks of temporary tables
|
2024-10-22 12:26:06 +00:00
|
|
|
readData(query_state.value());
|
2018-06-01 15:32:27 +00:00
|
|
|
|
|
|
|
/// Reset the input stream, as we received an empty block while receiving external table data.
|
|
|
|
/// So, the stream has been marked as cancelled and we can't read from it anymore.
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->block_in.reset();
|
|
|
|
query_state->maybe_compressed_in.reset(); /// For more accurate accounting by MemoryTracker.
|
2018-06-01 15:32:27 +00:00
|
|
|
});
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-05-28 18:30:10 +00:00
|
|
|
/// Send structure of columns to client for function input()
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->query_context->setInputInitializer([this, &query_state] (ContextPtr context, const StoragePtr & input_storage)
|
2019-05-28 18:30:10 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
if (context != query_state->query_context)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected context in Input initializer");
|
2019-05-28 18:30:10 +00:00
|
|
|
|
2020-06-17 16:39:58 +00:00
|
|
|
auto metadata_snapshot = input_storage->getInMemoryMetadataPtr();
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
std::lock_guard lock(callback_mutex);
|
|
|
|
|
|
|
|
query_state->need_receive_data_for_input = true;
|
2019-05-28 18:30:10 +00:00
|
|
|
|
|
|
|
/// Send ColumnsDescription for input storage.
|
2020-09-17 12:15:05 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA
|
2024-10-22 12:26:06 +00:00
|
|
|
&& query_state->query_context->getSettingsRef()[Setting::input_format_defaults_for_omitted_fields])
|
2019-05-28 18:30:10 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
sendTableColumns(query_state.value(), metadata_snapshot->getColumns());
|
2019-05-28 18:30:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Send block to the client - input storage structure.
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->input_header = metadata_snapshot->getSampleBlock();
|
|
|
|
sendData(query_state.value(), query_state->input_header);
|
|
|
|
sendTimezone(query_state.value());
|
2019-05-28 18:30:10 +00:00
|
|
|
});
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->query_context->setInputBlocksReaderCallback([this, &query_state] (ContextPtr context) -> Block
|
2019-05-28 18:30:10 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
if (context != query_state->query_context)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected context in InputBlocksReader");
|
2019-05-28 18:30:10 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
std::lock_guard lock(callback_mutex);
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
if (receivePacketsExpectData(query_state.value()))
|
|
|
|
return query_state->block_for_input;
|
|
|
|
|
|
|
|
query_state->read_all_data = true;
|
|
|
|
query_state->block_in.reset();
|
|
|
|
query_state->maybe_compressed_in.reset();
|
|
|
|
return {};
|
2019-05-28 18:30:10 +00:00
|
|
|
});
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
customizeContext(query_state->query_context);
|
2019-03-06 16:41:35 +00:00
|
|
|
|
2021-04-12 17:07:01 +00:00
|
|
|
/// This callback is needed for requesting read tasks inside pipeline for distributed processing
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->query_context->setReadTaskCallback([this, &query_state]() -> String
|
2021-04-06 11:05:47 +00:00
|
|
|
{
|
2023-02-12 09:04:05 +00:00
|
|
|
Stopwatch watch;
|
|
|
|
CurrentMetrics::Increment callback_metric_increment(CurrentMetrics::ReadTaskRequestsSent);
|
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
std::lock_guard lock(callback_mutex);
|
|
|
|
|
|
|
|
sendReadTaskRequestAssumeLocked();
|
|
|
|
|
2023-02-12 09:04:05 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::ReadTaskRequestsSent);
|
2024-10-22 12:26:06 +00:00
|
|
|
auto res = receiveReadTaskResponseAssumeLocked(query_state.value());
|
2023-02-12 09:04:05 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::ReadTaskRequestsSentElapsedMicroseconds, watch.elapsedMicroseconds());
|
|
|
|
return res;
|
2021-04-06 11:05:47 +00:00
|
|
|
});
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->query_context->setMergeTreeAllRangesCallback([this, &query_state](InitialAllRangesAnnouncement announcement)
|
2023-02-03 13:34:18 +00:00
|
|
|
{
|
2023-02-12 09:04:05 +00:00
|
|
|
Stopwatch watch;
|
|
|
|
CurrentMetrics::Increment callback_metric_increment(CurrentMetrics::MergeTreeAllRangesAnnouncementsSent);
|
2023-02-03 13:34:18 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
std::lock_guard lock(callback_mutex);
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
sendMergeTreeAllRangesAnnouncementAssumeLocked(query_state.value(), announcement);
|
2023-02-12 09:04:05 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::MergeTreeAllRangesAnnouncementsSent);
|
|
|
|
ProfileEvents::increment(ProfileEvents::MergeTreeAllRangesAnnouncementsSentElapsedMicroseconds, watch.elapsedMicroseconds());
|
2023-02-03 13:34:18 +00:00
|
|
|
});
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->query_context->setMergeTreeReadTaskCallback([this, &query_state](ParallelReadRequest request) -> std::optional<ParallelReadResponse>
|
2021-12-09 10:39:28 +00:00
|
|
|
{
|
2023-02-12 09:04:05 +00:00
|
|
|
Stopwatch watch;
|
|
|
|
CurrentMetrics::Increment callback_metric_increment(CurrentMetrics::MergeTreeReadTaskRequestsSent);
|
2021-12-09 10:39:28 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
std::lock_guard lock(callback_mutex);
|
|
|
|
|
|
|
|
sendMergeTreeReadTaskRequestAssumeLocked(std::move(request));
|
2021-12-09 10:39:28 +00:00
|
|
|
|
2023-02-12 09:04:05 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::MergeTreeReadTaskRequestsSent);
|
2024-10-22 12:26:06 +00:00
|
|
|
auto res = receivePartitionMergeTreeReadTaskResponseAssumeLocked(query_state.value());
|
2023-02-12 09:04:05 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::MergeTreeReadTaskRequestsSentElapsedMicroseconds, watch.elapsedMicroseconds());
|
|
|
|
return res;
|
2021-12-09 10:39:28 +00:00
|
|
|
});
|
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
LOG_DEBUG(log, "executeQuery before");
|
|
|
|
|
2017-04-17 16:02:48 +00:00
|
|
|
/// Processing Query
|
2024-10-22 12:26:06 +00:00
|
|
|
std::tie(query_state->parsed_query, query_state->io) = executeQuery(query_state->query, query_state->query_context, QueryFlags{}, query_state->stage);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
LOG_DEBUG(log, "executeQuery started");
|
|
|
|
|
2012-05-09 15:15:45 +00:00
|
|
|
after_check_cancelled.restart();
|
|
|
|
after_send_progress.restart();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
if (query_state->io.pipeline.pushing())
|
2020-10-10 01:43:07 +00:00
|
|
|
{
|
2022-10-21 03:33:17 +00:00
|
|
|
/// FIXME: check explicitly that insert query suggests to receive data via native protocol,
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->need_receive_data_for_insert = true;
|
|
|
|
processInsertQuery(query_state.value());
|
|
|
|
query_state->io.onFinish();
|
2020-10-10 01:43:07 +00:00
|
|
|
}
|
2024-10-22 12:26:06 +00:00
|
|
|
else if (query_state->io.pipeline.pulling())
|
2021-09-16 17:40:42 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
processOrdinaryQuery(query_state.value());
|
|
|
|
query_state->io.onFinish();
|
2021-09-16 17:40:42 +00:00
|
|
|
}
|
2024-10-22 12:26:06 +00:00
|
|
|
else if (query_state->io.pipeline.completed())
|
2019-05-28 18:30:10 +00:00
|
|
|
{
|
2024-10-31 11:22:26 +00:00
|
|
|
LOG_DEBUG(log, " pipeline completed started");
|
2021-09-22 13:29:58 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
CompletedPipelineExecutor executor(query_state->io.pipeline);
|
2022-10-21 03:33:17 +00:00
|
|
|
|
|
|
|
/// Should not check for cancel in case of input.
|
2024-10-22 12:26:06 +00:00
|
|
|
if (!query_state->need_receive_data_for_input)
|
2021-09-22 13:29:58 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
auto callback = [this, &query_state]()
|
2022-10-21 03:33:17 +00:00
|
|
|
{
|
2024-10-31 11:22:26 +00:00
|
|
|
std::lock_guard lock(callback_mutex);
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
receivePacketsExpectCancel(query_state.value());
|
2023-01-10 15:34:52 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
if (query_state->stop_read_return_partial_result)
|
2023-01-17 15:17:14 +00:00
|
|
|
return true;
|
2021-09-22 13:29:58 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
sendProgress(query_state.value());
|
|
|
|
sendSelectProfileEvents(query_state.value());
|
|
|
|
sendLogs(query_state.value());
|
2022-10-21 03:33:17 +00:00
|
|
|
return false;
|
|
|
|
};
|
2021-09-22 13:29:58 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
executor.setCancelCallback(std::move(callback), interactive_delay / 1000);
|
2022-10-21 03:33:17 +00:00
|
|
|
}
|
2024-10-03 10:23:56 +00:00
|
|
|
|
2022-10-21 03:33:17 +00:00
|
|
|
executor.execute();
|
2021-09-22 13:29:58 +00:00
|
|
|
}
|
2021-11-18 21:24:28 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
LOG_DEBUG(log, " pipeline completed finish");
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->io.onFinish();
|
2023-01-17 15:17:14 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
/// Send final progress after calling onFinish(), since it will update the progress.
|
|
|
|
///
|
|
|
|
/// NOTE: we cannot send Progress for regular INSERT (with VALUES)
|
|
|
|
/// without breaking protocol compatibility, but it can be done
|
|
|
|
/// by increasing revision.
|
|
|
|
sendProgress(query_state.value());
|
|
|
|
sendSelectProfileEvents(query_state.value());
|
2019-05-28 18:30:10 +00:00
|
|
|
}
|
2022-10-04 20:49:43 +00:00
|
|
|
else
|
|
|
|
{
|
2024-10-31 11:22:26 +00:00
|
|
|
LOG_DEBUG(log, "other");
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->io.onFinish();
|
2022-10-04 20:49:43 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-26 21:19:49 +00:00
|
|
|
/// Do it before sending end of stream, to have a chance to show log message in client.
|
|
|
|
query_scope->logPeakMemoryUsage();
|
2020-10-24 03:41:47 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
LOG_DEBUG(log, "send logs at final");
|
|
|
|
|
|
|
|
sendLogs(query_state.value());
|
|
|
|
sendEndOfStream(query_state.value());
|
2024-10-03 15:23:32 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->finalizeOut();
|
|
|
|
}
|
2013-10-26 03:20:51 +00:00
|
|
|
catch (const Exception & e)
|
2012-03-19 12:57:56 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
LOG_DEBUG(log, "XX Exception: {}", e.what());
|
|
|
|
exception.reset(e.clone());
|
2013-01-13 22:13:54 +00:00
|
|
|
}
|
|
|
|
catch (const Poco::Exception & e)
|
2012-05-08 05:42:05 +00:00
|
|
|
{
|
2024-10-24 16:50:22 +00:00
|
|
|
LOG_DEBUG(log, "XX Poco Exception: {}", e.what());
|
2022-04-15 23:56:45 +00:00
|
|
|
exception = std::make_unique<DB::Exception>(Exception::CreateFromPocoTag{}, e);
|
2012-05-08 05:42:05 +00:00
|
|
|
}
|
2020-07-17 14:25:13 +00:00
|
|
|
// Server should die on std logic errors in debug, like with assert()
|
2024-10-24 16:50:22 +00:00
|
|
|
// or ErrorCodes::LOGICAL_ERROR. This helps catch these errors in tests.
|
2024-07-17 15:59:31 +00:00
|
|
|
#ifdef DEBUG_OR_SANITIZER_BUILD
|
2020-07-15 12:08:01 +00:00
|
|
|
catch (const std::logic_error & e)
|
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
if (query_state.has_value())
|
|
|
|
query_state->io.onException();
|
2022-04-15 23:56:45 +00:00
|
|
|
exception = std::make_unique<DB::Exception>(Exception::CreateFromSTDTag{}, e);
|
2020-07-17 14:20:33 +00:00
|
|
|
sendException(*exception, send_exception_with_stack_trace);
|
2020-07-17 14:25:13 +00:00
|
|
|
std::abort();
|
2020-07-15 12:08:01 +00:00
|
|
|
}
|
2020-07-17 14:25:13 +00:00
|
|
|
#endif
|
2013-01-13 22:13:54 +00:00
|
|
|
catch (const std::exception & e)
|
2012-05-08 05:42:05 +00:00
|
|
|
{
|
2022-04-15 23:56:45 +00:00
|
|
|
exception = std::make_unique<DB::Exception>(Exception::CreateFromSTDTag{}, e);
|
2012-05-08 05:42:05 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
2023-12-23 12:37:57 +00:00
|
|
|
exception = std::make_unique<DB::Exception>(Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception"));
|
2012-03-19 12:57:56 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
LOG_DEBUG(log, "we have an exception {}", bool(exception));
|
2024-07-31 17:06:30 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
if (exception)
|
2013-01-13 22:13:54 +00:00
|
|
|
{
|
2024-10-24 16:50:22 +00:00
|
|
|
auto exception_code = exception->code();
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
if (!query_state.has_value())
|
2018-06-06 20:57:07 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
LOG_DEBUG(log, "we do not have an query state");
|
2022-07-07 09:42:22 +00:00
|
|
|
|
2024-10-24 16:50:22 +00:00
|
|
|
if (exception_code == ErrorCodes::QUERY_WAS_CANCELLED_BY_CLIENT)
|
2024-10-22 12:26:06 +00:00
|
|
|
LOG_INFO(log, getExceptionMessageAndPattern(*exception, send_exception_with_stack_trace));
|
|
|
|
else
|
|
|
|
LOG_ERROR(log, getExceptionMessageAndPattern(*exception, send_exception_with_stack_trace));
|
2024-10-24 16:50:22 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
return;
|
|
|
|
}
|
2018-06-09 15:52:59 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
exception->rethrow();
|
|
|
|
}
|
2024-10-22 14:11:01 +00:00
|
|
|
catch (...)
|
2024-10-22 12:26:06 +00:00
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "query_state->io.onException()");
|
2024-10-24 16:50:22 +00:00
|
|
|
query_state->io.onException(exception_code != ErrorCodes::QUERY_WAS_CANCELLED_BY_CLIENT);
|
2018-06-06 20:57:07 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-24 16:50:22 +00:00
|
|
|
/// Authentication failure with interserver secret
|
|
|
|
/// - early exit without trying to send the exception to the client.
|
|
|
|
/// Because the server should not try to skip (parse, decompress) the remaining packets sent by the client,
|
|
|
|
/// as it will lead to additional work and unneeded exposure to unauthenticated connections.
|
2023-11-23 11:41:14 +00:00
|
|
|
|
2024-10-24 16:50:22 +00:00
|
|
|
/// Note that the exception AUTHENTICATION_FAILED can be here in two cases:
|
|
|
|
/// 1. The authentication in receiveHello is skipped with "interserver secret",
|
|
|
|
/// postponed to receiving the query, and then failed.
|
|
|
|
/// 2. Receiving exception from a query using a table function to authenticate with another server.
|
|
|
|
/// In this case, the user is already authenticated with this server,
|
|
|
|
/// is_interserver_mode is false, and we can send the exception to the client normally.
|
|
|
|
|
|
|
|
if (is_interserver_mode
|
|
|
|
&& !is_interserver_authenticated)
|
|
|
|
{
|
|
|
|
/// Interserver authentication is done only after we read the query.
|
|
|
|
/// This fact can be abused by producing exception before or while we read the query.
|
|
|
|
/// To avoid any potential exploits, we simply close connection on any exceptions
|
|
|
|
/// that happen before the first query is authenticated with the cluster secret.
|
|
|
|
LOG_DEBUG(log, "is_interserver_mode && !is_interserver_authenticated");
|
|
|
|
query_state->cancelOut();
|
2024-10-22 12:26:06 +00:00
|
|
|
return;
|
2024-10-24 16:50:22 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (exception_code == ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT)
|
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "XX Exception UNKNOWN_PACKET_FROM_CLIENT");
|
|
|
|
query_state->cancelOut();
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (thread_trace_context)
|
|
|
|
thread_trace_context->root_span.addAttribute(*exception);
|
2019-12-07 23:41:51 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "try send logs");
|
|
|
|
/// Try to send logs to client, but it could be risky too
|
|
|
|
/// Assume that we can't break output here
|
|
|
|
sendLogs(query_state.value());
|
2022-08-08 04:55:41 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
LOG_DEBUG(log, "try skip data");
|
|
|
|
/// A query packet is always followed by one or more data packets.
|
|
|
|
/// If some of those data packets are left, try to skip them.
|
|
|
|
if (!query_state->read_all_data)
|
|
|
|
skipData(query_state.value());
|
2024-10-02 08:45:31 +00:00
|
|
|
|
2024-10-24 16:50:22 +00:00
|
|
|
if (exception_code == ErrorCodes::QUERY_WAS_CANCELLED_BY_CLIENT)
|
2024-10-22 12:26:06 +00:00
|
|
|
{
|
2024-10-24 16:50:22 +00:00
|
|
|
LOG_DEBUG(log, "try send EndOfStream");
|
|
|
|
sendEndOfStream(query_state.value());
|
2024-10-22 12:26:06 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2024-10-24 16:50:22 +00:00
|
|
|
LOG_DEBUG(log, "try send exception");
|
|
|
|
sendException(*exception, send_exception_with_stack_trace);
|
2024-10-22 12:26:06 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
LOG_DEBUG(log, "Logs and exception has been sent");
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "failed");
|
|
|
|
query_state->cancelOut();
|
|
|
|
tryLogCurrentException(log, "Can't send logs or exception to client. Close connection.");
|
|
|
|
return;
|
|
|
|
}
|
2019-03-06 21:32:26 +00:00
|
|
|
|
2024-10-24 16:50:22 +00:00
|
|
|
if (exception->code() == ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT
|
|
|
|
|| exception->code() == ErrorCodes::USER_EXPIRED)
|
2024-10-22 12:26:06 +00:00
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "Going to close connection due to exception: {}", exception->message());
|
|
|
|
query_state->finalizeOut();
|
|
|
|
return;
|
|
|
|
}
|
2022-05-18 19:57:20 +00:00
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
query_state->finalizeOut();
|
2012-03-19 12:57:56 +00:00
|
|
|
}
|
2012-03-11 08:52:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::logQueryDuration(QueryState & state)
|
|
|
|
{
|
|
|
|
if (state.query_duration_already_logged)
|
|
|
|
return;
|
|
|
|
state.query_duration_already_logged = true;
|
|
|
|
auto elapsed_sec = state.watch.elapsedSeconds();
|
|
|
|
/// We already logged more detailed info if we read some rows
|
|
|
|
if (elapsed_sec < 1.0 && state.progress.read_rows)
|
|
|
|
return;
|
|
|
|
LOG_DEBUG(log, "Processed in {} sec.", elapsed_sec);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
void TCPHandler::extractConnectionSettingsFromContext(const ContextPtr & context)
|
|
|
|
{
|
|
|
|
const auto & settings = context->getSettingsRef();
|
2024-09-18 12:20:53 +00:00
|
|
|
send_exception_with_stack_trace = settings[Setting::calculate_text_stack_trace];
|
|
|
|
send_timeout = settings[Setting::send_timeout];
|
|
|
|
receive_timeout = settings[Setting::receive_timeout];
|
|
|
|
poll_interval = settings[Setting::poll_interval];
|
|
|
|
idle_connection_timeout = settings[Setting::idle_connection_timeout];
|
|
|
|
interactive_delay = settings[Setting::interactive_delay];
|
|
|
|
sleep_in_send_tables_status = settings[Setting::sleep_in_send_tables_status_ms];
|
|
|
|
unknown_packet_in_send_data = settings[Setting::unknown_packet_in_send_data];
|
|
|
|
sleep_after_receiving_query = settings[Setting::sleep_after_receiving_query_ms];
|
2021-08-01 14:12:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
bool TCPHandler::receivePacketsExpectQuery(std::optional<QueryState> & state)
|
2019-05-28 18:30:10 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
UInt64 packet_type = 0;
|
|
|
|
readVarUInt(packet_type, *in);
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "receivePacketsExpectQuery got {}", Protocol::Client::toString(packet_type));
|
|
|
|
|
|
|
|
switch (packet_type)
|
|
|
|
{
|
|
|
|
case Protocol::Client::Hello:
|
|
|
|
processUnexpectedHello();
|
|
|
|
|
|
|
|
case Protocol::Client::Data:
|
|
|
|
case Protocol::Client::Scalar:
|
|
|
|
processUnexpectedData();
|
|
|
|
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet Data received from client");
|
|
|
|
|
|
|
|
case Protocol::Client::Ping:
|
|
|
|
writeVarUInt(Protocol::Server::Pong, *out);
|
|
|
|
out->finishChunk();
|
|
|
|
out->next();
|
|
|
|
return false;
|
|
|
|
|
|
|
|
case Protocol::Client::Cancel:
|
|
|
|
return false;
|
|
|
|
|
|
|
|
case Protocol::Client::TablesStatusRequest:
|
|
|
|
processTablesStatusRequest();
|
|
|
|
return false;
|
|
|
|
|
|
|
|
case Protocol::Client::IgnoredPartUUIDs:
|
|
|
|
/// Part uuids packet if any comes before query.
|
|
|
|
processIgnoredPartUUIDs();
|
|
|
|
return true;
|
|
|
|
|
|
|
|
case Protocol::Client::Query:
|
|
|
|
processQuery(state);
|
|
|
|
return true;
|
|
|
|
|
|
|
|
default:
|
|
|
|
throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT, "Unknown packet {} from client", toString(packet_type));
|
|
|
|
}
|
2019-05-28 18:30:10 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
chassert(server.isCancelled() || !tcp_server.isOpen());
|
|
|
|
throw Exception(ErrorCodes::ABORTED, "Server shutdown is called");
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
bool TCPHandler::receivePacketsExpectDataConcurrentWithExecutor(QueryState & state)
|
|
|
|
{
|
|
|
|
std::lock_guard lock(callback_mutex);
|
|
|
|
return receivePacketsExpectData(state);
|
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
bool TCPHandler::receivePacketsExpectData(QueryState & state)
|
|
|
|
{
|
2021-08-01 14:12:34 +00:00
|
|
|
/// Poll interval should not be greater than receive_timeout
|
2022-08-05 11:33:27 +00:00
|
|
|
constexpr UInt64 min_timeout_us = 5000; // 5 ms
|
2024-10-22 12:26:06 +00:00
|
|
|
UInt64 timeout_us = std::max(
|
|
|
|
min_timeout_us,
|
|
|
|
std::min(
|
|
|
|
poll_interval * 1000000,
|
|
|
|
static_cast<UInt64>(receive_timeout.totalMicroseconds())));
|
2021-08-01 14:12:34 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
Stopwatch watch;
|
2024-10-31 11:22:26 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
while (!server.isCancelled() && tcp_server.isOpen())
|
2019-05-28 18:30:10 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
if (!in->poll(timeout_us))
|
2021-08-01 14:12:34 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
size_t elapsed = size_t(watch.elapsedSeconds());
|
|
|
|
if (elapsed > size_t(receive_timeout.totalSeconds()))
|
2021-08-01 14:12:34 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
throw NetException(ErrorCodes::SOCKET_TIMEOUT,
|
|
|
|
"Timeout exceeded while receiving data from client. Waited for {} seconds, timeout is {} seconds.",
|
|
|
|
elapsed, receive_timeout.totalSeconds());
|
2021-08-01 14:12:34 +00:00
|
|
|
}
|
|
|
|
}
|
2019-05-28 18:30:10 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
UInt64 packet_type = 0;
|
|
|
|
readVarUInt(packet_type, *in);
|
2019-05-28 18:30:10 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
LOG_DEBUG(log, "receivePacketsExpectData got {}", Protocol::Client::toString(packet_type));
|
|
|
|
|
|
|
|
switch (packet_type)
|
2019-05-28 18:30:10 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
case Protocol::Client::IgnoredPartUUIDs:
|
|
|
|
processUnexpectedIgnoredPartUUIDs();
|
2019-05-28 18:30:10 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
case Protocol::Client::Query:
|
|
|
|
processUnexpectedQuery();
|
|
|
|
|
|
|
|
case Protocol::Client::Hello:
|
|
|
|
processUnexpectedHello();
|
|
|
|
|
|
|
|
case Protocol::Client::TablesStatusRequest:
|
|
|
|
processUnexpectedTablesStatusRequest();
|
|
|
|
|
|
|
|
case Protocol::Client::Data:
|
|
|
|
case Protocol::Client::Scalar:
|
|
|
|
if (state.skipping_data)
|
|
|
|
return processUnexpectedData();
|
|
|
|
return processData(state, packet_type == Protocol::Client::Scalar);
|
|
|
|
|
|
|
|
case Protocol::Client::Ping:
|
|
|
|
writeVarUInt(Protocol::Server::Pong, *out);
|
|
|
|
out->finishChunk();
|
|
|
|
out->next();
|
|
|
|
continue;
|
|
|
|
|
|
|
|
case Protocol::Client::Cancel:
|
|
|
|
processCancel(state);
|
|
|
|
return false; // We return false from this function as if no more data received
|
|
|
|
|
|
|
|
default:
|
|
|
|
throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT, "Unknown packet {} from client", toString(packet_type));
|
|
|
|
}
|
2022-05-20 15:19:40 +00:00
|
|
|
}
|
2019-05-28 18:30:10 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
chassert(server.isCancelled() || !tcp_server.isOpen());
|
|
|
|
throw Exception(ErrorCodes::ABORTED, "Server shutdown is called");
|
2019-05-28 18:30:10 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::readData(QueryState & state)
|
2012-05-21 06:49:05 +00:00
|
|
|
{
|
2024-10-31 11:22:26 +00:00
|
|
|
sendLogs(state);
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2024-10-22 14:11:01 +00:00
|
|
|
/// no sense in partial_result_on_first_cancel setting when temporary data is read.
|
2024-10-22 12:26:06 +00:00
|
|
|
auto off_setting_guard = TurnOffBoolSettingTemporary(state.allow_partial_result_on_first_cancel);
|
2018-04-04 12:11:38 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
while (receivePacketsExpectData(state))
|
|
|
|
{
|
|
|
|
sendLogs(state);
|
|
|
|
sendInsertProfileEvents(state);
|
|
|
|
}
|
2023-02-21 19:04:40 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
state.read_all_data = true;
|
2019-05-28 18:30:10 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::skipData(QueryState & state)
|
2019-05-28 18:30:10 +00:00
|
|
|
{
|
2021-08-01 14:12:34 +00:00
|
|
|
state.skipping_data = true;
|
|
|
|
SCOPE_EXIT({ state.skipping_data = false; });
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
while (receivePacketsExpectData(state))
|
|
|
|
{
|
|
|
|
/// no op
|
|
|
|
}
|
2023-02-21 19:04:40 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
state.read_all_data = true;
|
2014-03-04 15:31:56 +00:00
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::startInsertQuery(QueryState & state)
|
2023-10-20 16:39:48 +00:00
|
|
|
{
|
2024-10-31 11:22:26 +00:00
|
|
|
|
|
|
|
std::lock_guard lock(callback_mutex);
|
|
|
|
|
2023-10-20 16:39:48 +00:00
|
|
|
/// Send ColumnsDescription for insertion table
|
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA)
|
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
const auto & table_id = state.query_context->getInsertionTable();
|
|
|
|
if (state.query_context->getSettingsRef()[Setting::input_format_defaults_for_omitted_fields])
|
2023-10-20 16:39:48 +00:00
|
|
|
{
|
|
|
|
if (!table_id.empty())
|
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
auto storage_ptr = DatabaseCatalog::instance().getTable(table_id, state.query_context);
|
|
|
|
sendTableColumns(state, storage_ptr->getInMemoryMetadataPtr()->getColumns());
|
2023-10-20 16:39:48 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
/// Send block to the client - table structure.
|
|
|
|
sendData(state, state.io.pipeline.getHeader());
|
|
|
|
sendLogs(state);
|
2023-10-20 16:39:48 +00:00
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(QueryState & state, AsynchronousInsertQueue & insert_queue)
|
2023-10-20 16:39:48 +00:00
|
|
|
{
|
|
|
|
using PushResult = AsynchronousInsertQueue::PushResult;
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
startInsertQuery(state);
|
|
|
|
Squashing squashing(state.input_header, 0, state.query_context->getSettingsRef()[Setting::async_insert_max_data_size]);
|
2023-10-20 16:39:48 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
while (receivePacketsExpectData(state))
|
2023-10-20 16:39:48 +00:00
|
|
|
{
|
2024-07-05 14:52:51 +00:00
|
|
|
squashing.setHeader(state.block_for_insert.cloneEmpty());
|
|
|
|
auto result_chunk = Squashing::squash(squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}));
|
|
|
|
if (result_chunk)
|
2023-10-20 16:39:48 +00:00
|
|
|
{
|
2024-07-05 14:52:51 +00:00
|
|
|
auto result = squashing.getHeader().cloneWithColumns(result_chunk.detachColumns());
|
2023-10-20 16:39:48 +00:00
|
|
|
return PushResult
|
|
|
|
{
|
|
|
|
.status = PushResult::TOO_MUCH_DATA,
|
|
|
|
.insert_block = std::move(result),
|
|
|
|
};
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
state.read_all_data = true;
|
|
|
|
|
2024-07-05 14:52:51 +00:00
|
|
|
Chunk result_chunk = Squashing::squash(squashing.flush());
|
|
|
|
if (!result_chunk)
|
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
return insert_queue.pushQueryWithBlock(state.parsed_query, squashing.getHeader(), state.query_context);
|
2024-07-05 14:52:51 +00:00
|
|
|
}
|
2024-06-21 18:40:49 +00:00
|
|
|
|
2024-07-05 14:52:51 +00:00
|
|
|
auto result = squashing.getHeader().cloneWithColumns(result_chunk.detachColumns());
|
2024-10-22 12:26:06 +00:00
|
|
|
return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), state.query_context);
|
2023-10-20 16:39:48 +00:00
|
|
|
}
|
2014-03-04 15:31:56 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::processInsertQuery(QueryState & state)
|
2014-03-04 15:31:56 +00:00
|
|
|
{
|
2021-09-15 19:35:48 +00:00
|
|
|
size_t num_threads = state.io.pipeline.getNumThreads();
|
2021-09-10 14:52:24 +00:00
|
|
|
|
2023-10-20 16:39:48 +00:00
|
|
|
auto run_executor = [&](auto & executor, Block processed_data)
|
2018-11-15 15:03:13 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
/// Made above the rest of the lines,
|
|
|
|
/// so that in case of `start` function throws an exception,
|
|
|
|
/// client receive exception before sending data.
|
|
|
|
executor.start();
|
2021-11-18 20:38:44 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
if (processed_data)
|
|
|
|
executor.push(std::move(processed_data));
|
|
|
|
else
|
|
|
|
startInsertQuery(state);
|
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
while (receivePacketsExpectDataConcurrentWithExecutor(state))
|
2024-10-22 12:26:06 +00:00
|
|
|
executor.push(std::move(state.block_for_insert));
|
2021-09-10 14:52:24 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
state.read_all_data = true;
|
2021-09-10 14:52:24 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
executor.finish();
|
|
|
|
}
|
|
|
|
catch (...)
|
2024-10-03 10:23:56 +00:00
|
|
|
{
|
2023-02-21 19:04:40 +00:00
|
|
|
executor.cancel();
|
2024-10-22 12:26:06 +00:00
|
|
|
throw;
|
2024-10-03 10:23:56 +00:00
|
|
|
}
|
2021-11-18 20:38:44 +00:00
|
|
|
};
|
|
|
|
|
2023-10-20 16:39:48 +00:00
|
|
|
Block processed_block;
|
2024-10-22 12:26:06 +00:00
|
|
|
const auto & settings = state.query_context->getSettingsRef();
|
2023-10-20 16:39:48 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
auto * insert_queue = state.query_context->tryGetAsynchronousInsertQueue();
|
2023-10-20 16:39:48 +00:00
|
|
|
const auto & insert_query = assert_cast<const ASTInsertQuery &>(*state.parsed_query);
|
|
|
|
|
2024-09-18 12:20:53 +00:00
|
|
|
bool async_insert_enabled = settings[Setting::async_insert];
|
2023-10-20 16:39:48 +00:00
|
|
|
if (insert_query.table_id)
|
2024-10-22 12:26:06 +00:00
|
|
|
if (auto table = DatabaseCatalog::instance().tryGetTable(insert_query.table_id, state.query_context))
|
2023-10-20 16:39:48 +00:00
|
|
|
async_insert_enabled |= table->areAsynchronousInsertsEnabled();
|
|
|
|
|
|
|
|
if (insert_queue && async_insert_enabled && !insert_query.select)
|
|
|
|
{
|
2024-03-07 19:39:56 +00:00
|
|
|
/// Let's agree on terminology and say that a mini-INSERT is an asynchronous INSERT
|
|
|
|
/// which typically contains not a lot of data inside and a big-INSERT in an INSERT
|
|
|
|
/// which was formed by concatenating several mini-INSERTs together.
|
|
|
|
/// In case when the client had to retry some mini-INSERTs then they will be properly deduplicated
|
|
|
|
/// by the source tables. This functionality is controlled by a setting `async_insert_deduplicate`.
|
|
|
|
/// But then they will be glued together into a block and pushed through a chain of Materialized Views if any.
|
|
|
|
/// The process of forming such blocks is not deteministic so each time we retry mini-INSERTs the resulting
|
|
|
|
/// block may be concatenated differently.
|
|
|
|
/// That's why deduplication in dependent Materialized Views doesn't make sense in presence of async INSERTs.
|
2024-09-18 12:20:53 +00:00
|
|
|
if (settings[Setting::throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert]
|
|
|
|
&& settings[Setting::deduplicate_blocks_in_dependent_materialized_views])
|
2024-03-07 19:39:56 +00:00
|
|
|
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
|
2024-07-24 12:05:50 +00:00
|
|
|
"Deduplication in dependent materialized view cannot work together with async inserts. "\
|
|
|
|
"Please disable either `deduplicate_blocks_in_dependent_materialized_views` or `async_insert` setting.");
|
2024-03-07 19:39:56 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
auto result = processAsyncInsertQuery(state, *insert_queue);
|
2023-10-20 16:39:48 +00:00
|
|
|
if (result.status == AsynchronousInsertQueue::PushResult::OK)
|
|
|
|
{
|
2024-03-07 17:42:50 +00:00
|
|
|
/// Reset pipeline because it may hold write lock for some storages.
|
|
|
|
state.io.pipeline.reset();
|
2024-09-18 12:20:53 +00:00
|
|
|
if (settings[Setting::wait_for_async_insert])
|
2023-10-20 16:39:48 +00:00
|
|
|
{
|
2024-09-18 12:20:53 +00:00
|
|
|
size_t timeout_ms = settings[Setting::wait_for_async_insert_timeout].totalMilliseconds();
|
2023-10-20 16:39:48 +00:00
|
|
|
auto wait_status = result.future.wait_for(std::chrono::milliseconds(timeout_ms));
|
|
|
|
|
|
|
|
if (wait_status == std::future_status::deferred)
|
2024-02-26 22:34:50 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got future in deferred state");
|
2023-10-20 16:39:48 +00:00
|
|
|
|
|
|
|
if (wait_status == std::future_status::timeout)
|
|
|
|
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Wait for async insert timeout ({} ms) exceeded)", timeout_ms);
|
|
|
|
|
|
|
|
result.future.get();
|
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
sendInsertProfileEvents(state);
|
2023-10-20 16:39:48 +00:00
|
|
|
return;
|
|
|
|
}
|
|
|
|
if (result.status == AsynchronousInsertQueue::PushResult::TOO_MUCH_DATA)
|
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "Setting async_insert=1, but INSERT query will be executed synchronously because it has too much data");
|
|
|
|
processed_block = std::move(result.insert_block);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-18 20:38:44 +00:00
|
|
|
if (num_threads > 1)
|
|
|
|
{
|
|
|
|
PushingAsyncPipelineExecutor executor(state.io.pipeline);
|
2023-10-20 16:39:48 +00:00
|
|
|
run_executor(executor, std::move(processed_block));
|
2018-11-15 15:03:13 +00:00
|
|
|
}
|
2021-09-10 14:52:24 +00:00
|
|
|
else
|
|
|
|
{
|
2021-09-15 19:35:48 +00:00
|
|
|
PushingPipelineExecutor executor(state.io.pipeline);
|
2024-03-07 17:42:50 +00:00
|
|
|
run_executor(executor, std::move(processed_block));
|
2021-09-10 14:52:24 +00:00
|
|
|
}
|
2022-05-20 15:19:40 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
sendInsertProfileEvents(state);
|
2012-05-21 06:49:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::processOrdinaryQuery(QueryState & state)
|
2019-03-26 18:28:37 +00:00
|
|
|
{
|
|
|
|
auto & pipeline = state.io.pipeline;
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
if (state.query_context->getSettingsRef()[Setting::allow_experimental_query_deduplication])
|
2022-12-07 17:00:10 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
sendPartUUIDs(state);
|
2022-12-07 17:00:10 +00:00
|
|
|
}
|
2020-11-20 17:23:53 +00:00
|
|
|
|
2019-03-26 18:28:37 +00:00
|
|
|
/// Send header-block, to allow client to prepare output format for data to send.
|
|
|
|
{
|
2020-05-14 21:03:38 +00:00
|
|
|
const auto & header = pipeline.getHeader();
|
2019-03-26 18:28:37 +00:00
|
|
|
|
|
|
|
if (header)
|
2022-12-07 17:00:10 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
sendData(state, header);
|
2022-12-07 17:00:10 +00:00
|
|
|
}
|
2019-03-26 18:28:37 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
{
|
2023-10-21 01:14:22 +00:00
|
|
|
PullingAsyncPipelineExecutor executor(pipeline);
|
2024-10-22 14:11:01 +00:00
|
|
|
pipeline.setConcurrencyControl(state.query_context->getSettingsRef()[Setting::use_concurrency_control]);
|
2020-05-14 21:03:38 +00:00
|
|
|
CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread};
|
2019-07-09 12:46:50 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
try
|
2019-03-26 18:28:37 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
Block block;
|
|
|
|
while (executor.pull(block, interactive_delay / 1000))
|
2019-03-26 18:28:37 +00:00
|
|
|
{
|
2024-10-31 11:22:26 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
receivePacketsExpectCancel(state);
|
|
|
|
|
|
|
|
if (state.stop_read_return_partial_result)
|
|
|
|
{
|
|
|
|
executor.cancelReading();
|
|
|
|
}
|
2019-03-26 18:28:37 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
{
|
2024-10-31 11:22:26 +00:00
|
|
|
std::lock_guard lock(callback_mutex);
|
2024-06-26 17:49:14 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
if (after_send_progress.elapsed() / 1000 >= interactive_delay)
|
|
|
|
{
|
|
|
|
/// Some time passed and there is a progress.
|
|
|
|
after_send_progress.restart();
|
|
|
|
sendProgress(state);
|
|
|
|
sendSelectProfileEvents(state);
|
|
|
|
}
|
2019-03-26 18:28:37 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
sendLogs(state);
|
2019-04-05 10:52:07 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
if (block)
|
|
|
|
{
|
|
|
|
if (!state.io.null_format)
|
|
|
|
sendData(state, block);
|
|
|
|
}
|
|
|
|
}
|
2019-03-26 18:28:37 +00:00
|
|
|
}
|
2020-01-23 10:04:18 +00:00
|
|
|
}
|
2024-10-22 12:26:06 +00:00
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
executor.cancel();
|
|
|
|
throw;
|
|
|
|
}
|
2023-01-17 15:17:14 +00:00
|
|
|
|
2020-01-23 10:04:18 +00:00
|
|
|
/** If data has run out, we will send the profiling data and total values to
|
|
|
|
* the last zero block to be able to use
|
|
|
|
* this information in the suffix output of stream.
|
|
|
|
* If the request was interrupted, then `sendTotals` and other methods could not be called,
|
|
|
|
* because we have not read all the data yet,
|
|
|
|
* and there could be ongoing calculations in other threads at the same time.
|
|
|
|
*/
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
receivePacketsExpectCancel(state);
|
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
sendTotals(state, executor.getTotalsBlock());
|
|
|
|
sendExtremes(state, executor.getExtremesBlock());
|
|
|
|
sendProfileInfo(state, executor.getProfileInfo());
|
|
|
|
sendProgress(state);
|
|
|
|
sendLogs(state);
|
|
|
|
sendSelectProfileEvents(state);
|
2020-01-23 10:04:18 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
sendData(state, {});
|
2020-10-24 03:41:47 +00:00
|
|
|
|
2024-10-31 11:22:26 +00:00
|
|
|
sendProgress(state);
|
2019-03-26 18:28:37 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2012-07-21 07:02:55 +00:00
|
|
|
|
2017-04-17 16:02:48 +00:00
|
|
|
void TCPHandler::processTablesStatusRequest()
|
|
|
|
{
|
|
|
|
TablesStatusRequest request;
|
2020-09-17 12:15:05 +00:00
|
|
|
request.read(*in, client_tcp_protocol_version);
|
2017-04-17 16:02:48 +00:00
|
|
|
|
2022-08-04 16:04:06 +00:00
|
|
|
ContextPtr context_to_resolve_table_names;
|
|
|
|
if (is_interserver_mode)
|
|
|
|
{
|
2024-05-17 08:23:32 +00:00
|
|
|
/// In the interserver mode session context does not exist, because authentication is done for each query.
|
2022-08-04 16:04:06 +00:00
|
|
|
/// We also cannot create query context earlier, because it cannot be created before authentication,
|
|
|
|
/// but query is not received yet. So we have to do this trick.
|
|
|
|
ContextMutablePtr fake_interserver_context = Context::createCopy(server.context());
|
|
|
|
if (!default_database.empty())
|
|
|
|
fake_interserver_context->setCurrentDatabase(default_database);
|
|
|
|
context_to_resolve_table_names = fake_interserver_context;
|
|
|
|
}
|
|
|
|
else
|
2022-07-27 16:57:56 +00:00
|
|
|
{
|
2022-08-04 16:04:06 +00:00
|
|
|
assert(session);
|
|
|
|
context_to_resolve_table_names = session->sessionContext();
|
2022-07-27 16:57:56 +00:00
|
|
|
}
|
2017-04-17 16:02:48 +00:00
|
|
|
|
|
|
|
TablesStatusResponse response;
|
|
|
|
for (const QualifiedTableName & table_name: request.tables)
|
|
|
|
{
|
2021-08-01 14:12:34 +00:00
|
|
|
auto resolved_id = context_to_resolve_table_names->tryResolveStorageID({table_name.database, table_name.table});
|
|
|
|
StoragePtr table = DatabaseCatalog::instance().tryGetTable(resolved_id, context_to_resolve_table_names);
|
2017-04-20 16:19:30 +00:00
|
|
|
if (!table)
|
|
|
|
continue;
|
|
|
|
|
2017-04-17 16:02:48 +00:00
|
|
|
TableStatus status;
|
|
|
|
if (auto * replicated_table = dynamic_cast<StorageReplicatedMergeTree *>(table.get()))
|
|
|
|
{
|
|
|
|
status.is_replicated = true;
|
2022-10-07 10:46:45 +00:00
|
|
|
status.absolute_delay = static_cast<UInt32>(replicated_table->getAbsoluteDelay());
|
2024-01-23 21:22:23 +00:00
|
|
|
status.is_readonly = replicated_table->isTableReadOnly();
|
2017-04-17 16:02:48 +00:00
|
|
|
}
|
|
|
|
else
|
2023-02-19 22:15:09 +00:00
|
|
|
status.is_replicated = false;
|
2017-04-17 16:02:48 +00:00
|
|
|
|
2022-06-07 11:53:10 +00:00
|
|
|
response.table_states_by_id.emplace(table_name, std::move(status));
|
2017-04-17 16:02:48 +00:00
|
|
|
}
|
|
|
|
|
2021-02-17 17:34:52 +00:00
|
|
|
writeVarUInt(Protocol::Server::TablesStatusResponse, *out);
|
|
|
|
|
2021-01-19 19:21:06 +00:00
|
|
|
/// For testing hedged requests
|
2022-04-28 13:10:27 +00:00
|
|
|
if (unlikely(sleep_in_send_tables_status.totalMilliseconds()))
|
2021-01-19 19:21:06 +00:00
|
|
|
{
|
2021-02-17 17:34:52 +00:00
|
|
|
out->next();
|
2021-08-01 14:12:34 +00:00
|
|
|
std::chrono::milliseconds ms(sleep_in_send_tables_status.totalMilliseconds());
|
2021-03-22 21:21:52 +00:00
|
|
|
std::this_thread::sleep_for(ms);
|
2021-01-19 19:21:06 +00:00
|
|
|
}
|
|
|
|
|
2020-09-17 12:15:05 +00:00
|
|
|
response.write(*out, client_tcp_protocol_version);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2024-10-22 12:26:06 +00:00
|
|
|
out->next();
|
2017-04-17 16:02:48 +00:00
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::processUnexpectedTablesStatusRequest()
|
2019-09-03 09:36:16 +00:00
|
|
|
{
|
|
|
|
TablesStatusRequest skip_request;
|
2020-09-17 12:15:05 +00:00
|
|
|
skip_request.read(*in, client_tcp_protocol_version);
|
2019-09-03 09:36:16 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet TablesStatusRequest received from client");
|
2019-09-03 09:36:16 +00:00
|
|
|
}
|
2017-04-17 16:02:48 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::sendPartUUIDs(QueryState & state)
|
2020-11-20 17:23:53 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
auto uuids = state.query_context->getPartUUIDs()->get();
|
|
|
|
if (uuids.empty())
|
|
|
|
return;
|
2020-11-20 17:23:53 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
for (const auto & uuid : uuids)
|
|
|
|
LOG_TRACE(log, "Sending UUID: {}", toString(uuid));
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
writeVarUInt(Protocol::Server::PartUUIDs, *out);
|
|
|
|
writeVectorBinary(uuids, *out);
|
|
|
|
|
|
|
|
out->finishChunk();
|
|
|
|
out->next();
|
2020-11-20 17:23:53 +00:00
|
|
|
}
|
|
|
|
|
2021-03-22 17:12:31 +00:00
|
|
|
|
2021-04-10 02:21:18 +00:00
|
|
|
void TCPHandler::sendReadTaskRequestAssumeLocked()
|
2021-03-22 17:12:31 +00:00
|
|
|
{
|
2021-04-06 11:05:47 +00:00
|
|
|
writeVarUInt(Protocol::Server::ReadTaskRequest, *out);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2021-03-22 17:12:31 +00:00
|
|
|
out->next();
|
|
|
|
}
|
|
|
|
|
2021-12-09 10:39:28 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::sendMergeTreeAllRangesAnnouncementAssumeLocked(QueryState &, InitialAllRangesAnnouncement announcement)
|
2023-02-03 13:34:18 +00:00
|
|
|
{
|
2023-09-04 13:53:06 +00:00
|
|
|
writeVarUInt(Protocol::Server::MergeTreeAllRangesAnnouncement, *out);
|
2024-09-12 14:40:51 +00:00
|
|
|
announcement.serialize(*out, client_parallel_replicas_protocol_version);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2023-02-03 13:34:18 +00:00
|
|
|
out->next();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest request)
|
2021-12-09 10:39:28 +00:00
|
|
|
{
|
|
|
|
writeVarUInt(Protocol::Server::MergeTreeReadTaskRequest, *out);
|
2024-09-12 14:40:51 +00:00
|
|
|
request.serialize(*out, client_parallel_replicas_protocol_version);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2021-12-09 10:39:28 +00:00
|
|
|
out->next();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::sendProfileInfo(QueryState &, const ProfileInfo & info)
|
2013-05-22 14:57:43 +00:00
|
|
|
{
|
2019-01-23 14:48:50 +00:00
|
|
|
writeVarUInt(Protocol::Server::ProfileInfo, *out);
|
2024-07-16 03:00:55 +00:00
|
|
|
info.write(*out, client_tcp_protocol_version);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2019-01-23 14:48:50 +00:00
|
|
|
out->next();
|
2013-05-22 14:57:43 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::sendTotals(QueryState & state, const Block & totals)
|
2013-09-05 20:22:43 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
if (!totals)
|
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
initBlockOutput(state, totals);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
writeVarUInt(Protocol::Server::Totals, *out);
|
|
|
|
writeStringBinary("", *out);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
state.block_out->write(totals);
|
|
|
|
state.maybe_compressed_out->next();
|
|
|
|
|
|
|
|
out->finishChunk();
|
|
|
|
out->next();
|
2013-09-05 20:22:43 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::sendExtremes(QueryState & state, const Block & extremes)
|
2013-09-07 02:03:13 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
if (!extremes)
|
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
initBlockOutput(state, extremes);
|
|
|
|
|
|
|
|
writeVarUInt(Protocol::Server::Extremes, *out);
|
|
|
|
writeStringBinary("", *out);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
state.block_out->write(extremes);
|
|
|
|
state.maybe_compressed_out->next();
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
out->finishChunk();
|
|
|
|
out->next();
|
2013-09-07 02:03:13 +00:00
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::sendProfileEvents(QueryState & state)
|
2021-08-30 11:04:59 +00:00
|
|
|
{
|
2024-04-18 10:37:09 +00:00
|
|
|
Stopwatch stopwatch;
|
2024-10-22 12:26:06 +00:00
|
|
|
Block block = ProfileEvents::getProfileEvents(host_name, state.profile_queue, state.last_sent_snapshots);
|
2022-03-05 06:22:56 +00:00
|
|
|
if (block.rows() != 0)
|
2021-09-07 12:07:24 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
initProfileEventsBlockOutput(state, block);
|
2021-09-02 14:27:19 +00:00
|
|
|
|
|
|
|
writeVarUInt(Protocol::Server::ProfileEvents, *out);
|
|
|
|
writeStringBinary("", *out);
|
|
|
|
|
|
|
|
state.profile_events_block_out->write(block);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2021-09-02 14:27:19 +00:00
|
|
|
out->next();
|
2024-04-18 10:37:09 +00:00
|
|
|
|
2024-04-18 13:39:23 +00:00
|
|
|
auto elapsed_milliseconds = stopwatch.elapsedMilliseconds();
|
|
|
|
if (elapsed_milliseconds > 100)
|
|
|
|
LOG_DEBUG(log, "Sending profile events block with {} rows, {} bytes took {} milliseconds",
|
|
|
|
block.rows(), block.bytes(), elapsed_milliseconds);
|
2021-09-02 14:27:19 +00:00
|
|
|
}
|
2021-08-30 11:04:59 +00:00
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::sendSelectProfileEvents(QueryState & state)
|
2022-05-20 15:19:40 +00:00
|
|
|
{
|
|
|
|
if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS)
|
|
|
|
return;
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
sendProfileEvents(state);
|
2022-05-20 15:19:40 +00:00
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::sendInsertProfileEvents(QueryState & state)
|
2022-05-20 15:19:40 +00:00
|
|
|
{
|
|
|
|
if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS_IN_INSERT)
|
|
|
|
return;
|
2022-06-22 11:55:54 +00:00
|
|
|
if (query_kind != ClientInfo::QueryKind::INITIAL_QUERY)
|
|
|
|
return;
|
2022-05-20 15:19:40 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
sendProfileEvents(state);
|
2022-05-20 15:19:40 +00:00
|
|
|
}
|
2021-08-30 11:04:59 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::sendTimezone(QueryState & state)
|
2023-03-07 15:05:23 +00:00
|
|
|
{
|
2023-04-12 15:45:11 +00:00
|
|
|
if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_TIMEZONE_UPDATES)
|
|
|
|
return;
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
const String & tz = state.query_context->getSettingsRef()[Setting::session_timezone].value;
|
2023-04-12 15:45:11 +00:00
|
|
|
|
|
|
|
LOG_DEBUG(log, "TCPHandler::sendTimezone(): {}", tz);
|
|
|
|
writeVarUInt(Protocol::Server::TimezoneUpdate, *out);
|
|
|
|
writeStringBinary(tz, *out);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2023-04-12 15:45:11 +00:00
|
|
|
out->next();
|
2023-03-07 15:05:23 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-12-02 21:05:51 +00:00
|
|
|
bool TCPHandler::receiveProxyHeader()
|
|
|
|
{
|
|
|
|
if (in->eof())
|
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Client has not sent any data.");
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
String forwarded_address;
|
|
|
|
|
|
|
|
/// Only PROXYv1 is supported.
|
|
|
|
/// Validation of protocol is not fully performed.
|
|
|
|
|
2024-10-01 14:38:14 +00:00
|
|
|
LimitReadBuffer limit_in(*in, {.read_no_more=107, .expect_eof=true}); /// Maximum length from the specs.
|
2020-12-02 21:05:51 +00:00
|
|
|
|
|
|
|
assertString("PROXY ", limit_in);
|
|
|
|
|
|
|
|
if (limit_in.eof())
|
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Incomplete PROXY header is received.");
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// TCP4 / TCP6 / UNKNOWN
|
|
|
|
if ('T' == *limit_in.position())
|
|
|
|
{
|
|
|
|
assertString("TCP", limit_in);
|
|
|
|
|
|
|
|
if (limit_in.eof())
|
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Incomplete PROXY header is received.");
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
if ('4' != *limit_in.position() && '6' != *limit_in.position())
|
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Unexpected protocol in PROXY header is received.");
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
++limit_in.position();
|
|
|
|
assertChar(' ', limit_in);
|
|
|
|
|
|
|
|
/// Read the first field and ignore other.
|
|
|
|
readStringUntilWhitespace(forwarded_address, limit_in);
|
|
|
|
|
|
|
|
/// Skip until \r\n
|
|
|
|
while (!limit_in.eof() && *limit_in.position() != '\r')
|
|
|
|
++limit_in.position();
|
|
|
|
assertString("\r\n", limit_in);
|
|
|
|
}
|
|
|
|
else if (checkString("UNKNOWN", limit_in))
|
|
|
|
{
|
|
|
|
/// This is just a health check, there is no subsequent data in this connection.
|
|
|
|
|
|
|
|
while (!limit_in.eof() && *limit_in.position() != '\r')
|
|
|
|
++limit_in.position();
|
|
|
|
assertString("\r\n", limit_in);
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Unexpected protocol in PROXY header is received.");
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Forwarded client address from PROXY header: {}", forwarded_address);
|
2022-05-21 00:05:02 +00:00
|
|
|
forwarded_for = std::move(forwarded_address);
|
2020-12-02 21:05:51 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-08-20 09:56:39 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
|
|
|
std::string formatHTTPErrorResponseWhenUserIsConnectedToWrongPort(const Poco::Util::AbstractConfiguration& config)
|
|
|
|
{
|
|
|
|
std::string result = fmt::format(
|
|
|
|
"HTTP/1.0 400 Bad Request\r\n\r\n"
|
|
|
|
"Port {} is for clickhouse-client program\r\n",
|
|
|
|
config.getString("tcp_port"));
|
|
|
|
|
|
|
|
if (config.has("http_port"))
|
|
|
|
{
|
|
|
|
result += fmt::format(
|
|
|
|
"You must use port {} for HTTP.\r\n",
|
|
|
|
config.getString("http_port"));
|
|
|
|
}
|
|
|
|
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2022-05-21 00:05:02 +00:00
|
|
|
std::unique_ptr<Session> TCPHandler::makeSession()
|
|
|
|
{
|
|
|
|
auto interface = is_interserver_mode ? ClientInfo::Interface::TCP_INTERSERVER : ClientInfo::Interface::TCP;
|
|
|
|
|
2022-09-18 07:11:52 +00:00
|
|
|
auto res = std::make_unique<Session>(server.context(), interface, socket().secure(), certificate);
|
2022-05-21 00:05:02 +00:00
|
|
|
|
2023-07-07 10:49:50 +00:00
|
|
|
res->setForwardedFor(forwarded_for);
|
|
|
|
res->setClientName(client_name);
|
|
|
|
res->setClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version);
|
|
|
|
res->setConnectionClientVersion(client_version_major, client_version_minor, client_version_patch, client_tcp_protocol_version);
|
|
|
|
res->setClientInterface(interface);
|
2022-05-21 00:05:02 +00:00
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
2021-08-20 09:56:39 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2012-05-21 06:49:05 +00:00
|
|
|
void TCPHandler::receiveHello()
|
2012-05-16 18:03:00 +00:00
|
|
|
{
|
2017-03-09 00:56:38 +00:00
|
|
|
/// Receive `hello` packet.
|
2012-05-16 18:03:00 +00:00
|
|
|
UInt64 packet_type = 0;
|
2020-09-14 21:55:43 +00:00
|
|
|
String user;
|
2013-08-10 09:04:45 +00:00
|
|
|
String password;
|
2022-09-19 02:01:09 +00:00
|
|
|
String default_db;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2012-05-21 06:49:05 +00:00
|
|
|
readVarUInt(packet_type, *in);
|
2023-09-26 15:50:19 +00:00
|
|
|
|
2012-05-16 18:03:00 +00:00
|
|
|
if (packet_type != Protocol::Client::Hello)
|
2014-03-02 19:52:55 +00:00
|
|
|
{
|
2017-03-09 00:56:38 +00:00
|
|
|
/** If you accidentally accessed the HTTP protocol for a port destined for an internal TCP protocol,
|
2017-04-17 16:02:48 +00:00
|
|
|
* Then instead of the packet type, there will be G (GET) or P (POST), in most cases.
|
2014-03-02 19:52:55 +00:00
|
|
|
*/
|
|
|
|
if (packet_type == 'G' || packet_type == 'P')
|
|
|
|
{
|
2021-08-20 09:56:39 +00:00
|
|
|
writeString(formatHTTPErrorResponseWhenUserIsConnectedToWrongPort(server.config()), *out);
|
2024-09-26 08:27:37 +00:00
|
|
|
out->next();
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::CLIENT_HAS_CONNECTED_TO_WRONG_PORT, "Client has connected to wrong port");
|
2014-03-02 19:52:55 +00:00
|
|
|
}
|
2024-10-22 12:26:06 +00:00
|
|
|
else
|
|
|
|
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT,
|
|
|
|
"Unexpected packet from client (expected Hello, got {})", packet_type);
|
2014-03-02 19:52:55 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2012-05-21 06:49:05 +00:00
|
|
|
readStringBinary(client_name, *in);
|
|
|
|
readVarUInt(client_version_major, *in);
|
|
|
|
readVarUInt(client_version_minor, *in);
|
2018-07-31 21:36:18 +00:00
|
|
|
// NOTE For backward compatibility of the protocol, client cannot send its version_patch.
|
2020-09-17 12:15:05 +00:00
|
|
|
readVarUInt(client_tcp_protocol_version, *in);
|
2022-09-19 02:01:09 +00:00
|
|
|
readStringBinary(default_db, *in);
|
|
|
|
if (!default_db.empty())
|
|
|
|
default_database = default_db;
|
2016-07-31 03:53:16 +00:00
|
|
|
readStringBinary(user, *in);
|
|
|
|
readStringBinary(password, *in);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-09-14 21:55:43 +00:00
|
|
|
if (user.empty())
|
2024-10-22 12:26:06 +00:00
|
|
|
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet from client (no user in Hello package)");
|
2020-09-14 21:55:43 +00:00
|
|
|
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_DEBUG(log, "Connected {} version {}.{}.{}, revision: {}{}{}.",
|
2020-05-23 22:21:29 +00:00
|
|
|
client_name,
|
|
|
|
client_version_major, client_version_minor, client_version_patch,
|
2020-09-17 12:15:05 +00:00
|
|
|
client_tcp_protocol_version,
|
2020-05-23 22:21:29 +00:00
|
|
|
(!default_database.empty() ? ", database: " + default_database : ""),
|
2020-09-14 21:55:43 +00:00
|
|
|
(!user.empty() ? ", user: " + user : "")
|
|
|
|
);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2023-09-26 15:50:19 +00:00
|
|
|
is_interserver_mode = (user == EncodedUserInfo::USER_INTERSERVER_MARKER) && password.empty();
|
2021-08-01 14:12:34 +00:00
|
|
|
if (is_interserver_mode)
|
2020-09-14 21:55:43 +00:00
|
|
|
{
|
2023-04-14 17:46:44 +00:00
|
|
|
if (client_tcp_protocol_version < DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2)
|
|
|
|
LOG_WARNING(LogFrequencyLimiter(log, 10),
|
|
|
|
"Using deprecated interserver protocol because the client is too old. Consider upgrading all nodes in cluster.");
|
2024-10-22 12:26:06 +00:00
|
|
|
processClusterNameAndSalt();
|
2021-08-01 14:12:34 +00:00
|
|
|
return;
|
2020-09-14 21:55:43 +00:00
|
|
|
}
|
2021-08-01 14:12:34 +00:00
|
|
|
|
2024-04-04 20:58:35 +00:00
|
|
|
is_ssh_based_auth = user.starts_with(EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER) && password.empty();
|
2023-09-26 15:50:19 +00:00
|
|
|
if (is_ssh_based_auth)
|
2024-04-04 20:58:35 +00:00
|
|
|
user.erase(0, std::string_view(EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER).size());
|
2023-09-26 15:50:19 +00:00
|
|
|
|
2022-05-21 00:05:02 +00:00
|
|
|
session = makeSession();
|
2023-07-07 10:49:50 +00:00
|
|
|
const auto & client_info = session->getClientInfo();
|
2023-03-14 22:10:08 +00:00
|
|
|
|
2023-03-15 00:09:29 +00:00
|
|
|
#if USE_SSL
|
2023-03-14 22:49:10 +00:00
|
|
|
/// Authentication with SSL user certificate
|
2023-03-14 22:10:08 +00:00
|
|
|
if (dynamic_cast<Poco::Net::SecureStreamSocketImpl*>(socket().impl()))
|
|
|
|
{
|
|
|
|
Poco::Net::SecureStreamSocket secure_socket(socket());
|
|
|
|
if (secure_socket.havePeerCertificate())
|
|
|
|
{
|
2023-04-20 20:02:05 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
session->authenticate(
|
2024-06-03 15:38:40 +00:00
|
|
|
SSLCertificateCredentials{user, extractSSLCertificateSubjects(secure_socket.peerCertificate())},
|
2023-04-20 20:02:05 +00:00
|
|
|
getClientAddress(client_info));
|
|
|
|
return;
|
|
|
|
}
|
2023-11-18 17:55:38 +00:00
|
|
|
catch (const Exception & e)
|
2023-04-20 20:02:05 +00:00
|
|
|
{
|
2023-11-18 17:55:38 +00:00
|
|
|
if (e.code() != DB::ErrorCodes::AUTHENTICATION_FAILED)
|
|
|
|
throw;
|
|
|
|
|
2024-11-04 17:26:02 +00:00
|
|
|
tryLogCurrentException(log, "SSL authentication failed, falling back to password authentication", LogsLevel::information);
|
2024-11-03 19:16:35 +00:00
|
|
|
/// ^^ Log at debug level instead of default error level as authentication failures are not an unusual event.
|
2023-04-20 20:02:05 +00:00
|
|
|
}
|
2023-03-14 22:10:08 +00:00
|
|
|
}
|
|
|
|
}
|
2024-04-04 20:58:35 +00:00
|
|
|
#endif
|
2023-09-26 15:50:19 +00:00
|
|
|
|
2024-04-04 20:58:35 +00:00
|
|
|
#if USE_SSH
|
2023-09-26 15:50:19 +00:00
|
|
|
/// Perform handshake for SSH authentication
|
|
|
|
if (is_ssh_based_auth)
|
|
|
|
{
|
2024-06-20 18:07:16 +00:00
|
|
|
const auto authentication_types = session->getAuthenticationTypesOrLogInFailure(user);
|
|
|
|
|
|
|
|
bool user_supports_ssh_authentication = std::find_if(
|
|
|
|
authentication_types.begin(),
|
|
|
|
authentication_types.end(),
|
|
|
|
[](auto authentication_type)
|
|
|
|
{
|
|
|
|
return authentication_type == AuthenticationType::SSH_KEY;
|
|
|
|
}) != authentication_types.end();
|
|
|
|
|
|
|
|
if (!user_supports_ssh_authentication)
|
2023-09-26 15:50:19 +00:00
|
|
|
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Expected authentication with SSH key");
|
|
|
|
|
|
|
|
if (client_tcp_protocol_version < DBMS_MIN_REVISION_WITH_SSH_AUTHENTICATION)
|
|
|
|
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot authenticate user with SSH key, because client version is too old");
|
|
|
|
|
|
|
|
readVarUInt(packet_type, *in);
|
|
|
|
if (packet_type != Protocol::Client::SSHChallengeRequest)
|
|
|
|
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Server expected to receive a packet for requesting a challenge string");
|
|
|
|
|
2024-04-04 20:58:35 +00:00
|
|
|
auto create_challenge = []()
|
|
|
|
{
|
|
|
|
pcg64_fast rng(randomSeed());
|
|
|
|
UInt64 rand = rng();
|
|
|
|
return encodeSHA256(&rand, sizeof(rand));
|
|
|
|
};
|
|
|
|
|
|
|
|
String challenge = create_challenge();
|
2023-09-26 15:50:19 +00:00
|
|
|
writeVarUInt(Protocol::Server::SSHChallenge, *out);
|
|
|
|
writeStringBinary(challenge, *out);
|
|
|
|
out->next();
|
|
|
|
|
|
|
|
String signature;
|
|
|
|
readVarUInt(packet_type, *in);
|
|
|
|
if (packet_type != Protocol::Client::SSHChallengeResponse)
|
|
|
|
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Server expected to receive a packet with a response for a challenge");
|
|
|
|
readStringBinary(signature, *in);
|
|
|
|
|
2024-04-04 20:58:35 +00:00
|
|
|
auto prepare_string_for_ssh_validation = [&](const String & username, const String & challenge_)
|
|
|
|
{
|
|
|
|
String output;
|
|
|
|
output.append(std::to_string(client_tcp_protocol_version));
|
|
|
|
output.append(default_database);
|
|
|
|
output.append(username);
|
|
|
|
output.append(challenge_);
|
|
|
|
return output;
|
|
|
|
};
|
|
|
|
|
|
|
|
auto cred = SshCredentials(user, signature, prepare_string_for_ssh_validation(user, challenge));
|
2023-09-26 15:50:19 +00:00
|
|
|
session->authenticate(cred, getClientAddress(client_info));
|
|
|
|
return;
|
|
|
|
}
|
2023-03-15 00:09:29 +00:00
|
|
|
#endif
|
2023-03-14 22:10:08 +00:00
|
|
|
|
2023-03-03 15:25:42 +00:00
|
|
|
session->authenticate(user, password, getClientAddress(client_info));
|
2012-05-16 18:03:00 +00:00
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2022-08-03 19:44:08 +00:00
|
|
|
void TCPHandler::receiveAddendum()
|
|
|
|
{
|
2022-08-03 20:36:52 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_QUOTA_KEY)
|
|
|
|
readStringBinary(quota_key, *in);
|
2023-07-30 22:09:03 +00:00
|
|
|
|
|
|
|
if (!is_interserver_mode)
|
|
|
|
session->setQuotaClientKey(quota_key);
|
2024-05-23 22:01:32 +00:00
|
|
|
|
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS)
|
|
|
|
{
|
|
|
|
readStringBinary(proto_send_chunked_cl, *in);
|
|
|
|
readStringBinary(proto_recv_chunked_cl, *in);
|
|
|
|
}
|
2024-09-12 14:40:51 +00:00
|
|
|
|
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL)
|
|
|
|
readVarUInt(client_parallel_replicas_protocol_version, *in);
|
2022-08-03 19:44:08 +00:00
|
|
|
}
|
|
|
|
|
2012-05-16 18:03:00 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::processUnexpectedHello()
|
2019-09-03 09:36:16 +00:00
|
|
|
{
|
|
|
|
UInt64 skip_uint_64;
|
|
|
|
String skip_string;
|
|
|
|
|
|
|
|
readStringBinary(skip_string, *in);
|
|
|
|
readVarUInt(skip_uint_64, *in);
|
|
|
|
readVarUInt(skip_uint_64, *in);
|
2020-04-15 01:58:10 +00:00
|
|
|
readVarUInt(skip_uint_64, *in);
|
2019-09-03 09:36:16 +00:00
|
|
|
readStringBinary(skip_string, *in);
|
|
|
|
readStringBinary(skip_string, *in);
|
|
|
|
readStringBinary(skip_string, *in);
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet Hello received from client");
|
2019-09-03 09:36:16 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2012-05-21 06:49:05 +00:00
|
|
|
void TCPHandler::sendHello()
|
2012-03-11 08:52:56 +00:00
|
|
|
{
|
2012-05-21 06:49:05 +00:00
|
|
|
writeVarUInt(Protocol::Server::Hello, *out);
|
2023-08-09 03:02:50 +00:00
|
|
|
writeStringBinary(VERSION_NAME, *out);
|
|
|
|
writeVarUInt(VERSION_MAJOR, *out);
|
|
|
|
writeVarUInt(VERSION_MINOR, *out);
|
2020-09-17 12:15:05 +00:00
|
|
|
writeVarUInt(DBMS_TCP_PROTOCOL_VERSION, *out);
|
2024-09-12 14:40:51 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL)
|
|
|
|
writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, *out);
|
2020-09-17 12:15:05 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE)
|
2016-11-15 10:11:05 +00:00
|
|
|
writeStringBinary(DateLUT::instance().getTimeZone(), *out);
|
2020-09-17 12:15:05 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME)
|
2018-03-08 07:36:58 +00:00
|
|
|
writeStringBinary(server_display_name, *out);
|
2020-09-17 12:15:05 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSION_PATCH)
|
2023-08-09 03:02:50 +00:00
|
|
|
writeVarUInt(VERSION_PATCH, *out);
|
2024-05-23 22:01:32 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS)
|
|
|
|
{
|
2024-07-10 00:20:11 +00:00
|
|
|
writeStringBinary(server.config().getString("proto_caps.send", "notchunked"), *out);
|
|
|
|
writeStringBinary(server.config().getString("proto_caps.recv", "notchunked"), *out);
|
2024-05-23 22:01:32 +00:00
|
|
|
}
|
2022-12-11 23:47:43 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES)
|
|
|
|
{
|
|
|
|
auto rules = server.context()->getAccessControl().getPasswordComplexityRules();
|
|
|
|
|
|
|
|
writeVarUInt(rules.size(), *out);
|
|
|
|
for (const auto & [original_pattern, exception_message] : rules)
|
|
|
|
{
|
|
|
|
writeStringBinary(original_pattern, *out);
|
|
|
|
writeStringBinary(exception_message, *out);
|
|
|
|
}
|
|
|
|
}
|
2023-03-03 13:52:54 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2)
|
|
|
|
{
|
|
|
|
chassert(!nonce.has_value());
|
|
|
|
/// Contains lots of stuff (including time), so this should be enough for NONCE.
|
|
|
|
nonce.emplace(thread_local_rng());
|
|
|
|
writeIntBinary(nonce.value(), *out);
|
|
|
|
}
|
2012-05-21 06:49:05 +00:00
|
|
|
out->next();
|
2012-03-11 08:52:56 +00:00
|
|
|
}
|
|
|
|
|
2012-05-17 19:15:53 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::processIgnoredPartUUIDs()
|
2020-11-20 17:23:53 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
readVectorBinary(part_uuids_to_ignore.emplace(), *in);
|
2021-08-01 14:12:34 +00:00
|
|
|
}
|
2020-11-20 17:23:53 +00:00
|
|
|
|
2023-03-15 19:53:58 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::processUnexpectedIgnoredPartUUIDs()
|
2021-08-01 14:12:34 +00:00
|
|
|
{
|
|
|
|
std::vector<UUID> skip_part_uuids;
|
|
|
|
readVectorBinary(skip_part_uuids, *in);
|
2024-10-22 12:26:06 +00:00
|
|
|
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet IgnoredPartUUIDs received from client");
|
2020-11-20 17:23:53 +00:00
|
|
|
}
|
|
|
|
|
2023-03-15 19:53:58 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
String TCPHandler::receiveReadTaskResponseAssumeLocked(QueryState & state)
|
2021-04-06 11:05:47 +00:00
|
|
|
{
|
|
|
|
UInt64 packet_type = 0;
|
|
|
|
readVarUInt(packet_type, *in);
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
LOG_DEBUG(log, "receiveReadTaskResponseAssumeLocked got {}", Protocol::Client::toString(packet_type));
|
|
|
|
|
|
|
|
switch (packet_type)
|
2021-04-10 02:21:18 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
case Protocol::Client::Cancel:
|
|
|
|
processCancel(state);
|
2021-04-10 02:21:18 +00:00
|
|
|
return {};
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
case Protocol::Client::ReadTaskResponse:
|
|
|
|
{
|
|
|
|
UInt64 version = 0;
|
|
|
|
readVarUInt(version, *in);
|
|
|
|
if (version != DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION)
|
|
|
|
throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol version for distributed processing mismatched");
|
|
|
|
String response;
|
|
|
|
readStringBinary(response, *in);
|
|
|
|
return response;
|
2021-04-10 02:21:18 +00:00
|
|
|
}
|
2024-09-19 11:51:02 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
default:
|
|
|
|
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Received {} packet after requesting read task",
|
|
|
|
Protocol::Client::toString(packet_type));
|
2021-04-10 02:21:18 +00:00
|
|
|
}
|
2021-04-06 11:05:47 +00:00
|
|
|
}
|
|
|
|
|
2023-03-16 14:23:17 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
std::optional<ParallelReadResponse> TCPHandler::receivePartitionMergeTreeReadTaskResponseAssumeLocked(QueryState & state)
|
2021-12-09 10:39:28 +00:00
|
|
|
{
|
|
|
|
UInt64 packet_type = 0;
|
|
|
|
readVarUInt(packet_type, *in);
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
LOG_DEBUG(log, "receivePartitionMergeTreeReadTaskResponseAssumeLocked got {}", Protocol::Client::toString(packet_type));
|
|
|
|
|
|
|
|
switch (packet_type)
|
2021-12-09 10:39:28 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
case Protocol::Client::Cancel:
|
|
|
|
processCancel(state);
|
|
|
|
return {};
|
|
|
|
|
|
|
|
case Protocol::Client::MergeTreeReadTaskResponse:
|
2021-12-09 10:39:28 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
ParallelReadResponse response;
|
|
|
|
response.deserialize(*in);
|
|
|
|
return response;
|
2021-12-09 10:39:28 +00:00
|
|
|
}
|
2024-09-19 11:51:02 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
default:
|
|
|
|
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT,
|
|
|
|
"Received {} packet after requesting read task",
|
|
|
|
Protocol::Client::toString(packet_type));
|
2021-12-09 10:39:28 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-03-15 19:53:58 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::processClusterNameAndSalt()
|
2020-09-14 21:55:43 +00:00
|
|
|
{
|
2021-04-06 19:18:45 +00:00
|
|
|
readStringBinary(cluster, *in);
|
|
|
|
readStringBinary(salt, *in, 32);
|
2020-09-14 21:55:43 +00:00
|
|
|
}
|
2012-05-17 19:15:53 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::processQuery(std::optional<QueryState> & state)
|
2012-03-11 08:52:56 +00:00
|
|
|
{
|
|
|
|
UInt64 stage = 0;
|
|
|
|
UInt64 compression = 0;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
state.emplace();
|
|
|
|
|
|
|
|
if (part_uuids_to_ignore.has_value())
|
|
|
|
state->part_uuids_to_ignore = std::move(part_uuids_to_ignore);
|
|
|
|
|
|
|
|
readStringBinary(state->query_id, *in);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-05-17 16:22:52 +00:00
|
|
|
/// In interserver mode,
|
Do not allow to reuse previous credentials in case of inter-server secret
Before this patch INSERT via Buffer/Kafka may re-use previously set user
for that connection, while this is not correct, it should reset the
user, and use global context.
Note, before [1] there was a fallback to default user, but that code had
been removed, and now it got back.
[1]: 0159c74f217ec764060c480819e3ccc9d5a99a63 ("Secure inter-cluster query execution (with initial_user as current query user) [v3]")
Also note, that context for Buffer table (and others) cannot be changed,
since they don't have any user only profile.
I've tested this patch manually using the following:
create table dist (key Int) engine=Distributed(test_cluster_two_shards_secure, default, data, key);
create table buffer (key Int) engine=Buffer(default, dist, 1, 0, 0, 0, 0, 0, 0);
create table data (key Int) engine=Memory();
# to start the connection with readonly user
$ clickhouse-client --user readonly -q 'select * from dist'
$ clickhouse-client -q 'insert into buffer values (1)'
# before this patch this produces errors like:
# 2021.09.27 23:46:48.384920 [ 19474 ] {} <Error> default.dist.DirectoryMonitor: Code: 164. DB::Exception: Received from 127.0.0.2:9000. DB::Exception: readonly: Cannot execute query in readonly mode. Stack trace:
v2: reset the authentication instead of using default user (as suggested by @vitlibar)
v3: reset Session::user and introduce ClientInfo::resetAuthentication (as suggested by @vitlibar)
v4: reset the session every time in interserver mode (suggested by @vitlibar)
2021-09-30 07:15:38 +00:00
|
|
|
/// initial_user can be empty in case of Distributed INSERT via Buffer/Kafka,
|
2022-04-17 23:02:49 +00:00
|
|
|
/// (i.e. when the INSERT is done with the global context without user),
|
Do not allow to reuse previous credentials in case of inter-server secret
Before this patch INSERT via Buffer/Kafka may re-use previously set user
for that connection, while this is not correct, it should reset the
user, and use global context.
Note, before [1] there was a fallback to default user, but that code had
been removed, and now it got back.
[1]: 0159c74f217ec764060c480819e3ccc9d5a99a63 ("Secure inter-cluster query execution (with initial_user as current query user) [v3]")
Also note, that context for Buffer table (and others) cannot be changed,
since they don't have any user only profile.
I've tested this patch manually using the following:
create table dist (key Int) engine=Distributed(test_cluster_two_shards_secure, default, data, key);
create table buffer (key Int) engine=Buffer(default, dist, 1, 0, 0, 0, 0, 0, 0);
create table data (key Int) engine=Memory();
# to start the connection with readonly user
$ clickhouse-client --user readonly -q 'select * from dist'
$ clickhouse-client -q 'insert into buffer values (1)'
# before this patch this produces errors like:
# 2021.09.27 23:46:48.384920 [ 19474 ] {} <Error> default.dist.DirectoryMonitor: Code: 164. DB::Exception: Received from 127.0.0.2:9000. DB::Exception: readonly: Cannot execute query in readonly mode. Stack trace:
v2: reset the authentication instead of using default user (as suggested by @vitlibar)
v3: reset Session::user and introduce ClientInfo::resetAuthentication (as suggested by @vitlibar)
v4: reset the session every time in interserver mode (suggested by @vitlibar)
2021-09-30 07:15:38 +00:00
|
|
|
/// so it is better to reset session to avoid using old user.
|
|
|
|
if (is_interserver_mode)
|
|
|
|
{
|
2022-05-21 00:05:02 +00:00
|
|
|
session = makeSession();
|
Do not allow to reuse previous credentials in case of inter-server secret
Before this patch INSERT via Buffer/Kafka may re-use previously set user
for that connection, while this is not correct, it should reset the
user, and use global context.
Note, before [1] there was a fallback to default user, but that code had
been removed, and now it got back.
[1]: 0159c74f217ec764060c480819e3ccc9d5a99a63 ("Secure inter-cluster query execution (with initial_user as current query user) [v3]")
Also note, that context for Buffer table (and others) cannot be changed,
since they don't have any user only profile.
I've tested this patch manually using the following:
create table dist (key Int) engine=Distributed(test_cluster_two_shards_secure, default, data, key);
create table buffer (key Int) engine=Buffer(default, dist, 1, 0, 0, 0, 0, 0, 0);
create table data (key Int) engine=Memory();
# to start the connection with readonly user
$ clickhouse-client --user readonly -q 'select * from dist'
$ clickhouse-client -q 'insert into buffer values (1)'
# before this patch this produces errors like:
# 2021.09.27 23:46:48.384920 [ 19474 ] {} <Error> default.dist.DirectoryMonitor: Code: 164. DB::Exception: Received from 127.0.0.2:9000. DB::Exception: readonly: Cannot execute query in readonly mode. Stack trace:
v2: reset the authentication instead of using default user (as suggested by @vitlibar)
v3: reset Session::user and introduce ClientInfo::resetAuthentication (as suggested by @vitlibar)
v4: reset the session every time in interserver mode (suggested by @vitlibar)
2021-09-30 07:15:38 +00:00
|
|
|
}
|
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
/// Read client info.
|
|
|
|
ClientInfo client_info = session->getClientInfo();
|
2020-09-17 12:15:05 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_CLIENT_INFO)
|
2023-08-22 03:52:57 +00:00
|
|
|
{
|
2020-09-17 12:15:05 +00:00
|
|
|
client_info.read(*in, client_tcp_protocol_version);
|
2023-08-29 00:15:55 +00:00
|
|
|
|
2023-08-30 06:14:39 +00:00
|
|
|
correctQueryClientInfo(session->getClientInfo(), client_info);
|
2023-08-29 00:15:55 +00:00
|
|
|
const auto & config_ref = Context::getGlobalContextInstance()->getServerSettings();
|
2024-10-16 19:13:26 +00:00
|
|
|
if (config_ref[ServerSetting::validate_tcp_client_information])
|
2023-08-29 00:15:55 +00:00
|
|
|
validateClientInfo(session->getClientInfo(), client_info);
|
2023-08-22 03:52:57 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-02-28 18:55:21 +00:00
|
|
|
/// Per query settings are also passed via TCP.
|
|
|
|
/// We need to check them before applying due to they can violate the settings constraints.
|
2021-05-20 13:21:42 +00:00
|
|
|
auto settings_format = (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS)
|
|
|
|
? SettingsWriteFormat::STRINGS_WITH_FLAGS
|
|
|
|
: SettingsWriteFormat::BINARY;
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2020-02-28 18:55:21 +00:00
|
|
|
Settings passed_settings;
|
2020-07-20 09:57:17 +00:00
|
|
|
passed_settings.read(*in, settings_format);
|
2020-09-14 21:55:43 +00:00
|
|
|
|
|
|
|
/// Interserver secret.
|
|
|
|
std::string received_hash;
|
2020-09-17 14:55:41 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET)
|
2020-09-14 21:55:43 +00:00
|
|
|
{
|
|
|
|
readStringBinary(received_hash, *in, 32);
|
|
|
|
}
|
|
|
|
|
|
|
|
readVarUInt(stage, *in);
|
2024-10-22 12:26:06 +00:00
|
|
|
state->stage = QueryProcessingStage::Enum(stage);
|
2020-09-14 21:55:43 +00:00
|
|
|
|
|
|
|
readVarUInt(compression, *in);
|
2024-10-22 12:26:06 +00:00
|
|
|
state->compression = static_cast<Protocol::Compression>(compression);
|
|
|
|
last_block_in.compression = state->compression;
|
2020-09-14 21:55:43 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
readStringBinary(state->query, *in);
|
2020-09-14 21:55:43 +00:00
|
|
|
|
2022-08-12 12:28:35 +00:00
|
|
|
Settings passed_params;
|
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PARAMETERS)
|
|
|
|
passed_params.read(*in, settings_format);
|
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
if (is_interserver_mode)
|
2020-09-14 21:55:43 +00:00
|
|
|
{
|
2022-05-18 12:06:52 +00:00
|
|
|
client_info.interface = ClientInfo::Interface::TCP_INTERSERVER;
|
2020-09-14 21:55:43 +00:00
|
|
|
#if USE_SSL
|
2023-11-15 14:04:07 +00:00
|
|
|
|
|
|
|
String cluster_secret;
|
|
|
|
try
|
|
|
|
{
|
|
|
|
cluster_secret = server.context()->getCluster(cluster)->getSecret();
|
|
|
|
}
|
|
|
|
catch (const Exception & e)
|
|
|
|
{
|
|
|
|
auto exception = Exception::createRuntime(ErrorCodes::AUTHENTICATION_FAILED, e.message());
|
|
|
|
session->onAuthenticationFailure(/* user_name= */ std::nullopt, socket().peerAddress(), exception);
|
|
|
|
throw exception; /// NOLINT
|
|
|
|
}
|
2023-03-03 13:52:54 +00:00
|
|
|
|
2022-05-18 12:06:52 +00:00
|
|
|
if (salt.empty() || cluster_secret.empty())
|
2022-05-17 16:22:52 +00:00
|
|
|
{
|
2023-03-03 13:52:54 +00:00
|
|
|
auto exception = Exception(ErrorCodes::AUTHENTICATION_FAILED, "Interserver authentication failed (no salt/cluster secret)");
|
|
|
|
session->onAuthenticationFailure(/* user_name= */ std::nullopt, socket().peerAddress(), exception);
|
|
|
|
throw exception; /// NOLINT
|
|
|
|
}
|
|
|
|
|
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET_V2 && !nonce.has_value())
|
|
|
|
{
|
|
|
|
auto exception = Exception(ErrorCodes::AUTHENTICATION_FAILED, "Interserver authentication failed (no nonce)");
|
|
|
|
session->onAuthenticationFailure(/* user_name= */ std::nullopt, socket().peerAddress(), exception);
|
2022-05-18 19:57:20 +00:00
|
|
|
throw exception; /// NOLINT
|
2022-05-17 16:22:52 +00:00
|
|
|
}
|
|
|
|
|
2020-09-14 21:55:43 +00:00
|
|
|
std::string data(salt);
|
2023-03-03 13:52:54 +00:00
|
|
|
// For backward compatibility
|
|
|
|
if (nonce.has_value())
|
|
|
|
data += std::to_string(nonce.value());
|
2020-09-14 21:55:43 +00:00
|
|
|
data += cluster_secret;
|
2024-10-22 12:26:06 +00:00
|
|
|
data += state->query;
|
|
|
|
data += state->query_id;
|
2020-09-14 21:55:43 +00:00
|
|
|
data += client_info.initial_user;
|
|
|
|
|
2020-09-10 18:24:53 +00:00
|
|
|
std::string calculated_hash = encodeSHA256(data);
|
2022-05-17 16:22:52 +00:00
|
|
|
assert(calculated_hash.size() == 32);
|
2020-09-14 21:55:43 +00:00
|
|
|
|
2022-05-17 16:22:52 +00:00
|
|
|
/// TODO maybe also check that peer address actually belongs to the cluster?
|
2020-09-14 21:55:43 +00:00
|
|
|
if (calculated_hash != received_hash)
|
2022-05-17 16:22:52 +00:00
|
|
|
{
|
|
|
|
auto exception = Exception(ErrorCodes::AUTHENTICATION_FAILED, "Interserver authentication failed");
|
2022-05-23 19:55:17 +00:00
|
|
|
session->onAuthenticationFailure(/* user_name */ std::nullopt, socket().peerAddress(), exception);
|
2022-05-18 19:57:20 +00:00
|
|
|
throw exception; /// NOLINT
|
2022-05-17 16:22:52 +00:00
|
|
|
}
|
2020-09-14 21:55:43 +00:00
|
|
|
|
2022-05-18 12:06:52 +00:00
|
|
|
/// NOTE Usually we get some fields of client_info (including initial_address and initial_user) from user input,
|
|
|
|
/// so we should not rely on that. However, in this particular case we got client_info from other clickhouse-server, so it's ok.
|
Do not allow to reuse previous credentials in case of inter-server secret
Before this patch INSERT via Buffer/Kafka may re-use previously set user
for that connection, while this is not correct, it should reset the
user, and use global context.
Note, before [1] there was a fallback to default user, but that code had
been removed, and now it got back.
[1]: 0159c74f217ec764060c480819e3ccc9d5a99a63 ("Secure inter-cluster query execution (with initial_user as current query user) [v3]")
Also note, that context for Buffer table (and others) cannot be changed,
since they don't have any user only profile.
I've tested this patch manually using the following:
create table dist (key Int) engine=Distributed(test_cluster_two_shards_secure, default, data, key);
create table buffer (key Int) engine=Buffer(default, dist, 1, 0, 0, 0, 0, 0, 0);
create table data (key Int) engine=Memory();
# to start the connection with readonly user
$ clickhouse-client --user readonly -q 'select * from dist'
$ clickhouse-client -q 'insert into buffer values (1)'
# before this patch this produces errors like:
# 2021.09.27 23:46:48.384920 [ 19474 ] {} <Error> default.dist.DirectoryMonitor: Code: 164. DB::Exception: Received from 127.0.0.2:9000. DB::Exception: readonly: Cannot execute query in readonly mode. Stack trace:
v2: reset the authentication instead of using default user (as suggested by @vitlibar)
v3: reset Session::user and introduce ClientInfo::resetAuthentication (as suggested by @vitlibar)
v4: reset the session every time in interserver mode (suggested by @vitlibar)
2021-09-30 07:15:38 +00:00
|
|
|
if (client_info.initial_user.empty())
|
|
|
|
{
|
2023-03-03 15:25:42 +00:00
|
|
|
LOG_DEBUG(log, "User (no user, interserver mode) (client: {})", getClientAddress(client_info).toString());
|
Do not allow to reuse previous credentials in case of inter-server secret
Before this patch INSERT via Buffer/Kafka may re-use previously set user
for that connection, while this is not correct, it should reset the
user, and use global context.
Note, before [1] there was a fallback to default user, but that code had
been removed, and now it got back.
[1]: 0159c74f217ec764060c480819e3ccc9d5a99a63 ("Secure inter-cluster query execution (with initial_user as current query user) [v3]")
Also note, that context for Buffer table (and others) cannot be changed,
since they don't have any user only profile.
I've tested this patch manually using the following:
create table dist (key Int) engine=Distributed(test_cluster_two_shards_secure, default, data, key);
create table buffer (key Int) engine=Buffer(default, dist, 1, 0, 0, 0, 0, 0, 0);
create table data (key Int) engine=Memory();
# to start the connection with readonly user
$ clickhouse-client --user readonly -q 'select * from dist'
$ clickhouse-client -q 'insert into buffer values (1)'
# before this patch this produces errors like:
# 2021.09.27 23:46:48.384920 [ 19474 ] {} <Error> default.dist.DirectoryMonitor: Code: 164. DB::Exception: Received from 127.0.0.2:9000. DB::Exception: readonly: Cannot execute query in readonly mode. Stack trace:
v2: reset the authentication instead of using default user (as suggested by @vitlibar)
v3: reset Session::user and introduce ClientInfo::resetAuthentication (as suggested by @vitlibar)
v4: reset the session every time in interserver mode (suggested by @vitlibar)
2021-09-30 07:15:38 +00:00
|
|
|
}
|
|
|
|
else
|
2020-09-14 21:55:43 +00:00
|
|
|
{
|
2023-03-03 15:25:42 +00:00
|
|
|
LOG_DEBUG(log, "User (initial, interserver mode): {} (client: {})", client_info.initial_user, getClientAddress(client_info).toString());
|
|
|
|
/// In case of inter-server mode authorization is done with the
|
|
|
|
/// initial address of the client, not the real address from which
|
|
|
|
/// the query was come, since the real address is the address of
|
|
|
|
/// the initiator server, while we are interested in client's
|
|
|
|
/// address.
|
2021-08-01 14:12:34 +00:00
|
|
|
session->authenticate(AlwaysAllowCredentials{client_info.initial_user}, client_info.initial_address);
|
2020-09-14 21:55:43 +00:00
|
|
|
}
|
2023-11-23 11:41:14 +00:00
|
|
|
|
|
|
|
is_interserver_authenticated = true;
|
2020-09-14 21:55:43 +00:00
|
|
|
#else
|
2023-01-24 00:30:26 +00:00
|
|
|
auto exception = Exception(ErrorCodes::AUTHENTICATION_FAILED,
|
|
|
|
"Inter-server secret support is disabled, because ClickHouse was built without SSL library");
|
2022-05-23 19:55:17 +00:00
|
|
|
session->onAuthenticationFailure(/* user_name */ std::nullopt, socket().peerAddress(), exception);
|
2022-05-18 19:57:20 +00:00
|
|
|
throw exception; /// NOLINT
|
2020-09-14 21:55:43 +00:00
|
|
|
#endif
|
|
|
|
}
|
2021-08-01 14:12:34 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
state->query_context = session->makeQueryContext(client_info);
|
2021-08-01 14:12:34 +00:00
|
|
|
|
|
|
|
/// Sets the default database if it wasn't set earlier for the session context.
|
2022-08-04 16:04:06 +00:00
|
|
|
if (is_interserver_mode && !default_database.empty())
|
2024-10-22 12:26:06 +00:00
|
|
|
state->query_context->setCurrentDatabase(default_database);
|
2021-08-01 14:12:34 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
if (state->part_uuids_to_ignore)
|
|
|
|
state->query_context->getIgnoredPartUUIDs()->add(*state->part_uuids_to_ignore);
|
2021-08-01 14:12:34 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
state->query_context->setProgressCallback(
|
|
|
|
[this, &state] (const Progress & value) { this->updateProgress(state.value(), value); });
|
|
|
|
state->query_context->setFileProgressCallback(
|
|
|
|
[this, &state](const FileProgress & value) { this->updateProgress(state.value(), Progress(value)); });
|
2020-09-14 21:55:43 +00:00
|
|
|
|
|
|
|
///
|
|
|
|
/// Settings
|
|
|
|
///
|
2024-07-02 22:51:21 +00:00
|
|
|
|
|
|
|
/// FIXME: Remove when allow_experimental_analyzer will become obsolete.
|
2024-07-03 16:42:01 +00:00
|
|
|
/// Analyzer became Beta in 24.3 and started to be enabled by default.
|
|
|
|
/// We have to disable it for ourselves to make sure we don't have different settings on
|
|
|
|
/// different servers.
|
2024-09-18 12:20:53 +00:00
|
|
|
if (query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && client_info.getVersionNumber() < VersionNumber(23, 3, 0)
|
|
|
|
&& !passed_settings[Setting::allow_experimental_analyzer].changed)
|
2024-07-03 16:42:01 +00:00
|
|
|
passed_settings.set("allow_experimental_analyzer", false);
|
2024-07-02 22:51:21 +00:00
|
|
|
|
2020-02-28 18:55:21 +00:00
|
|
|
auto settings_changes = passed_settings.changes();
|
2024-10-22 12:26:06 +00:00
|
|
|
query_kind = state->query_context->getClientInfo().query_kind;
|
2021-08-01 14:12:34 +00:00
|
|
|
if (query_kind == ClientInfo::QueryKind::INITIAL_QUERY)
|
2020-02-28 18:55:21 +00:00
|
|
|
{
|
|
|
|
/// Throw an exception if the passed settings violate the constraints.
|
2024-10-22 12:26:06 +00:00
|
|
|
state->query_context->checkSettingsConstraints(settings_changes, SettingSource::QUERY);
|
2020-02-28 18:55:21 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// Quietly clamp to the constraints if it's not an initial query.
|
2024-10-22 12:26:06 +00:00
|
|
|
state->query_context->clampToSettingsConstraints(settings_changes, SettingSource::QUERY);
|
2020-02-28 18:55:21 +00:00
|
|
|
}
|
2024-10-22 12:26:06 +00:00
|
|
|
state->query_context->applySettingsChanges(settings_changes);
|
2020-09-08 13:19:27 +00:00
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
/// Use the received query id, or generate a random default. It is convenient
|
|
|
|
/// to also generate the default OpenTelemetry trace id at the same time, and
|
|
|
|
/// set the trace parent.
|
|
|
|
/// Notes:
|
|
|
|
/// 1) ClientInfo might contain upstream trace id, so we decide whether to use
|
|
|
|
/// the default ids after we have received the ClientInfo.
|
|
|
|
/// 2) There is the opentelemetry_start_trace_probability setting that
|
|
|
|
/// controls when we start a new trace. It can be changed via Native protocol,
|
|
|
|
/// so we have to apply the changes first.
|
2024-10-22 12:26:06 +00:00
|
|
|
state->query_context->setCurrentQueryId(state->query_id);
|
|
|
|
|
|
|
|
state->query_context->addQueryParameters(passed_params.toNameToNameMap());
|
2021-08-01 14:12:34 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
state->allow_partial_result_on_first_cancel = state->query_context->getSettingsRef()[Setting::partial_result_on_first_cancel];
|
2022-08-12 12:28:35 +00:00
|
|
|
|
2022-04-28 13:10:27 +00:00
|
|
|
/// For testing hedged requests
|
|
|
|
if (unlikely(sleep_after_receiving_query.totalMilliseconds()))
|
|
|
|
{
|
|
|
|
std::chrono::milliseconds ms(sleep_after_receiving_query.totalMilliseconds());
|
|
|
|
std::this_thread::sleep_for(ms);
|
|
|
|
}
|
2012-03-11 08:52:56 +00:00
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::processUnexpectedQuery()
|
2019-09-03 09:36:16 +00:00
|
|
|
{
|
|
|
|
UInt64 skip_uint_64;
|
|
|
|
String skip_string;
|
|
|
|
|
|
|
|
readStringBinary(skip_string, *in);
|
|
|
|
|
2020-03-13 14:50:26 +00:00
|
|
|
ClientInfo skip_client_info;
|
2020-09-17 12:15:05 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_CLIENT_INFO)
|
|
|
|
skip_client_info.read(*in, client_tcp_protocol_version);
|
2019-09-03 09:36:16 +00:00
|
|
|
|
2020-03-13 14:50:26 +00:00
|
|
|
Settings skip_settings;
|
2020-09-17 12:15:05 +00:00
|
|
|
auto settings_format = (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsWriteFormat::STRINGS_WITH_FLAGS
|
2020-07-20 09:57:17 +00:00
|
|
|
: SettingsWriteFormat::BINARY;
|
|
|
|
skip_settings.read(*in, settings_format);
|
2019-09-03 09:36:16 +00:00
|
|
|
|
2020-09-14 21:55:43 +00:00
|
|
|
std::string skip_hash;
|
2020-09-17 14:55:41 +00:00
|
|
|
bool interserver_secret = client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET;
|
2020-09-14 21:55:43 +00:00
|
|
|
if (interserver_secret)
|
|
|
|
readStringBinary(skip_hash, *in, 32);
|
|
|
|
|
2019-09-03 09:36:16 +00:00
|
|
|
readVarUInt(skip_uint_64, *in);
|
2021-08-01 14:12:34 +00:00
|
|
|
|
2019-09-03 09:36:16 +00:00
|
|
|
readVarUInt(skip_uint_64, *in);
|
2021-08-01 14:12:34 +00:00
|
|
|
last_block_in.compression = static_cast<Protocol::Compression>(skip_uint_64);
|
|
|
|
|
2019-09-03 09:36:16 +00:00
|
|
|
readStringBinary(skip_string, *in);
|
|
|
|
|
2022-08-12 12:28:35 +00:00
|
|
|
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PARAMETERS)
|
|
|
|
skip_settings.read(*in, settings_format);
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
throw Exception(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet Query received from client");
|
2019-09-03 09:36:16 +00:00
|
|
|
}
|
2012-05-21 06:49:05 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
bool TCPHandler::processData(QueryState & state, bool scalar)
|
2013-09-05 20:22:43 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
initBlockInput(state);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-03-09 00:56:38 +00:00
|
|
|
/// The name of the temporary table for writing data, default to empty string
|
2020-02-12 18:14:12 +00:00
|
|
|
auto temporary_id = StorageID::createEmpty();
|
|
|
|
readStringBinary(temporary_id.table_name, *in);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-03-09 00:56:38 +00:00
|
|
|
/// Read one block from the network and write it down
|
2013-09-05 20:22:43 +00:00
|
|
|
Block block = state.block_in->read();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
if (!block)
|
|
|
|
return false;
|
2021-02-10 22:23:27 +00:00
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
if (scalar)
|
|
|
|
{
|
|
|
|
/// Scalar value
|
2024-10-22 12:26:06 +00:00
|
|
|
state.query_context->addScalar(temporary_id.table_name, block);
|
2021-08-01 14:12:34 +00:00
|
|
|
}
|
|
|
|
else if (!state.need_receive_data_for_insert && !state.need_receive_data_for_input)
|
|
|
|
{
|
|
|
|
/// Data for external tables
|
2021-02-10 22:23:27 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
auto resolved = state.query_context->tryResolveStorageID(temporary_id, Context::ResolveExternal);
|
2021-08-01 14:12:34 +00:00
|
|
|
StoragePtr storage;
|
|
|
|
/// If such a table does not exist, create it.
|
|
|
|
if (resolved)
|
2021-02-10 22:23:27 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
storage = DatabaseCatalog::instance().getTable(resolved, state.query_context);
|
2021-02-10 22:23:27 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2021-08-01 14:12:34 +00:00
|
|
|
NamesAndTypesList columns = block.getNamesAndTypesList();
|
2024-10-22 12:26:06 +00:00
|
|
|
auto temporary_table = TemporaryTableHolder(state.query_context, ColumnsDescription{columns}, {});
|
2021-08-01 14:12:34 +00:00
|
|
|
storage = temporary_table.getTable();
|
2024-10-22 12:26:06 +00:00
|
|
|
state.query_context->addExternalTable(temporary_id.table_name, std::move(temporary_table));
|
2014-03-04 15:31:56 +00:00
|
|
|
}
|
2021-08-01 14:12:34 +00:00
|
|
|
auto metadata_snapshot = storage->getInMemoryMetadataPtr();
|
|
|
|
/// The data will be written directly to the table.
|
2024-10-22 12:26:06 +00:00
|
|
|
QueryPipeline temporary_table_out(storage->write(ASTPtr(), metadata_snapshot, state.query_context, /*async_insert=*/false));
|
2021-09-09 17:30:23 +00:00
|
|
|
PushingPipelineExecutor executor(temporary_table_out);
|
|
|
|
executor.start();
|
|
|
|
executor.push(block);
|
|
|
|
executor.finish();
|
2021-08-01 14:12:34 +00:00
|
|
|
}
|
|
|
|
else if (state.need_receive_data_for_input)
|
|
|
|
{
|
|
|
|
/// 'input' table function.
|
|
|
|
state.block_for_input = block;
|
2013-09-05 20:22:43 +00:00
|
|
|
}
|
|
|
|
else
|
2021-08-01 14:12:34 +00:00
|
|
|
{
|
|
|
|
/// INSERT query.
|
2021-09-09 17:30:23 +00:00
|
|
|
state.block_for_insert = block;
|
2021-08-01 14:12:34 +00:00
|
|
|
}
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
return true;
|
2013-09-05 20:22:43 +00:00
|
|
|
}
|
|
|
|
|
2023-03-15 19:53:58 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
bool TCPHandler::processUnexpectedData()
|
2019-09-03 09:36:16 +00:00
|
|
|
{
|
|
|
|
String skip_external_table_name;
|
|
|
|
readStringBinary(skip_external_table_name, *in);
|
|
|
|
|
|
|
|
std::shared_ptr<ReadBuffer> maybe_compressed_in;
|
|
|
|
if (last_block_in.compression == Protocol::Compression::Enable)
|
2024-09-04 16:50:22 +00:00
|
|
|
maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in, /* allow_different_codecs */ true, /* external_data */ query_kind != ClientInfo::QueryKind::SECONDARY_QUERY);
|
2019-09-03 09:36:16 +00:00
|
|
|
else
|
|
|
|
maybe_compressed_in = in;
|
|
|
|
|
2021-10-08 17:21:19 +00:00
|
|
|
auto skip_block_in = std::make_shared<NativeReader>(*maybe_compressed_in, client_tcp_protocol_version);
|
2022-05-20 15:29:54 +00:00
|
|
|
bool read_ok = !!skip_block_in->read();
|
2019-09-03 09:36:16 +00:00
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
return read_ok;
|
2019-09-03 09:36:16 +00:00
|
|
|
}
|
2013-09-05 20:22:43 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::initBlockInput(QueryState & state)
|
2012-03-11 08:52:56 +00:00
|
|
|
{
|
2012-03-19 12:57:56 +00:00
|
|
|
if (!state.block_in)
|
|
|
|
{
|
2021-01-06 00:24:42 +00:00
|
|
|
/// 'allow_different_codecs' is set to true, because some parts of compressed data can be precompressed in advance
|
|
|
|
/// with another codec that the rest of the data. Example: data sent by Distributed tables.
|
|
|
|
|
2012-05-21 06:49:05 +00:00
|
|
|
if (state.compression == Protocol::Compression::Enable)
|
2024-09-04 16:50:22 +00:00
|
|
|
state.maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in, /* allow_different_codecs */ true, /* external_data */ query_kind != ClientInfo::QueryKind::SECONDARY_QUERY);
|
2012-05-21 06:49:05 +00:00
|
|
|
else
|
|
|
|
state.maybe_compressed_in = in;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-12-06 11:24:07 +00:00
|
|
|
Block header;
|
2021-09-16 17:40:42 +00:00
|
|
|
if (state.io.pipeline.pushing())
|
|
|
|
header = state.io.pipeline.getHeader();
|
2019-05-28 18:30:10 +00:00
|
|
|
else if (state.need_receive_data_for_input)
|
|
|
|
header = state.input_header;
|
2018-12-06 11:24:07 +00:00
|
|
|
|
2021-10-08 17:21:19 +00:00
|
|
|
state.block_in = std::make_unique<NativeReader>(
|
2012-03-19 12:57:56 +00:00
|
|
|
*state.maybe_compressed_in,
|
2018-12-06 11:24:07 +00:00
|
|
|
header,
|
2020-09-17 12:15:05 +00:00
|
|
|
client_tcp_protocol_version);
|
2012-03-19 12:57:56 +00:00
|
|
|
}
|
2013-09-05 20:22:43 +00:00
|
|
|
}
|
|
|
|
|
2023-03-16 14:23:17 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::initBlockOutput(QueryState & state, const Block & block)
|
2013-09-05 20:22:43 +00:00
|
|
|
{
|
|
|
|
if (!state.block_out)
|
2012-03-19 12:57:56 +00:00
|
|
|
{
|
2024-10-02 08:45:31 +00:00
|
|
|
state.raw_out = out;
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
const Settings & query_settings = state.query_context->getSettingsRef();
|
2018-06-14 15:33:59 +00:00
|
|
|
if (!state.maybe_compressed_out)
|
|
|
|
{
|
2024-09-18 12:20:53 +00:00
|
|
|
std::string method = Poco::toUpper(query_settings[Setting::network_compression_method].toString());
|
2018-12-21 12:17:30 +00:00
|
|
|
std::optional<int> level;
|
2018-12-21 13:25:39 +00:00
|
|
|
if (method == "ZSTD")
|
2024-09-18 12:20:53 +00:00
|
|
|
level = query_settings[Setting::network_zstd_compression_level];
|
2018-12-21 12:17:30 +00:00
|
|
|
|
2018-06-14 15:33:59 +00:00
|
|
|
if (state.compression == Protocol::Compression::Enable)
|
2020-08-28 17:40:45 +00:00
|
|
|
{
|
2024-09-18 12:20:53 +00:00
|
|
|
CompressionCodecFactory::instance().validateCodec(
|
|
|
|
method,
|
|
|
|
level,
|
|
|
|
!query_settings[Setting::allow_suspicious_codecs],
|
|
|
|
query_settings[Setting::allow_experimental_codecs],
|
|
|
|
query_settings[Setting::enable_zstd_qat_codec]);
|
2020-08-28 17:40:45 +00:00
|
|
|
|
2024-09-26 15:55:41 +00:00
|
|
|
state.maybe_compressed_out = std::make_shared<CompressedWriteBuffer>(
|
2021-05-24 03:43:25 +00:00
|
|
|
*out, CompressionCodecFactory::instance().get(method, level));
|
2020-08-28 17:40:45 +00:00
|
|
|
}
|
2018-06-14 15:33:59 +00:00
|
|
|
else
|
|
|
|
state.maybe_compressed_out = out;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-10-08 17:21:19 +00:00
|
|
|
state.block_out = std::make_unique<NativeWriter>(
|
2013-09-05 20:22:43 +00:00
|
|
|
*state.maybe_compressed_out,
|
2020-09-17 12:15:05 +00:00
|
|
|
client_tcp_protocol_version,
|
2018-12-19 16:47:30 +00:00
|
|
|
block.cloneEmpty(),
|
2024-06-21 18:20:35 +00:00
|
|
|
std::nullopt,
|
2024-09-18 12:20:53 +00:00
|
|
|
!query_settings[Setting::low_cardinality_allow_in_native_format]);
|
2012-03-19 12:57:56 +00:00
|
|
|
}
|
|
|
|
}
|
2012-03-11 08:52:56 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::initLogsBlockOutput(QueryState & state, const Block & block)
|
2018-06-06 20:57:07 +00:00
|
|
|
{
|
|
|
|
if (!state.logs_block_out)
|
|
|
|
{
|
2018-06-14 15:33:59 +00:00
|
|
|
/// Use uncompressed stream since log blocks usually contain only one row
|
2024-10-22 12:26:06 +00:00
|
|
|
const Settings & query_settings = state.query_context->getSettingsRef();
|
2021-10-08 17:21:19 +00:00
|
|
|
state.logs_block_out = std::make_unique<NativeWriter>(
|
2024-09-18 12:20:53 +00:00
|
|
|
*out, client_tcp_protocol_version, block.cloneEmpty(), std::nullopt, !query_settings[Setting::low_cardinality_allow_in_native_format]);
|
2018-06-06 20:57:07 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-03-15 19:53:58 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::initProfileEventsBlockOutput(QueryState & state, const Block & block)
|
2021-08-30 11:04:59 +00:00
|
|
|
{
|
|
|
|
if (!state.profile_events_block_out)
|
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
const Settings & query_settings = state.query_context->getSettingsRef();
|
2021-08-30 11:04:59 +00:00
|
|
|
state.profile_events_block_out = std::make_unique<NativeWriter>(
|
2024-09-18 12:20:53 +00:00
|
|
|
*out, client_tcp_protocol_version, block.cloneEmpty(), std::nullopt, !query_settings[Setting::low_cardinality_allow_in_native_format]);
|
2018-06-06 20:57:07 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-03-15 06:06:55 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::processCancel(QueryState & state)
|
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "processCancel");
|
2023-03-15 13:05:38 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
if (state.allow_partial_result_on_first_cancel && !state.stop_read_return_partial_result)
|
2023-03-15 06:06:55 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
state.stop_read_return_partial_result = true;
|
|
|
|
LOG_INFO(log, "Received 'Cancel' packet from the client, returning partial result.");
|
2024-10-24 16:50:22 +00:00
|
|
|
return;
|
2023-03-15 06:06:55 +00:00
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
state.read_all_data = true;
|
2024-10-24 16:50:22 +00:00
|
|
|
throw Exception(ErrorCodes::QUERY_WAS_CANCELLED_BY_CLIENT, "Received 'Cancel' packet from the client, canceling the query.");
|
2023-03-15 06:06:55 +00:00
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::receivePacketsExpectCancel(QueryState & state)
|
|
|
|
{
|
2024-10-31 11:22:26 +00:00
|
|
|
std::lock_guard lock(callback_mutex);
|
|
|
|
|
2021-08-01 14:12:34 +00:00
|
|
|
if (after_check_cancelled.elapsed() / 1000 < interactive_delay)
|
2024-10-22 12:26:06 +00:00
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2012-05-09 15:15:45 +00:00
|
|
|
after_check_cancelled.restart();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-03-09 00:56:38 +00:00
|
|
|
/// During request execution the only packet that can come from the client is stopping the query.
|
2024-05-14 15:37:20 +00:00
|
|
|
if (in->poll(0))
|
2012-05-09 08:16:09 +00:00
|
|
|
{
|
2020-10-24 03:41:47 +00:00
|
|
|
if (in->eof())
|
2024-10-22 12:26:06 +00:00
|
|
|
throw NetException(ErrorCodes::ABORTED, "Client has dropped the connection, cancel the query.");
|
2020-10-24 03:41:47 +00:00
|
|
|
|
2012-05-09 08:16:09 +00:00
|
|
|
UInt64 packet_type = 0;
|
2012-05-21 06:49:05 +00:00
|
|
|
readVarUInt(packet_type, *in);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
LOG_DEBUG(log, "receivePacketsExpectCancel got {}", Protocol::Client::toString(packet_type));
|
|
|
|
|
2012-05-09 08:16:09 +00:00
|
|
|
switch (packet_type)
|
|
|
|
{
|
|
|
|
case Protocol::Client::Cancel:
|
2024-10-22 12:26:06 +00:00
|
|
|
processCancel(state);
|
|
|
|
break;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2012-05-09 08:16:09 +00:00
|
|
|
default:
|
2023-01-23 21:13:58 +00:00
|
|
|
throw NetException(ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT, "Unknown packet from client {}", toString(packet_type));
|
2012-05-09 08:16:09 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-03-15 19:53:58 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::sendData(QueryState & state, const Block & block)
|
2012-03-19 12:57:56 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
initBlockOutput(state, block);
|
2012-03-11 08:52:56 +00:00
|
|
|
|
2023-07-06 14:56:05 +00:00
|
|
|
size_t prev_bytes_written_out = out->count();
|
|
|
|
size_t prev_bytes_written_compressed_out = state.maybe_compressed_out->count();
|
2012-05-21 06:49:05 +00:00
|
|
|
|
2021-04-01 04:12:07 +00:00
|
|
|
try
|
2021-02-17 17:34:52 +00:00
|
|
|
{
|
2021-04-19 14:12:08 +00:00
|
|
|
/// For testing hedged requests
|
2021-04-19 17:55:18 +00:00
|
|
|
if (unknown_packet_in_send_data)
|
2021-04-19 14:12:08 +00:00
|
|
|
{
|
2024-03-24 01:06:52 +00:00
|
|
|
constexpr UInt64 marker = (1ULL << 63) - 1;
|
2021-04-19 17:55:18 +00:00
|
|
|
--unknown_packet_in_send_data;
|
|
|
|
if (unknown_packet_in_send_data == 0)
|
2023-07-06 14:56:05 +00:00
|
|
|
writeVarUInt(marker, *out);
|
2021-04-19 14:12:08 +00:00
|
|
|
}
|
|
|
|
|
2021-04-01 04:12:07 +00:00
|
|
|
writeVarUInt(Protocol::Server::Data, *out);
|
|
|
|
|
|
|
|
/// For testing hedged requests
|
2024-10-22 12:26:06 +00:00
|
|
|
if (block.rows() > 0 && state.query_context->getSettingsRef()[Setting::sleep_in_send_data_ms].totalMilliseconds())
|
2021-04-01 04:12:07 +00:00
|
|
|
{
|
2024-06-02 07:25:48 +00:00
|
|
|
/// This strange sequence is needed in case of chunked protocol is enabled, in order for client not to
|
2024-06-02 13:03:48 +00:00
|
|
|
/// hang on receiving of at least packet type - chunk will not be processed unless either chunk footer
|
|
|
|
/// or chunk continuation header is received - first 'next' is sending starting chunk containing packet type
|
2024-06-02 07:25:48 +00:00
|
|
|
/// and second 'next' is sending chunk continuation header.
|
|
|
|
out->next();
|
|
|
|
/// Send external table name (empty name is the main table)
|
|
|
|
writeStringBinary("", *out);
|
2021-04-01 04:12:07 +00:00
|
|
|
out->next();
|
2024-10-22 12:26:06 +00:00
|
|
|
std::chrono::milliseconds ms(state.query_context->getSettingsRef()[Setting::sleep_in_send_data_ms].totalMilliseconds());
|
2021-04-01 04:12:07 +00:00
|
|
|
std::this_thread::sleep_for(ms);
|
|
|
|
}
|
2024-06-02 07:25:48 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
/// Send external table name (empty name is the main table)
|
|
|
|
writeStringBinary("", *out);
|
|
|
|
}
|
2021-04-01 04:12:07 +00:00
|
|
|
|
|
|
|
state.block_out->write(block);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
if (state.maybe_compressed_out != out)
|
|
|
|
state.maybe_compressed_out->next();
|
|
|
|
|
|
|
|
out->finishChunk();
|
2021-02-17 17:34:52 +00:00
|
|
|
out->next();
|
|
|
|
}
|
2021-04-01 04:12:07 +00:00
|
|
|
catch (...)
|
|
|
|
{
|
2024-09-26 15:55:41 +00:00
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
2021-04-01 04:12:07 +00:00
|
|
|
/// In case of unsuccessful write, if the buffer with written data was not flushed,
|
|
|
|
/// we will rollback write to avoid breaking the protocol.
|
|
|
|
/// (otherwise the client will not be able to receive exception after unfinished data
|
|
|
|
/// as it will expect the continuation of the data).
|
|
|
|
/// It looks like hangs on client side or a message like "Data compressed with different methods".
|
2021-02-17 17:34:52 +00:00
|
|
|
|
2021-04-01 04:12:07 +00:00
|
|
|
if (state.compression == Protocol::Compression::Enable)
|
|
|
|
{
|
|
|
|
auto extra_bytes_written_compressed = state.maybe_compressed_out->count() - prev_bytes_written_compressed_out;
|
|
|
|
if (state.maybe_compressed_out->offset() >= extra_bytes_written_compressed)
|
|
|
|
state.maybe_compressed_out->position() -= extra_bytes_written_compressed;
|
|
|
|
}
|
|
|
|
|
|
|
|
auto extra_bytes_written_out = out->count() - prev_bytes_written_out;
|
|
|
|
if (out->offset() >= extra_bytes_written_out)
|
|
|
|
out->position() -= extra_bytes_written_out;
|
|
|
|
|
|
|
|
throw;
|
|
|
|
}
|
2012-03-19 12:57:56 +00:00
|
|
|
}
|
|
|
|
|
2023-03-15 19:53:58 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::sendLogData(QueryState & state, const Block & block)
|
2018-06-06 20:57:07 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
initLogsBlockOutput(state, block);
|
2018-06-06 20:57:07 +00:00
|
|
|
|
2024-10-02 08:45:31 +00:00
|
|
|
if (out->isCanceled())
|
|
|
|
return;
|
|
|
|
|
2018-06-06 20:57:07 +00:00
|
|
|
writeVarUInt(Protocol::Server::Log, *out);
|
|
|
|
/// Send log tag (empty tag is the default tag)
|
|
|
|
writeStringBinary("", *out);
|
|
|
|
|
|
|
|
state.logs_block_out->write(block);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2018-06-06 20:57:07 +00:00
|
|
|
out->next();
|
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
|
|
|
void TCPHandler::sendTableColumns(QueryState &, const ColumnsDescription & columns)
|
2018-12-04 20:03:04 +00:00
|
|
|
{
|
|
|
|
writeVarUInt(Protocol::Server::TableColumns, *out);
|
|
|
|
|
|
|
|
/// Send external table name (empty name is the main table)
|
|
|
|
writeStringBinary("", *out);
|
|
|
|
writeStringBinary(columns.toString(), *out);
|
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2018-12-04 20:03:04 +00:00
|
|
|
out->next();
|
|
|
|
}
|
2018-06-06 20:57:07 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2018-08-24 07:30:53 +00:00
|
|
|
void TCPHandler::sendException(const Exception & e, bool with_stack_trace)
|
2012-03-19 12:57:56 +00:00
|
|
|
{
|
2024-10-02 08:45:31 +00:00
|
|
|
if (out->isCanceled())
|
|
|
|
return;
|
|
|
|
|
2012-05-21 06:49:05 +00:00
|
|
|
writeVarUInt(Protocol::Server::Exception, *out);
|
2018-08-24 07:30:53 +00:00
|
|
|
writeException(e, *out, with_stack_trace);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2012-05-21 06:49:05 +00:00
|
|
|
out->next();
|
2012-03-19 12:57:56 +00:00
|
|
|
}
|
|
|
|
|
2012-05-21 06:49:05 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::sendEndOfStream(QueryState & state)
|
2012-05-08 11:19:00 +00:00
|
|
|
{
|
2012-07-15 21:43:04 +00:00
|
|
|
state.sent_all_data = true;
|
2022-04-30 00:04:12 +00:00
|
|
|
state.io.setAllDataSent();
|
2022-04-26 06:17:17 +00:00
|
|
|
|
2012-05-21 06:49:05 +00:00
|
|
|
writeVarUInt(Protocol::Server::EndOfStream, *out);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2012-05-21 06:49:05 +00:00
|
|
|
out->next();
|
2012-05-08 11:19:00 +00:00
|
|
|
}
|
|
|
|
|
2012-05-21 06:49:05 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::updateProgress(QueryState & state, const Progress & value)
|
2012-03-19 12:57:56 +00:00
|
|
|
{
|
2014-10-25 18:33:52 +00:00
|
|
|
state.progress.incrementPiecewiseAtomically(value);
|
2013-11-02 21:18:54 +00:00
|
|
|
}
|
2012-05-09 15:15:45 +00:00
|
|
|
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::sendProgress(QueryState & state)
|
2013-11-02 21:18:54 +00:00
|
|
|
{
|
2012-05-21 06:49:05 +00:00
|
|
|
writeVarUInt(Protocol::Server::Progress, *out);
|
2022-05-06 15:04:03 +00:00
|
|
|
auto increment = state.progress.fetchValuesAndResetPiecewiseAtomically();
|
2022-08-08 04:55:41 +00:00
|
|
|
UInt64 current_elapsed_ns = state.watch.elapsedNanoseconds();
|
|
|
|
increment.elapsed_ns = current_elapsed_ns - state.prev_elapsed_ns;
|
|
|
|
state.prev_elapsed_ns = current_elapsed_ns;
|
2020-09-17 12:15:05 +00:00
|
|
|
increment.write(*out, client_tcp_protocol_version);
|
2024-05-14 15:37:20 +00:00
|
|
|
|
2024-06-07 01:45:56 +00:00
|
|
|
out->finishChunk();
|
2012-05-21 06:49:05 +00:00
|
|
|
out->next();
|
2012-03-11 08:52:56 +00:00
|
|
|
}
|
|
|
|
|
2012-03-09 15:46:52 +00:00
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
void TCPHandler::sendLogs(QueryState & state)
|
2018-06-06 20:57:07 +00:00
|
|
|
{
|
|
|
|
if (!state.logs_queue)
|
|
|
|
return;
|
|
|
|
|
|
|
|
MutableColumns logs_columns;
|
|
|
|
MutableColumns curr_logs_columns;
|
|
|
|
size_t rows = 0;
|
|
|
|
|
|
|
|
for (; state.logs_queue->tryPop(curr_logs_columns); ++rows)
|
|
|
|
{
|
|
|
|
if (rows == 0)
|
|
|
|
{
|
|
|
|
logs_columns = std::move(curr_logs_columns);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
for (size_t j = 0; j < logs_columns.size(); ++j)
|
|
|
|
logs_columns[j]->insertRangeFrom(*curr_logs_columns[j], 0, curr_logs_columns[j]->size());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (rows > 0)
|
|
|
|
{
|
2018-06-15 17:32:35 +00:00
|
|
|
Block block = InternalTextLogsQueue::getSampleBlock();
|
2018-06-06 20:57:07 +00:00
|
|
|
block.setColumns(std::move(logs_columns));
|
2024-10-22 12:26:06 +00:00
|
|
|
sendLogData(state, block);
|
2018-06-06 20:57:07 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2012-03-09 15:46:52 +00:00
|
|
|
void TCPHandler::run()
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
runImpl();
|
2020-10-10 17:47:34 +00:00
|
|
|
LOG_DEBUG(log, "Done processing connection.");
|
2012-03-09 15:46:52 +00:00
|
|
|
}
|
2024-10-22 12:26:06 +00:00
|
|
|
catch (...)
|
2012-03-09 15:46:52 +00:00
|
|
|
{
|
2024-10-22 12:26:06 +00:00
|
|
|
tryLogCurrentException(log, "TCPHandler");
|
|
|
|
throw;
|
2012-03-09 15:46:52 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2023-03-03 15:25:42 +00:00
|
|
|
Poco::Net::SocketAddress TCPHandler::getClientAddress(const ClientInfo & client_info)
|
|
|
|
{
|
|
|
|
/// Extract the last entry from comma separated list of forwarded_for addresses.
|
|
|
|
/// Only the last proxy can be trusted (if any).
|
|
|
|
String forwarded_address = client_info.getLastForwardedFor();
|
|
|
|
if (!forwarded_address.empty() && server.config().getBool("auth_use_forwarded_address", false))
|
|
|
|
return Poco::Net::SocketAddress(forwarded_address, socket().peerAddress().port());
|
|
|
|
return socket().peerAddress();
|
|
|
|
}
|
|
|
|
|
2024-10-22 12:26:06 +00:00
|
|
|
|
2012-03-09 15:46:52 +00:00
|
|
|
}
|