2021-09-02 14:27:19 +00:00
|
|
|
#include <Common/ConcurrentBoundedQueue.h>
|
2021-10-15 20:18:20 +00:00
|
|
|
#include <QueryPipeline/RemoteQueryExecutor.h>
|
|
|
|
#include <QueryPipeline/RemoteQueryExecutorReadContext.h>
|
2020-06-02 15:59:57 +00:00
|
|
|
|
2020-06-02 16:27:05 +00:00
|
|
|
#include <Columns/ColumnConst.h>
|
|
|
|
#include <Common/CurrentThread.h>
|
2024-02-02 15:05:14 +00:00
|
|
|
#include <Core/Protocol.h>
|
2022-05-23 19:47:32 +00:00
|
|
|
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
|
|
|
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
2020-06-02 16:27:05 +00:00
|
|
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
2021-11-25 15:41:50 +00:00
|
|
|
#include <Processors/Transforms/LimitsCheckingTransform.h>
|
2022-05-23 19:47:32 +00:00
|
|
|
#include <Processors/QueryPlan/QueryPlan.h>
|
|
|
|
#include <QueryPipeline/QueryPipelineBuilder.h>
|
2020-09-10 19:55:36 +00:00
|
|
|
#include <Storages/SelectQueryInfo.h>
|
2020-06-02 16:27:05 +00:00
|
|
|
#include <Interpreters/castColumn.h>
|
|
|
|
#include <Interpreters/Cluster.h>
|
2020-12-10 22:05:02 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2020-06-02 16:27:05 +00:00
|
|
|
#include <Interpreters/InternalTextLogsQueue.h>
|
2023-02-03 10:54:49 +00:00
|
|
|
#include <IO/ConnectionTimeouts.h>
|
2024-02-02 15:05:14 +00:00
|
|
|
#include <Client/ConnectionEstablisher.h>
|
2021-01-19 19:21:06 +00:00
|
|
|
#include <Client/MultiplexedConnections.h>
|
|
|
|
#include <Client/HedgedConnections.h>
|
2020-11-20 17:23:53 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
|
2023-02-28 07:56:10 +00:00
|
|
|
#include <Storages/StorageMemory.h>
|
2023-10-11 15:04:59 +00:00
|
|
|
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
|
2020-06-02 16:27:05 +00:00
|
|
|
|
2021-08-31 23:47:52 +00:00
|
|
|
|
2023-02-12 09:04:05 +00:00
|
|
|
namespace ProfileEvents
|
|
|
|
{
|
2023-04-20 11:56:20 +00:00
|
|
|
extern const Event SuspendSendingQueryToShard;
|
2023-02-12 09:04:05 +00:00
|
|
|
extern const Event ReadTaskRequestsReceived;
|
|
|
|
extern const Event MergeTreeReadTaskRequestsReceived;
|
2024-02-05 20:21:48 +00:00
|
|
|
extern const Event ParallelReplicasAvailableCount;
|
2023-02-12 09:04:05 +00:00
|
|
|
}
|
|
|
|
|
2020-06-02 16:27:05 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2021-04-12 17:07:01 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2020-06-02 16:27:05 +00:00
|
|
|
extern const int UNKNOWN_PACKET_FROM_SERVER;
|
2020-11-20 17:23:53 +00:00
|
|
|
extern const int DUPLICATED_PART_UUIDS;
|
2021-10-12 21:15:05 +00:00
|
|
|
extern const int SYSTEM_ERROR;
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
|
|
|
|
2021-07-23 07:40:03 +00:00
|
|
|
RemoteQueryExecutor::RemoteQueryExecutor(
|
2024-01-19 12:51:30 +00:00
|
|
|
const String & query_,
|
|
|
|
const Block & header_,
|
|
|
|
ContextPtr context_,
|
|
|
|
const Scalars & scalars_,
|
|
|
|
const Tables & external_tables_,
|
|
|
|
QueryProcessingStage::Enum stage_,
|
|
|
|
std::optional<Extension> extension_,
|
|
|
|
GetPriorityForLoadBalancing::Func priority_func_)
|
|
|
|
: header(header_)
|
|
|
|
, query(query_)
|
|
|
|
, context(context_)
|
|
|
|
, scalars(scalars_)
|
|
|
|
, external_tables(external_tables_)
|
|
|
|
, stage(stage_)
|
2023-05-31 20:10:33 +00:00
|
|
|
, extension(extension_)
|
2024-01-19 12:51:30 +00:00
|
|
|
, priority_func(priority_func_)
|
|
|
|
{
|
|
|
|
}
|
2021-07-23 07:40:03 +00:00
|
|
|
|
2024-02-01 21:06:29 +00:00
|
|
|
RemoteQueryExecutor::RemoteQueryExecutor(
|
2024-02-05 12:56:03 +00:00
|
|
|
ConnectionPoolPtr pool,
|
2024-02-01 21:06:29 +00:00
|
|
|
const String & query_,
|
|
|
|
const Block & header_,
|
|
|
|
ContextPtr context_,
|
|
|
|
ThrottlerPtr throttler,
|
|
|
|
const Scalars & scalars_,
|
|
|
|
const Tables & external_tables_,
|
|
|
|
QueryProcessingStage::Enum stage_,
|
|
|
|
std::optional<Extension> extension_)
|
|
|
|
: RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_)
|
|
|
|
{
|
|
|
|
create_connections = [this, pool, throttler, extension_](AsyncCallback)
|
|
|
|
{
|
|
|
|
const Settings & current_settings = context->getSettingsRef();
|
|
|
|
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
|
|
|
|
|
|
|
|
ConnectionPoolWithFailover::TryResult result;
|
|
|
|
std::string fail_message;
|
|
|
|
if (main_table)
|
|
|
|
{
|
|
|
|
auto table_name = main_table.getQualifiedName();
|
|
|
|
|
|
|
|
ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, &table_name);
|
|
|
|
connection_establisher.run(result, fail_message);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, nullptr);
|
|
|
|
connection_establisher.run(result, fail_message);
|
|
|
|
}
|
|
|
|
|
|
|
|
std::vector<IConnectionPool::Entry> connection_entries;
|
|
|
|
if (!result.entry.isNull() && result.is_usable)
|
2024-02-05 20:21:48 +00:00
|
|
|
{
|
|
|
|
if (extension_ && extension_->parallel_reading_coordinator)
|
|
|
|
ProfileEvents::increment(ProfileEvents::ParallelReplicasAvailableCount);
|
|
|
|
|
2024-02-01 21:06:29 +00:00
|
|
|
connection_entries.emplace_back(std::move(result.entry));
|
2024-02-05 20:21:48 +00:00
|
|
|
}
|
2024-07-09 17:53:34 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "Failed to connect to replica {}. {}", pool->getAddress(), fail_message);
|
|
|
|
}
|
2024-02-01 21:06:29 +00:00
|
|
|
|
2024-06-19 12:44:26 +00:00
|
|
|
auto res = std::make_unique<MultiplexedConnections>(std::move(connection_entries), context, throttler);
|
2024-02-01 21:06:29 +00:00
|
|
|
if (extension_ && extension_->replica_info)
|
|
|
|
res->setReplicaInfo(*extension_->replica_info);
|
|
|
|
|
|
|
|
return res;
|
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2020-06-02 16:27:05 +00:00
|
|
|
RemoteQueryExecutor::RemoteQueryExecutor(
|
|
|
|
Connection & connection,
|
2024-01-30 13:06:47 +00:00
|
|
|
const String & query_,
|
|
|
|
const Block & header_,
|
|
|
|
ContextPtr context_,
|
|
|
|
ThrottlerPtr throttler,
|
|
|
|
const Scalars & scalars_,
|
|
|
|
const Tables & external_tables_,
|
|
|
|
QueryProcessingStage::Enum stage_,
|
|
|
|
std::optional<Extension> extension_)
|
2021-12-09 10:39:28 +00:00
|
|
|
: RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_)
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
2023-03-03 19:30:43 +00:00
|
|
|
create_connections = [this, &connection, throttler, extension_](AsyncCallback)
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
2024-06-19 12:44:26 +00:00
|
|
|
auto res = std::make_unique<MultiplexedConnections>(connection, context, throttler);
|
2021-12-09 10:39:28 +00:00
|
|
|
if (extension_ && extension_->replica_info)
|
|
|
|
res->setReplicaInfo(*extension_->replica_info);
|
|
|
|
return res;
|
2020-06-02 16:27:05 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
2021-07-23 07:40:03 +00:00
|
|
|
RemoteQueryExecutor::RemoteQueryExecutor(
|
|
|
|
std::shared_ptr<Connection> connection_ptr,
|
2024-01-30 13:06:47 +00:00
|
|
|
const String & query_,
|
|
|
|
const Block & header_,
|
|
|
|
ContextPtr context_,
|
|
|
|
ThrottlerPtr throttler,
|
|
|
|
const Scalars & scalars_,
|
|
|
|
const Tables & external_tables_,
|
|
|
|
QueryProcessingStage::Enum stage_,
|
|
|
|
std::optional<Extension> extension_)
|
2021-12-09 10:39:28 +00:00
|
|
|
: RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_)
|
2021-07-23 07:40:03 +00:00
|
|
|
{
|
2023-03-03 19:30:43 +00:00
|
|
|
create_connections = [this, connection_ptr, throttler, extension_](AsyncCallback)
|
2021-07-23 07:40:03 +00:00
|
|
|
{
|
2024-06-19 12:44:26 +00:00
|
|
|
auto res = std::make_unique<MultiplexedConnections>(connection_ptr, context, throttler);
|
2021-12-09 10:39:28 +00:00
|
|
|
if (extension_ && extension_->replica_info)
|
|
|
|
res->setReplicaInfo(*extension_->replica_info);
|
|
|
|
return res;
|
2020-06-02 16:27:05 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
|
|
|
RemoteQueryExecutor::RemoteQueryExecutor(
|
2021-01-19 19:21:06 +00:00
|
|
|
std::vector<IConnectionPool::Entry> && connections_,
|
2024-01-30 13:06:47 +00:00
|
|
|
const String & query_,
|
|
|
|
const Block & header_,
|
|
|
|
ContextPtr context_,
|
|
|
|
const ThrottlerPtr & throttler,
|
|
|
|
const Scalars & scalars_,
|
|
|
|
const Tables & external_tables_,
|
|
|
|
QueryProcessingStage::Enum stage_,
|
|
|
|
std::optional<Extension> extension_)
|
2023-10-12 20:34:26 +00:00
|
|
|
: RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_)
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
2024-01-30 13:06:47 +00:00
|
|
|
create_connections = [this, connections_, throttler, extension_](AsyncCallback) mutable
|
|
|
|
{
|
2024-06-19 12:44:26 +00:00
|
|
|
auto res = std::make_unique<MultiplexedConnections>(std::move(connections_), context, throttler);
|
2021-12-09 10:39:28 +00:00
|
|
|
if (extension_ && extension_->replica_info)
|
|
|
|
res->setReplicaInfo(*extension_->replica_info);
|
|
|
|
return res;
|
2020-06-02 16:27:05 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
|
|
|
RemoteQueryExecutor::RemoteQueryExecutor(
|
2023-03-11 18:05:47 +00:00
|
|
|
const ConnectionPoolWithFailoverPtr & pool,
|
2024-01-19 12:51:30 +00:00
|
|
|
const String & query_,
|
|
|
|
const Block & header_,
|
|
|
|
ContextPtr context_,
|
|
|
|
const ThrottlerPtr & throttler,
|
|
|
|
const Scalars & scalars_,
|
|
|
|
const Tables & external_tables_,
|
|
|
|
QueryProcessingStage::Enum stage_,
|
|
|
|
std::optional<Extension> extension_,
|
|
|
|
GetPriorityForLoadBalancing::Func priority_func_)
|
|
|
|
: RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_, priority_func_)
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
2023-09-02 21:52:56 +00:00
|
|
|
create_connections = [this, pool, throttler](AsyncCallback async_callback)->std::unique_ptr<IConnections>
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
const Settings & current_settings = context->getSettingsRef();
|
2020-06-02 16:27:05 +00:00
|
|
|
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
|
2021-01-19 19:21:06 +00:00
|
|
|
|
2021-02-01 17:23:46 +00:00
|
|
|
#if defined(OS_LINUX)
|
|
|
|
if (current_settings.use_hedged_requests)
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
2021-01-19 19:21:06 +00:00
|
|
|
std::shared_ptr<QualifiedTableName> table_to_check = nullptr;
|
|
|
|
if (main_table)
|
|
|
|
table_to_check = std::make_shared<QualifiedTableName>(main_table.getQualifiedName());
|
|
|
|
|
2024-01-19 12:51:30 +00:00
|
|
|
auto res = std::make_unique<HedgedConnections>(
|
|
|
|
pool, context, timeouts, throttler, pool_mode, table_to_check, std::move(async_callback), priority_func);
|
2023-09-02 21:52:56 +00:00
|
|
|
if (extension && extension->replica_info)
|
|
|
|
res->setReplicaInfo(*extension->replica_info);
|
2021-12-09 10:39:28 +00:00
|
|
|
return res;
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
2021-02-01 17:23:46 +00:00
|
|
|
#endif
|
2020-06-02 16:27:05 +00:00
|
|
|
|
2021-02-01 17:23:46 +00:00
|
|
|
std::vector<IConnectionPool::Entry> connection_entries;
|
2023-09-02 21:52:56 +00:00
|
|
|
std::optional<bool> skip_unavailable_endpoints;
|
|
|
|
if (extension && extension->parallel_reading_coordinator)
|
|
|
|
skip_unavailable_endpoints = true;
|
|
|
|
|
2021-02-01 17:23:46 +00:00
|
|
|
if (main_table)
|
|
|
|
{
|
2023-11-21 16:04:54 +00:00
|
|
|
auto try_results = pool->getManyChecked(
|
|
|
|
timeouts,
|
|
|
|
current_settings,
|
|
|
|
pool_mode,
|
|
|
|
main_table.getQualifiedName(),
|
|
|
|
std::move(async_callback),
|
2024-01-19 12:51:30 +00:00
|
|
|
skip_unavailable_endpoints,
|
|
|
|
priority_func);
|
2021-02-01 17:23:46 +00:00
|
|
|
connection_entries.reserve(try_results.size());
|
|
|
|
for (auto & try_result : try_results)
|
|
|
|
connection_entries.emplace_back(std::move(try_result.entry));
|
2021-01-19 19:21:06 +00:00
|
|
|
}
|
2021-02-01 17:23:46 +00:00
|
|
|
else
|
2023-09-02 21:52:56 +00:00
|
|
|
{
|
2024-01-19 12:51:30 +00:00
|
|
|
connection_entries = pool->getMany(
|
|
|
|
timeouts, current_settings, pool_mode, std::move(async_callback), skip_unavailable_endpoints, priority_func);
|
2023-09-02 21:52:56 +00:00
|
|
|
}
|
2021-02-01 17:23:46 +00:00
|
|
|
|
2024-06-19 12:44:26 +00:00
|
|
|
auto res = std::make_unique<MultiplexedConnections>(std::move(connection_entries), context, throttler);
|
2023-09-02 21:52:56 +00:00
|
|
|
if (extension && extension->replica_info)
|
|
|
|
res->setReplicaInfo(*extension->replica_info);
|
2021-12-09 10:39:28 +00:00
|
|
|
return res;
|
2020-06-02 16:27:05 +00:00
|
|
|
};
|
|
|
|
}
|
|
|
|
|
|
|
|
RemoteQueryExecutor::~RemoteQueryExecutor()
|
|
|
|
{
|
2023-03-21 16:01:54 +00:00
|
|
|
/// We should finish establishing connections to disconnect it later,
|
|
|
|
/// so these connections won't be in the out-of-sync state.
|
|
|
|
if (read_context && !established)
|
|
|
|
{
|
|
|
|
/// Set was_cancelled, so the query won't be sent after creating connections.
|
|
|
|
was_cancelled = true;
|
Fix possible uncaught exception during distributed query cancellation
Cancellation of distributed queries may throw (i.e. some timeout), and
in case of pipeline had not been properly created properly
(EXCEPTION_BEFORE_START) cancel will not be sent, so cancellation will
be done from dtor and will throw.
<details>
<summary>stacktrace</summary>
```
<Fatal> BaseDaemon: (version 23.9.2.56 (official build), build id: 76109A79FA62B9BC630A6C39438DEA7D28147B68, git hash: a1bf3f1de55abf2354dc498ffbee270be043d633) (from thread 51895) Terminate called for uncaught exception:
<Fatal> BaseDaemon: Code: 209. DB::NetException: Timeout exceeded while reading from socket (socket (10.61.1.50:9000), receive timeout 20000 ms). (SOCKET_TIMEOUT), Stack trace (when copying this message, always include the lines below):
<Fatal> BaseDaemon:
<Fatal> BaseDaemon: 0. ./build_docker/./src/Common/Exception.cpp:98: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000c741d97 in /usr/lib/debug/usr/bin/clickhouse.debug
<Fatal> BaseDaemon: 1. ./build_docker/./contrib/llvm-project/libcxx/include/string:1499: DB::NetException::NetException<String const&>(int, String const&) @ 0x000000001148ace7 in /usr/lib/debug/usr/bin/clickhouse.debug
<Fatal> BaseDaemon: 2. ./build_docker/./src/QueryPipeline/RemoteQueryExecutorReadContext.cpp:101: DB::RemoteQueryExecutorReadContext::checkTimeout(bool) @ 0x000000001148a70a in /usr/lib/debug/usr/bin/clickhouse.debug
<Fatal> BaseDaemon: 3. ./build_docker/./src/Common/Fiber.h:42: DB::RemoteQueryExecutorReadContext::cancelBefore() @ 0x000000001148aded in /usr/lib/debug/usr/bin/clickhouse.debug
<Fatal> BaseDaemon: 4. ./build_docker/./src/Common/Fiber.h:27: DB::AsyncTaskExecutor::cancel() @ 0x000000001148969d in /usr/lib/debug/usr/bin/clickhouse.debug
<Fatal> BaseDaemon: 5. ./build_docker/./src/QueryPipeline/RemoteQueryExecutor.cpp:169: DB::RemoteQueryExecutor::~RemoteQueryExecutor() @ 0x000000001147a9d0 in /usr/lib/debug/usr/bin/clickhouse.debug
<Fatal> BaseDaemon: 6. ./build_docker/./contrib/llvm-project/libcxx/src/include/atomic_support.h:74: std::__shared_ptr_emplace<DB::RemoteSource, std::allocator<DB::RemoteSource>>::__on_zero_shared() @ 0x000000000c94a4a1 in /usr/lib/debug/usr/bin/clickhou
<Fatal> BaseDaemon: 7. ./build_docker/./contrib/llvm-project/libcxx/src/include/atomic_support.h:74: std::__shared_ptr_emplace<std::vector<std::shared_ptr<DB::IProcessor>, std::allocator<std::shared_ptr<DB::IProcessor>>>, std::allocator<std::vector<std::
or<std::shared_ptr<DB::IProcessor>>>>>::__on_zero_shared() @ 0x00000000114603a1 in /usr/lib/debug/usr/bin/clickhouse.debug
<Fatal> BaseDaemon: 8. ./build_docker/./contrib/llvm-project/libcxx/src/include/atomic_support.h:74: DB::QueryPipeline::~QueryPipeline() @ 0x000000001146347a in /usr/lib/debug/usr/bin/clickhouse.debug
<Fatal> BaseDaemon: 9. ./build_docker/./src/QueryPipeline/QueryPipeline.cpp:709: DB::QueryPipeline::reset() @ 0x0000000011468f78 in /usr/lib/debug/usr/bin/clickhouse.debug
<Fatal> BaseDaemon: 10. ./build_docker/./src/Interpreters/executeQuery.cpp:1427: DB::executeQuery(DB::ReadBuffer&, DB::WriteBuffer&, bool, std::shared_ptr<DB::Context>, std::function<void (DB::QueryResultDetails const&)>, std::optional<DB::FormatSettings> const&, std::function<void (DB::IOutputFormat&)>) @ 0x000000001249aa8b in /usr/lib/debug/usr/bin/clickhouse.debug
<Fatal> BaseDaemon: 11. ./build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:818: DB::HTTPHandler::processQuery(DB::HTTPServerRequest&, DB::HTMLForm&, DB::HTTPServerResponse&, DB::HTTPHandler::Output&, std::optional<DB::CurrentThread::QueryScope>&) @ 0x00000000132bddb4 in /usr/lib/debug/usr/bin/clickhouse.debug
<Fatal> BaseDaemon: 12. ./build_docker/./contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:290: DB::HTTPHandler::handleRequest(DB::HTTPServerRequest&, DB::HTTPServerResponse&) @ 0x00000000132c23a4 in /usr/lib/debug/usr/bin/clickhouse.debug
```
</details>
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2024-02-01 14:53:13 +00:00
|
|
|
|
|
|
|
/// Cancellation may throw (i.e. some timeout), and in case of pipeline
|
|
|
|
/// had not been properly created properly (EXCEPTION_BEFORE_START)
|
|
|
|
/// cancel will not be sent, so cancellation will be done from dtor and
|
|
|
|
/// will throw.
|
|
|
|
try
|
|
|
|
{
|
|
|
|
read_context->cancel();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(log ? log : getLogger("RemoteQueryExecutor"));
|
|
|
|
}
|
2023-03-21 16:01:54 +00:00
|
|
|
}
|
|
|
|
|
2020-06-02 16:27:05 +00:00
|
|
|
/** If interrupted in the middle of the loop of communication with replicas, then interrupt
|
|
|
|
* all connections, then read and skip the remaining packets to make sure
|
|
|
|
* these connections did not remain hanging in the out-of-sync state.
|
|
|
|
*/
|
2023-03-17 13:02:20 +00:00
|
|
|
if (established || (isQueryPending() && connections))
|
2024-02-01 19:56:31 +00:00
|
|
|
{
|
|
|
|
/// May also throw (so as cancel() above)
|
|
|
|
try
|
|
|
|
{
|
|
|
|
connections->disconnect();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(log ? log : getLogger("RemoteQueryExecutor"));
|
|
|
|
}
|
|
|
|
}
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/** If we receive a block with slightly different column types, or with excessive columns,
|
|
|
|
* we will adapt it to expected structure.
|
|
|
|
*/
|
2021-03-24 18:36:31 +00:00
|
|
|
static Block adaptBlockStructure(const Block & block, const Block & header)
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
|
|
|
/// Special case when reader doesn't care about result structure. Deprecated and used only in Benchmark, PerformanceTest.
|
|
|
|
if (!header)
|
|
|
|
return block;
|
|
|
|
|
|
|
|
Block res;
|
|
|
|
res.info = block.info;
|
|
|
|
|
|
|
|
for (const auto & elem : header)
|
|
|
|
{
|
|
|
|
ColumnPtr column;
|
|
|
|
|
|
|
|
if (elem.column && isColumnConst(*elem.column))
|
|
|
|
{
|
|
|
|
/// We expect constant column in block.
|
|
|
|
/// If block is not empty, then get value for constant from it,
|
|
|
|
/// because it may be different for remote server for functions like version(), uptime(), ...
|
|
|
|
if (block.rows() > 0 && block.has(elem.name))
|
|
|
|
{
|
|
|
|
/// Const column is passed as materialized. Get first value from it.
|
|
|
|
///
|
|
|
|
/// TODO: check that column contains the same value.
|
|
|
|
/// TODO: serialize const columns.
|
|
|
|
auto col = block.getByName(elem.name);
|
|
|
|
col.column = block.getByName(elem.name).column->cut(0, 1);
|
|
|
|
|
|
|
|
column = castColumn(col, elem.type);
|
|
|
|
|
|
|
|
if (!isColumnConst(*column))
|
|
|
|
column = ColumnConst::create(column, block.rows());
|
|
|
|
else
|
|
|
|
/// It is not possible now. Just in case we support const columns serialization.
|
|
|
|
column = column->cloneResized(block.rows());
|
|
|
|
}
|
|
|
|
else
|
|
|
|
column = elem.column->cloneResized(block.rows());
|
|
|
|
}
|
|
|
|
else
|
|
|
|
column = castColumn(block.getByName(elem.name), elem.type);
|
|
|
|
|
|
|
|
res.insert({column, elem.type, elem.name});
|
|
|
|
}
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
void RemoteQueryExecutor::sendQuery(ClientInfo::QueryKind query_kind, AsyncCallback async_callback)
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
Fix "Unexpected packet Data received from client" error
Fix query cancelation in case of Distributed queries with LIMIT (when
the initator does not required to read all the data), since this cannot
be done until the query was sent (from the Query packet up to the empty
data Block), otherwise you will get:
2020.11.21 21:47:23.297161 [ 184 ] {} <Error> TCPHandler: Code: 101, e.displayText() = DB::Exception: Unexpected packet Data received from client, Stack trace:
0. /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/exception:129: Poco::Exception::Exception(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, int) @ 0x244f5bc9 in /usr/bin/clickhouse
1. /build/obj-x86_64-linux-gnu/../src/Common/Exception.cpp:40: DB::Exception::Exception(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, int) @ 0xa14a421 in /usr/bin/clickhouse
2. /build/obj-x86_64-linux-gnu/../src/Common/NetException.h:0: DB::TCPHandler::receiveUnexpectedData() @ 0x1e032a74 in /usr/bin/clickhouse
3. /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:824: DB::TCPHandler::receivePacket() @ 0x1e024685 in /usr/bin/clickhouse
4. /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:178: DB::TCPHandler::runImpl() @ 0x1e01736b in /usr/bin/clickhouse
5. /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:0: DB::TCPHandler::run() @ 0x1e035c1b in /usr/bin/clickhouse
6. /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:57: Poco::Net::TCPServerConnection::start() @ 0x243559cf in /usr/bin/clickhouse
7. /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerDispatcher.cpp:0: Poco::Net::TCPServerDispatcher::run() @ 0x24356521 in /usr/bin/clickhouse
8. /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:0: Poco::PooledThread::run() @ 0x24609175 in /usr/bin/clickhouse
9. /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread_POSIX.cpp:0: Poco::ThreadImpl::runnableEntry(void*) @ 0x24603cb7 in /usr/bin/clickhouse
10. start_thread @ 0x9669 in /usr/lib/x86_64-linux-gnu/libpthread-2.30.so
11. __clone @ 0x1222b3 in /usr/lib/x86_64-linux-gnu/libc-2.30.so
2020-11-21 21:20:00 +00:00
|
|
|
/// Query cannot be canceled in the middle of the send query,
|
2020-11-28 05:37:54 +00:00
|
|
|
/// since there are multiple packets:
|
Fix "Unexpected packet Data received from client" error
Fix query cancelation in case of Distributed queries with LIMIT (when
the initator does not required to read all the data), since this cannot
be done until the query was sent (from the Query packet up to the empty
data Block), otherwise you will get:
2020.11.21 21:47:23.297161 [ 184 ] {} <Error> TCPHandler: Code: 101, e.displayText() = DB::Exception: Unexpected packet Data received from client, Stack trace:
0. /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/exception:129: Poco::Exception::Exception(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, int) @ 0x244f5bc9 in /usr/bin/clickhouse
1. /build/obj-x86_64-linux-gnu/../src/Common/Exception.cpp:40: DB::Exception::Exception(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, int) @ 0xa14a421 in /usr/bin/clickhouse
2. /build/obj-x86_64-linux-gnu/../src/Common/NetException.h:0: DB::TCPHandler::receiveUnexpectedData() @ 0x1e032a74 in /usr/bin/clickhouse
3. /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:824: DB::TCPHandler::receivePacket() @ 0x1e024685 in /usr/bin/clickhouse
4. /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:178: DB::TCPHandler::runImpl() @ 0x1e01736b in /usr/bin/clickhouse
5. /build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:0: DB::TCPHandler::run() @ 0x1e035c1b in /usr/bin/clickhouse
6. /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:57: Poco::Net::TCPServerConnection::start() @ 0x243559cf in /usr/bin/clickhouse
7. /build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerDispatcher.cpp:0: Poco::Net::TCPServerDispatcher::run() @ 0x24356521 in /usr/bin/clickhouse
8. /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/ThreadPool.cpp:0: Poco::PooledThread::run() @ 0x24609175 in /usr/bin/clickhouse
9. /build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread_POSIX.cpp:0: Poco::ThreadImpl::runnableEntry(void*) @ 0x24603cb7 in /usr/bin/clickhouse
10. start_thread @ 0x9669 in /usr/lib/x86_64-linux-gnu/libpthread-2.30.so
11. __clone @ 0x1222b3 in /usr/lib/x86_64-linux-gnu/libc-2.30.so
2020-11-21 21:20:00 +00:00
|
|
|
/// - Query
|
|
|
|
/// - Data (multiple times)
|
|
|
|
///
|
|
|
|
/// And after the Cancel packet none Data packet can be sent, otherwise the remote side will throw:
|
|
|
|
///
|
|
|
|
/// Unexpected packet Data received from client
|
|
|
|
///
|
|
|
|
std::lock_guard guard(was_cancelled_mutex);
|
2023-03-22 12:25:16 +00:00
|
|
|
sendQueryUnlocked(query_kind, async_callback);
|
|
|
|
}
|
|
|
|
|
|
|
|
void RemoteQueryExecutor::sendQueryUnlocked(ClientInfo::QueryKind query_kind, AsyncCallback async_callback)
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
2023-03-22 12:25:16 +00:00
|
|
|
if (sent_query || was_cancelled)
|
2020-06-02 16:27:05 +00:00
|
|
|
return;
|
|
|
|
|
2023-03-22 12:25:16 +00:00
|
|
|
connections = create_connections(async_callback);
|
|
|
|
AsyncCallbackSetter async_callback_setter(connections.get(), async_callback);
|
2020-06-02 16:27:05 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
const auto & settings = context->getSettingsRef();
|
2023-09-02 21:52:56 +00:00
|
|
|
if (isReplicaUnavailable() || needToSkipUnavailableShard())
|
2023-04-13 20:09:10 +00:00
|
|
|
{
|
|
|
|
/// To avoid sending the query again in the read(), we need to update the following flags:
|
|
|
|
was_cancelled = true;
|
|
|
|
finished = true;
|
|
|
|
sent_query = true;
|
|
|
|
|
2023-05-31 20:10:33 +00:00
|
|
|
/// We need to tell the coordinator not to wait for this replica.
|
|
|
|
if (extension && extension->parallel_reading_coordinator)
|
|
|
|
{
|
|
|
|
chassert(extension->replica_info);
|
|
|
|
extension->parallel_reading_coordinator->markReplicaAsUnavailable(extension->replica_info->number_of_current_replica);
|
|
|
|
}
|
|
|
|
|
2020-06-02 16:27:05 +00:00
|
|
|
return;
|
2023-04-13 20:09:10 +00:00
|
|
|
}
|
2020-06-02 16:27:05 +00:00
|
|
|
|
|
|
|
established = true;
|
|
|
|
|
|
|
|
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings);
|
2021-04-10 23:33:54 +00:00
|
|
|
ClientInfo modified_client_info = context->getClientInfo();
|
2022-03-08 15:42:29 +00:00
|
|
|
modified_client_info.query_kind = query_kind;
|
2020-06-02 16:27:05 +00:00
|
|
|
|
2023-04-11 18:18:11 +00:00
|
|
|
if (!duplicated_part_uuids.empty())
|
|
|
|
connections->sendIgnoredPartUUIDs(duplicated_part_uuids);
|
2020-11-20 17:23:53 +00:00
|
|
|
|
2021-01-19 19:21:06 +00:00
|
|
|
connections->sendQuery(timeouts, query, query_id, stage, modified_client_info, true);
|
2020-06-02 16:27:05 +00:00
|
|
|
|
|
|
|
established = false;
|
|
|
|
sent_query = true;
|
|
|
|
|
|
|
|
if (settings.enable_scalar_subquery_optimization)
|
|
|
|
sendScalars();
|
|
|
|
sendExternalTables();
|
|
|
|
}
|
|
|
|
|
2023-03-17 13:02:20 +00:00
|
|
|
int RemoteQueryExecutor::sendQueryAsync()
|
|
|
|
{
|
2023-03-24 20:34:21 +00:00
|
|
|
#if defined(OS_LINUX)
|
2023-03-23 19:52:37 +00:00
|
|
|
std::lock_guard lock(was_cancelled_mutex);
|
|
|
|
if (was_cancelled)
|
|
|
|
return -1;
|
|
|
|
|
2023-03-17 13:02:20 +00:00
|
|
|
if (!read_context)
|
|
|
|
read_context = std::make_unique<ReadContext>(*this, /*suspend_when_query_sent*/ true);
|
|
|
|
|
|
|
|
/// If query already sent, do nothing. Note that we cannot use sent_query flag here,
|
|
|
|
/// because we can still be in process of sending scalars or external tables.
|
|
|
|
if (read_context->isQuerySent())
|
|
|
|
return -1;
|
|
|
|
|
|
|
|
read_context->resume();
|
|
|
|
|
2023-04-20 11:56:20 +00:00
|
|
|
if (read_context->isQuerySent())
|
|
|
|
return -1;
|
|
|
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::SuspendSendingQueryToShard); /// Mostly for testing purposes.
|
|
|
|
return read_context->getFileDescriptor();
|
2023-03-24 20:34:21 +00:00
|
|
|
#else
|
|
|
|
sendQuery();
|
|
|
|
return -1;
|
|
|
|
#endif
|
2023-03-17 13:02:20 +00:00
|
|
|
}
|
2023-02-03 13:34:18 +00:00
|
|
|
|
|
|
|
Block RemoteQueryExecutor::readBlock()
|
|
|
|
{
|
|
|
|
while (true)
|
|
|
|
{
|
|
|
|
auto res = read();
|
|
|
|
|
|
|
|
if (res.getType() == ReadResult::Type::Data)
|
|
|
|
return res.getBlock();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
RemoteQueryExecutor::ReadResult RemoteQueryExecutor::read()
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
|
|
|
if (!sent_query)
|
|
|
|
{
|
|
|
|
sendQuery();
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
if (context->getSettingsRef().skip_unavailable_shards && (0 == connections->size()))
|
2023-02-03 13:34:18 +00:00
|
|
|
return ReadResult(Block());
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
while (true)
|
|
|
|
{
|
Fix possible "No more packets are available" for distributed queries
CI founds the following case:
<details>
2022.05.25 22:36:06.778808 [ 3037 ] {77743723-1fcd-4b3d-babc-d0615e3ff40e} <Fatal> : Logical error: 'No more packets are available.'.
2022.05.25 22:42:24.960075 [ 17397 ] {} <Fatal> BaseDaemon: ########################################
2022.05.25 22:42:24.971173 [ 17397 ] {} <Fatal> BaseDaemon: (version 22.6.1.1, build id: 9A1F9489854CED36) (from thread 3037) (query_id: 77743723-1fcd-4b3d-babc-d0615e3ff40e) (query: SELECT * FROM
2022.05.25 22:42:25.046871 [ 17397 ] {} <Fatal> BaseDaemon: 5. ./build_docker/../src/Common/Exception.cpp:47: DB::abortOnFailedAssertion()
2022.05.25 22:42:25.181449 [ 17397 ] {} <Fatal> BaseDaemon: 6. ./build_docker/../src/Common/Exception.cpp:70: DB::Exception::Exception()
2022.05.25 22:42:25.367710 [ 17397 ] {} <Fatal> BaseDaemon: 7. ./build_docker/../src/Client/MultiplexedConnections.cpp:0: DB::MultiplexedConnections::receivePacketUnlocked()
2022.05.25 22:42:25.414201 [ 17397 ] {} <Fatal> BaseDaemon: 8. ./build_docker/../src/Client/MultiplexedConnections.cpp:0: DB::MultiplexedConnections::receivePacket()
2022.05.25 22:42:25.493066 [ 17397 ] {} <Fatal> BaseDaemon: 9. ./build_docker/../src/QueryPipeline/RemoteQueryExecutor.cpp:279: DB::RemoteQueryExecutor::read()
2022.05.25 22:42:25.612679 [ 17397 ] {} <Fatal> BaseDaemon: 10. ./build_docker/../src/Processors/Sources/RemoteSource.cpp:0: DB::RemoteSource::tryGenerate()
Here are additional logs for this query:
$ pigz -cd clickhouse-server.stress.log.gz | fgrep -a 77743723-1fcd-4b3d-babc-d0615e3ff40e | fgrep -e Connection -e Distributed -e Fatal
2022.05.25 22:36:04.698671 [ 6613 ] {77743723-1fcd-4b3d-babc-d0615e3ff40e} <Trace> Connection (127.0.0.2:9000): Connecting. Database: (not specified). User: default
2022.05.25 22:36:04.722568 [ 3419 ] {77743723-1fcd-4b3d-babc-d0615e3ff40e} <Trace> Connection (127.0.0.2:9000): Connecting. Database: (not specified). User: default
2022.05.25 22:36:05.014432 [ 6613 ] {77743723-1fcd-4b3d-babc-d0615e3ff40e} <Trace> Connection (127.0.0.2:9000): Connected to ClickHouse server version 22.6.1.
2022.05.25 22:36:05.091397 [ 6613 ] {77743723-1fcd-4b3d-babc-d0615e3ff40e} <Debug> Connection (127.0.0.2:9000): Sent data for 2 scalars, total 2 rows in 0.000125814 sec., 15602 rows/sec., 68.00 B (517.81 KiB/sec.), compressed 0.4594594594594595 times to 148.00 B (1.10 MiB/sec.)
2022.05.25 22:36:05.301301 [ 3419 ] {77743723-1fcd-4b3d-babc-d0615e3ff40e} <Trace> Connection (127.0.0.2:9000): Connected to ClickHouse server version 22.6.1.
2022.05.25 22:36:05.343140 [ 3419 ] {77743723-1fcd-4b3d-babc-d0615e3ff40e} <Debug> Connection (127.0.0.2:9000): Sent data for 2 scalars, total 2 rows in 0.000116304 sec., 16889 rows/sec., 68.00 B (559.80 KiB/sec.), compressed 0.4594594594594595 times to 148.00 B (1.19 MiB/sec.)
2022.05.25 22:36:06.682535 [ 6613 ] {77743723-1fcd-4b3d-babc-d0615e3ff40e} <Trace> StorageDistributed (remote): (127.0.0.2:9000) Cancelling query because enough data has been read
2022.05.25 22:36:06.778808 [ 3037 ] {77743723-1fcd-4b3d-babc-d0615e3ff40e} <Fatal> : Logical error: 'No more packets are available.'.
2022.05.25 22:36:06.789505 [ 3419 ] {77743723-1fcd-4b3d-babc-d0615e3ff40e} <Trace> StorageDistributed (remote): (127.0.0.2:9000) Cancelling query because enough data has been read
2022.05.25 22:42:24.971173 [ 17397 ] {} <Fatal> BaseDaemon: (version 22.6.1.1, build id: 9A1F9489854CED36) (from thread 3037) (query_id: 77743723-1fcd-4b3d-babc-d0615e3ff40e) (query: SELECT * FROM
</details>
So between cancelling different sources the LOGICAL_ERROR occured, I
believe that this is because of the race:
T1: T2:
RemoteQueryExecutor::read()
checks was_cancelled
RemoteQueryExecutor::tryCancel()
connections->cancel()
calls connections->receivePacket()
Note, for this problem async_socket_for_remote/use_hedged_requests
should be disabled, and original settings was:
- --max_parallel_replicas=3
- --use_hedged_requests=false
- --allow_experimental_parallel_reading_from_replicas=3
CI: https://s3.amazonaws.com/clickhouse-test-reports/37469/41cb029ed23e77f3a108e07e6b1b1bcb03dc7fcf/stress_test__undefined__actions_/fatal_messages.txt
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-06-03 11:49:22 +00:00
|
|
|
std::lock_guard lock(was_cancelled_mutex);
|
2020-06-02 16:27:05 +00:00
|
|
|
if (was_cancelled)
|
2023-02-03 13:34:18 +00:00
|
|
|
return ReadResult(Block());
|
2020-06-02 16:27:05 +00:00
|
|
|
|
2023-02-03 13:34:18 +00:00
|
|
|
auto packet = connections->receivePacket();
|
|
|
|
auto anything = processPacket(std::move(packet));
|
2020-06-02 16:27:05 +00:00
|
|
|
|
2023-02-03 13:34:18 +00:00
|
|
|
if (anything.getType() == ReadResult::Type::Data || anything.getType() == ReadResult::Type::ParallelReplicasToken)
|
|
|
|
return anything;
|
|
|
|
|
|
|
|
if (got_duplicated_part_uuids)
|
2023-03-23 19:52:37 +00:00
|
|
|
break;
|
2020-12-02 17:02:14 +00:00
|
|
|
}
|
2023-03-23 19:52:37 +00:00
|
|
|
|
|
|
|
return restartQueryWithoutDuplicatedUUIDs();
|
2020-12-02 17:02:14 +00:00
|
|
|
}
|
|
|
|
|
2023-03-17 13:02:20 +00:00
|
|
|
RemoteQueryExecutor::ReadResult RemoteQueryExecutor::readAsync()
|
2020-12-02 17:02:14 +00:00
|
|
|
{
|
2020-12-18 13:15:03 +00:00
|
|
|
#if defined(OS_LINUX)
|
2023-03-03 19:30:43 +00:00
|
|
|
if (!read_context || (resent_query && recreate_read_context))
|
2020-12-02 17:02:14 +00:00
|
|
|
{
|
2020-12-16 20:27:31 +00:00
|
|
|
std::lock_guard lock(was_cancelled_mutex);
|
2023-03-23 19:52:37 +00:00
|
|
|
if (was_cancelled)
|
2023-02-03 13:34:18 +00:00
|
|
|
return ReadResult(Block());
|
2023-03-23 19:52:37 +00:00
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
read_context = std::make_unique<ReadContext>(*this);
|
|
|
|
recreate_read_context = false;
|
2020-12-02 17:02:14 +00:00
|
|
|
}
|
|
|
|
|
2023-02-28 14:43:49 +00:00
|
|
|
while (true)
|
2020-12-16 20:27:31 +00:00
|
|
|
{
|
|
|
|
std::lock_guard lock(was_cancelled_mutex);
|
2020-12-17 10:07:28 +00:00
|
|
|
if (was_cancelled)
|
2023-02-03 13:34:18 +00:00
|
|
|
return ReadResult(Block());
|
2020-12-17 10:07:28 +00:00
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
read_context->resume();
|
2020-12-02 17:02:14 +00:00
|
|
|
|
2023-09-02 21:52:56 +00:00
|
|
|
if (isReplicaUnavailable() || needToSkipUnavailableShard())
|
2023-05-31 20:10:33 +00:00
|
|
|
{
|
|
|
|
/// We need to tell the coordinator not to wait for this replica.
|
|
|
|
/// But at this point it may lead to an incomplete result set, because
|
|
|
|
/// this replica committed to read some part of there data and then died.
|
|
|
|
if (extension && extension->parallel_reading_coordinator)
|
|
|
|
{
|
|
|
|
chassert(extension->parallel_reading_coordinator);
|
|
|
|
extension->parallel_reading_coordinator->markReplicaAsUnavailable(extension->replica_info->number_of_current_replica);
|
|
|
|
}
|
|
|
|
|
2023-02-03 13:34:18 +00:00
|
|
|
return ReadResult(Block());
|
2023-05-31 20:10:33 +00:00
|
|
|
}
|
2020-12-02 17:02:14 +00:00
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
/// Check if packet is not ready yet.
|
|
|
|
if (read_context->isInProgress())
|
|
|
|
return ReadResult(read_context->getFileDescriptor());
|
Fix possible data-race in case of query cancellation with async_socket_for_remote
RemoteQueryExecutor::cancel() (that is called in another thread), steal
the fiber, and process any pending packets, to leave the connection in
the correct state (so that it can be reused for further queries).
However this requires processing pending packets, and this will update
the RemoteQueryExecutorReadContext::packet field, which can be read in
another thread by the RemoteQueryExecutor::read().
This was pretty tricky due to fibers, but AFAICS I understand this
correctly and this should fix the race.
Note, that if you will look at the logs from the #28854, you will see,
that all those data races was triggered after query cancellation.
Fixes: #28854
Refs: #18715
v2: fix daedlock in case of duplicated parts
v3: do not hold the mutex, since was_cancelled is atomic
2021-11-11 20:35:20 +00:00
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
auto anything = processPacket(read_context->getPacket());
|
2023-02-03 13:34:18 +00:00
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
if (anything.getType() == ReadResult::Type::Data || anything.getType() == ReadResult::Type::ParallelReplicasToken)
|
|
|
|
return anything;
|
2023-02-03 13:34:18 +00:00
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
if (got_duplicated_part_uuids)
|
2023-03-23 19:52:37 +00:00
|
|
|
break;
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
2023-03-23 19:52:37 +00:00
|
|
|
|
|
|
|
return restartQueryWithoutDuplicatedUUIDs();
|
2020-12-18 13:15:03 +00:00
|
|
|
#else
|
|
|
|
return read();
|
2020-12-14 16:16:08 +00:00
|
|
|
#endif
|
2020-12-18 13:15:03 +00:00
|
|
|
}
|
2020-12-02 17:02:14 +00:00
|
|
|
|
2021-02-05 09:54:34 +00:00
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
RemoteQueryExecutor::ReadResult RemoteQueryExecutor::restartQueryWithoutDuplicatedUUIDs()
|
2021-02-05 09:54:34 +00:00
|
|
|
{
|
|
|
|
{
|
2023-03-23 19:52:37 +00:00
|
|
|
std::lock_guard lock(was_cancelled_mutex);
|
|
|
|
if (was_cancelled)
|
|
|
|
return ReadResult(Block());
|
|
|
|
|
|
|
|
/// Cancel previous query and disconnect before retry.
|
|
|
|
cancelUnlocked();
|
|
|
|
connections->disconnect();
|
|
|
|
|
|
|
|
/// Only resend once, otherwise throw an exception
|
|
|
|
if (resent_query)
|
|
|
|
throw Exception(ErrorCodes::DUPLICATED_PART_UUIDS, "Found duplicate uuids while processing query");
|
|
|
|
|
2021-02-05 09:54:34 +00:00
|
|
|
if (log)
|
|
|
|
LOG_DEBUG(log, "Found duplicate UUIDs, will retry query without those parts");
|
|
|
|
|
|
|
|
resent_query = true;
|
2023-03-03 19:30:43 +00:00
|
|
|
recreate_read_context = true;
|
2021-02-05 09:54:34 +00:00
|
|
|
sent_query = false;
|
|
|
|
got_duplicated_part_uuids = false;
|
2023-03-23 19:52:37 +00:00
|
|
|
was_cancelled = false;
|
2021-02-05 09:54:34 +00:00
|
|
|
}
|
2023-03-23 19:52:37 +00:00
|
|
|
|
|
|
|
/// Consecutive read will implicitly send query first.
|
|
|
|
if (!read_context)
|
|
|
|
return read();
|
|
|
|
else
|
|
|
|
return readAsync();
|
2021-02-05 09:54:34 +00:00
|
|
|
}
|
|
|
|
|
2023-02-03 13:34:18 +00:00
|
|
|
RemoteQueryExecutor::ReadResult RemoteQueryExecutor::processPacket(Packet packet)
|
2020-12-02 17:02:14 +00:00
|
|
|
{
|
|
|
|
switch (packet.type)
|
|
|
|
{
|
2021-12-09 10:39:28 +00:00
|
|
|
case Protocol::Server::MergeTreeReadTaskRequest:
|
2023-06-28 10:30:15 +00:00
|
|
|
chassert(packet.request.has_value());
|
|
|
|
processMergeTreeReadTaskRequest(packet.request.value());
|
2023-02-03 13:34:18 +00:00
|
|
|
return ReadResult(ReadResult::Type::ParallelReplicasToken);
|
|
|
|
|
2023-09-04 13:53:06 +00:00
|
|
|
case Protocol::Server::MergeTreeAllRangesAnnouncement:
|
2023-06-28 10:30:15 +00:00
|
|
|
chassert(packet.announcement.has_value());
|
2023-09-04 13:53:06 +00:00
|
|
|
processMergeTreeInitialReadAnnouncement(packet.announcement.value());
|
2023-02-03 13:34:18 +00:00
|
|
|
return ReadResult(ReadResult::Type::ParallelReplicasToken);
|
|
|
|
|
2021-04-06 11:05:47 +00:00
|
|
|
case Protocol::Server::ReadTaskRequest:
|
2021-04-10 02:21:18 +00:00
|
|
|
processReadTaskRequest();
|
2021-04-06 11:05:47 +00:00
|
|
|
break;
|
2020-11-20 17:23:53 +00:00
|
|
|
case Protocol::Server::PartUUIDs:
|
|
|
|
if (!setPartUUIDs(packet.part_uuids))
|
|
|
|
got_duplicated_part_uuids = true;
|
|
|
|
break;
|
2020-12-02 17:02:14 +00:00
|
|
|
case Protocol::Server::Data:
|
2023-01-02 12:15:31 +00:00
|
|
|
/// Note: `packet.block.rows() > 0` means it's a header block.
|
|
|
|
/// We can actually return it, and the first call to RemoteQueryExecutor::read
|
|
|
|
/// will return earlier. We should consider doing it.
|
|
|
|
if (packet.block && (packet.block.rows() > 0))
|
2023-02-03 13:34:18 +00:00
|
|
|
return ReadResult(adaptBlockStructure(packet.block, header));
|
2020-12-02 17:02:14 +00:00
|
|
|
break; /// If the block is empty - we will receive other packets before EndOfStream.
|
|
|
|
|
|
|
|
case Protocol::Server::Exception:
|
|
|
|
got_exception_from_replica = true;
|
|
|
|
packet.exception->rethrow();
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::EndOfStream:
|
2021-01-19 19:21:06 +00:00
|
|
|
if (!connections->hasActiveConnections())
|
2020-12-02 17:02:14 +00:00
|
|
|
{
|
|
|
|
finished = true;
|
2023-02-03 13:34:18 +00:00
|
|
|
/// TODO: Replace with Type::Finished
|
|
|
|
return ReadResult(Block{});
|
2020-12-02 17:02:14 +00:00
|
|
|
}
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::Progress:
|
|
|
|
/** We use the progress from a remote server.
|
|
|
|
* We also include in ProcessList,
|
|
|
|
* and we use it to check
|
|
|
|
* constraints (for example, the minimum speed of query execution)
|
|
|
|
* and quotas (for example, the number of lines to read).
|
|
|
|
*/
|
|
|
|
if (progress_callback)
|
|
|
|
progress_callback(packet.progress);
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::ProfileInfo:
|
|
|
|
/// Use own (client-side) info about read bytes, it is more correct info than server-side one.
|
|
|
|
if (profile_info_callback)
|
|
|
|
profile_info_callback(packet.profile_info);
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::Totals:
|
|
|
|
totals = packet.block;
|
2022-12-30 16:55:56 +00:00
|
|
|
if (totals)
|
|
|
|
totals = adaptBlockStructure(totals, header);
|
2020-12-02 17:02:14 +00:00
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::Extremes:
|
|
|
|
extremes = packet.block;
|
2022-12-30 16:55:56 +00:00
|
|
|
if (extremes)
|
|
|
|
extremes = adaptBlockStructure(packet.block, header);
|
2020-12-02 17:02:14 +00:00
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::Log:
|
|
|
|
/// Pass logs from remote server to client
|
|
|
|
if (auto log_queue = CurrentThread::getInternalTextLogsQueue())
|
|
|
|
log_queue->pushBlock(std::move(packet.block));
|
|
|
|
break;
|
|
|
|
|
2021-08-30 11:04:59 +00:00
|
|
|
case Protocol::Server::ProfileEvents:
|
2021-09-01 14:47:12 +00:00
|
|
|
/// Pass profile events from remote server to client
|
2021-09-20 21:52:01 +00:00
|
|
|
if (auto profile_queue = CurrentThread::getInternalProfileEventsQueue())
|
2021-10-12 21:15:05 +00:00
|
|
|
if (!profile_queue->emplace(std::move(packet.block)))
|
|
|
|
throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push into profile queue");
|
2021-09-20 21:52:01 +00:00
|
|
|
break;
|
2021-08-30 11:04:59 +00:00
|
|
|
|
2023-04-12 19:20:12 +00:00
|
|
|
case Protocol::Server::TimezoneUpdate:
|
|
|
|
break;
|
|
|
|
|
2020-12-02 17:02:14 +00:00
|
|
|
default:
|
|
|
|
got_unknown_packet_from_replica = true;
|
2022-04-24 10:34:50 +00:00
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::UNKNOWN_PACKET_FROM_SERVER,
|
|
|
|
"Unknown packet {} from one of the following replicas: {}",
|
|
|
|
packet.type,
|
2021-01-19 19:21:06 +00:00
|
|
|
connections->dumpAddresses());
|
2020-12-02 17:02:14 +00:00
|
|
|
}
|
|
|
|
|
2023-02-03 13:34:18 +00:00
|
|
|
return ReadResult(ReadResult::Type::Nothing);
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
|
|
|
|
2020-11-20 17:23:53 +00:00
|
|
|
bool RemoteQueryExecutor::setPartUUIDs(const std::vector<UUID> & uuids)
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
auto query_context = context->getQueryContext();
|
|
|
|
auto duplicates = query_context->getPartUUIDs()->add(uuids);
|
2020-11-20 17:23:53 +00:00
|
|
|
|
|
|
|
if (!duplicates.empty())
|
|
|
|
{
|
|
|
|
duplicated_part_uuids.insert(duplicated_part_uuids.begin(), duplicates.begin(), duplicates.end());
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2021-04-10 02:21:18 +00:00
|
|
|
void RemoteQueryExecutor::processReadTaskRequest()
|
2021-04-06 11:05:47 +00:00
|
|
|
{
|
2023-05-31 20:10:33 +00:00
|
|
|
if (!extension || !extension->task_iterator)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Distributed task iterator is not initialized");
|
2023-02-12 09:04:05 +00:00
|
|
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReadTaskRequestsReceived);
|
2023-05-31 20:10:33 +00:00
|
|
|
auto response = (*extension->task_iterator)();
|
2021-04-08 19:00:39 +00:00
|
|
|
connections->sendReadTaskResponse(response);
|
2021-04-06 11:05:47 +00:00
|
|
|
}
|
|
|
|
|
2023-02-03 13:34:18 +00:00
|
|
|
void RemoteQueryExecutor::processMergeTreeReadTaskRequest(ParallelReadRequest request)
|
2021-12-09 10:39:28 +00:00
|
|
|
{
|
2023-05-31 20:10:33 +00:00
|
|
|
if (!extension || !extension->parallel_reading_coordinator)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized");
|
2021-12-09 10:39:28 +00:00
|
|
|
|
2023-02-12 09:04:05 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::MergeTreeReadTaskRequestsReceived);
|
2023-05-31 20:10:33 +00:00
|
|
|
auto response = extension->parallel_reading_coordinator->handleRequest(std::move(request));
|
2021-12-09 10:39:28 +00:00
|
|
|
connections->sendMergeTreeReadTaskResponse(response);
|
|
|
|
}
|
|
|
|
|
2023-09-04 13:53:06 +00:00
|
|
|
void RemoteQueryExecutor::processMergeTreeInitialReadAnnouncement(InitialAllRangesAnnouncement announcement)
|
2023-02-03 13:34:18 +00:00
|
|
|
{
|
2023-05-31 20:10:33 +00:00
|
|
|
if (!extension || !extension->parallel_reading_coordinator)
|
2023-02-03 13:34:18 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized");
|
|
|
|
|
2023-10-18 12:20:14 +00:00
|
|
|
extension->parallel_reading_coordinator->handleInitialAllRangesAnnouncement(std::move(announcement));
|
2023-02-03 13:34:18 +00:00
|
|
|
}
|
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
void RemoteQueryExecutor::finish()
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
2023-03-23 19:52:37 +00:00
|
|
|
std::lock_guard guard(was_cancelled_mutex);
|
|
|
|
|
2020-06-02 16:27:05 +00:00
|
|
|
/** If one of:
|
|
|
|
* - nothing started to do;
|
|
|
|
* - received all packets before EndOfStream;
|
|
|
|
* - received exception from one replica;
|
|
|
|
* - received an unknown packet from one replica;
|
|
|
|
* then you do not need to read anything.
|
|
|
|
*/
|
|
|
|
if (!isQueryPending() || hasThrownException())
|
|
|
|
return;
|
|
|
|
|
|
|
|
/** If you have not read all the data yet, but they are no longer needed.
|
|
|
|
* This may be due to the fact that the data is sufficient (for example, when using LIMIT).
|
|
|
|
*/
|
|
|
|
|
|
|
|
/// Send the request to abort the execution of the request, if not already sent.
|
2023-03-03 19:30:43 +00:00
|
|
|
tryCancel("Cancelling query because enough data has been read");
|
2023-03-11 18:05:47 +00:00
|
|
|
|
2023-07-25 17:21:41 +00:00
|
|
|
/// If connections weren't created yet, query wasn't sent or was already finished, nothing to do.
|
|
|
|
if (!connections || !sent_query || finished)
|
2023-03-17 13:02:20 +00:00
|
|
|
return;
|
2023-03-11 18:05:47 +00:00
|
|
|
|
|
|
|
/// Get the remaining packets so that there is no out of sync in the connections to the replicas.
|
2023-09-04 13:53:06 +00:00
|
|
|
/// We do this manually instead of calling drain() because we want to process Log, ProfileEvents and Progress
|
|
|
|
/// packets that had been sent before the connection is fully finished in order to have final statistics of what
|
|
|
|
/// was executed in the remote queries
|
|
|
|
while (connections->hasActiveConnections() && !finished)
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
2023-09-04 13:53:06 +00:00
|
|
|
Packet packet = connections->receivePacket();
|
2023-04-12 19:31:13 +00:00
|
|
|
|
2023-09-04 13:53:06 +00:00
|
|
|
switch (packet.type)
|
|
|
|
{
|
|
|
|
case Protocol::Server::EndOfStream:
|
|
|
|
finished = true;
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::Exception:
|
|
|
|
got_exception_from_replica = true;
|
|
|
|
packet.exception->rethrow();
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::Log:
|
|
|
|
/// Pass logs from remote server to client
|
|
|
|
if (auto log_queue = CurrentThread::getInternalTextLogsQueue())
|
|
|
|
log_queue->pushBlock(std::move(packet.block));
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::ProfileEvents:
|
|
|
|
/// Pass profile events from remote server to client
|
|
|
|
if (auto profile_queue = CurrentThread::getInternalProfileEventsQueue())
|
|
|
|
if (!profile_queue->emplace(std::move(packet.block)))
|
|
|
|
throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push into profile queue");
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::ProfileInfo:
|
|
|
|
/// Use own (client-side) info about read bytes, it is more correct info than server-side one.
|
|
|
|
if (profile_info_callback)
|
|
|
|
profile_info_callback(packet.profile_info);
|
|
|
|
break;
|
|
|
|
|
|
|
|
case Protocol::Server::Progress:
|
|
|
|
if (progress_callback)
|
|
|
|
progress_callback(packet.progress);
|
|
|
|
break;
|
|
|
|
|
|
|
|
default:
|
|
|
|
break;
|
|
|
|
}
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
void RemoteQueryExecutor::cancel()
|
2023-03-23 19:52:37 +00:00
|
|
|
{
|
|
|
|
std::lock_guard guard(was_cancelled_mutex);
|
|
|
|
cancelUnlocked();
|
|
|
|
}
|
|
|
|
|
|
|
|
void RemoteQueryExecutor::cancelUnlocked()
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
|
|
|
{
|
|
|
|
std::lock_guard lock(external_tables_mutex);
|
|
|
|
|
|
|
|
/// Stop sending external data.
|
|
|
|
for (auto & vec : external_tables_data)
|
|
|
|
for (auto & elem : vec)
|
|
|
|
elem->is_cancelled = true;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!isQueryPending() || hasThrownException())
|
|
|
|
return;
|
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
tryCancel("Cancelling query");
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void RemoteQueryExecutor::sendScalars()
|
|
|
|
{
|
2021-01-19 19:21:06 +00:00
|
|
|
connections->sendScalarsData(scalars);
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void RemoteQueryExecutor::sendExternalTables()
|
|
|
|
{
|
2021-01-19 19:21:06 +00:00
|
|
|
size_t count = connections->size();
|
2020-06-02 16:27:05 +00:00
|
|
|
|
|
|
|
{
|
|
|
|
std::lock_guard lock(external_tables_mutex);
|
|
|
|
|
2020-11-20 17:23:53 +00:00
|
|
|
external_tables_data.clear();
|
2020-06-02 16:27:05 +00:00
|
|
|
external_tables_data.reserve(count);
|
|
|
|
|
2021-11-25 15:41:50 +00:00
|
|
|
StreamLocalLimits limits;
|
|
|
|
const auto & settings = context->getSettingsRef();
|
|
|
|
limits.mode = LimitsMode::LIMITS_TOTAL;
|
|
|
|
limits.speed_limits.max_execution_time = settings.max_execution_time;
|
|
|
|
limits.timeout_overflow_mode = settings.timeout_overflow_mode;
|
2024-01-19 08:11:30 +00:00
|
|
|
limits.speed_limits.max_estimated_execution_time = settings.max_estimated_execution_time;
|
2021-11-25 15:41:50 +00:00
|
|
|
|
2020-06-02 16:27:05 +00:00
|
|
|
for (size_t i = 0; i < count; ++i)
|
|
|
|
{
|
|
|
|
ExternalTablesData res;
|
|
|
|
for (const auto & table : external_tables)
|
|
|
|
{
|
|
|
|
StoragePtr cur = table.second;
|
2023-02-28 07:56:10 +00:00
|
|
|
/// Send only temporary tables with StorageMemory
|
|
|
|
if (!std::dynamic_pointer_cast<StorageMemory>(cur))
|
|
|
|
continue;
|
2020-06-02 16:27:05 +00:00
|
|
|
|
|
|
|
auto data = std::make_unique<ExternalTableData>();
|
|
|
|
data->table_name = table.first;
|
2023-05-13 00:57:31 +00:00
|
|
|
data->creating_pipe_callback = [cur, limits, my_context = this->context]()
|
2021-04-30 14:19:48 +00:00
|
|
|
{
|
|
|
|
SelectQueryInfo query_info;
|
|
|
|
auto metadata_snapshot = cur->getInMemoryMetadataPtr();
|
2023-05-13 00:57:31 +00:00
|
|
|
auto storage_snapshot = cur->getStorageSnapshot(metadata_snapshot, my_context);
|
2021-04-30 14:19:48 +00:00
|
|
|
QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage(
|
2023-05-13 00:57:31 +00:00
|
|
|
my_context, QueryProcessingStage::Complete, storage_snapshot, query_info);
|
2021-04-30 14:19:48 +00:00
|
|
|
|
2022-05-23 19:47:32 +00:00
|
|
|
QueryPlan plan;
|
|
|
|
cur->read(
|
|
|
|
plan,
|
2021-04-30 14:19:48 +00:00
|
|
|
metadata_snapshot->getColumns().getNamesOfPhysical(),
|
2023-05-13 00:57:31 +00:00
|
|
|
storage_snapshot, query_info, my_context,
|
2021-04-30 14:19:48 +00:00
|
|
|
read_from_table_stage, DEFAULT_BLOCK_SIZE, 1);
|
|
|
|
|
2022-05-23 19:47:32 +00:00
|
|
|
auto builder = plan.buildQueryPipeline(
|
2023-05-13 00:57:31 +00:00
|
|
|
QueryPlanOptimizationSettings::fromContext(my_context),
|
|
|
|
BuildQueryPipelineSettings::fromContext(my_context));
|
2020-06-02 16:27:05 +00:00
|
|
|
|
2022-05-23 19:47:32 +00:00
|
|
|
builder->resize(1);
|
|
|
|
builder->addTransform(std::make_shared<LimitsCheckingTransform>(builder->getHeader(), limits));
|
2021-11-25 15:41:50 +00:00
|
|
|
|
2022-05-23 19:47:32 +00:00
|
|
|
return builder;
|
2021-04-30 14:19:48 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
data->pipe = data->creating_pipe_callback();
|
2020-06-02 16:27:05 +00:00
|
|
|
res.emplace_back(std::move(data));
|
|
|
|
}
|
|
|
|
external_tables_data.push_back(std::move(res));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-01-19 19:21:06 +00:00
|
|
|
connections->sendExternalTablesData(external_tables_data);
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
void RemoteQueryExecutor::tryCancel(const char * reason)
|
2020-06-02 16:27:05 +00:00
|
|
|
{
|
2021-07-14 13:17:30 +00:00
|
|
|
if (was_cancelled)
|
|
|
|
return;
|
2020-06-02 16:27:05 +00:00
|
|
|
|
2021-07-14 13:17:30 +00:00
|
|
|
was_cancelled = true;
|
2020-12-17 10:07:28 +00:00
|
|
|
|
2023-03-03 19:30:43 +00:00
|
|
|
if (read_context)
|
|
|
|
read_context->cancel();
|
2020-12-17 10:07:28 +00:00
|
|
|
|
2023-05-03 16:16:23 +00:00
|
|
|
/// Query could be cancelled during connection creation, query sending or data receiving.
|
|
|
|
/// We should send cancel request if connections were already created, query were sent
|
|
|
|
/// and remote query is not finished.
|
|
|
|
if (connections && sent_query && !finished)
|
2021-08-19 06:06:54 +00:00
|
|
|
{
|
2023-03-17 13:02:20 +00:00
|
|
|
connections->sendCancel();
|
|
|
|
if (log)
|
|
|
|
LOG_TRACE(log, "({}) {}", connections->dumpAddresses(), reason);
|
2021-08-19 06:06:54 +00:00
|
|
|
}
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool RemoteQueryExecutor::isQueryPending() const
|
|
|
|
{
|
2023-03-21 16:01:54 +00:00
|
|
|
return (sent_query || read_context) && !finished;
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool RemoteQueryExecutor::hasThrownException() const
|
|
|
|
{
|
|
|
|
return got_exception_from_replica || got_unknown_packet_from_replica;
|
|
|
|
}
|
|
|
|
|
2023-10-12 20:34:26 +00:00
|
|
|
void RemoteQueryExecutor::setProgressCallback(ProgressCallback callback)
|
|
|
|
{
|
2023-11-21 13:07:14 +00:00
|
|
|
std::lock_guard guard(was_cancelled_mutex);
|
2023-10-12 20:34:26 +00:00
|
|
|
progress_callback = std::move(callback);
|
|
|
|
|
|
|
|
if (extension && extension->parallel_reading_coordinator)
|
|
|
|
extension->parallel_reading_coordinator->setProgressCallback(progress_callback);
|
|
|
|
}
|
|
|
|
|
2023-11-21 13:07:14 +00:00
|
|
|
void RemoteQueryExecutor::setProfileInfoCallback(ProfileInfoCallback callback)
|
|
|
|
{
|
|
|
|
std::lock_guard guard(was_cancelled_mutex);
|
|
|
|
profile_info_callback = std::move(callback);
|
|
|
|
}
|
2020-06-02 16:27:05 +00:00
|
|
|
}
|