ClickHouse/src/Client/LocalConnection.cpp

405 lines
10 KiB
C++
Raw Normal View History

2021-08-17 19:59:51 +00:00
#include "LocalConnection.h"
#include <Interpreters/executeQuery.h>
2021-09-29 19:03:30 +00:00
#include <Processors/Executors/CompletedPipelineExecutor.h>
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
#include <Processors/Executors/PushingPipelineExecutor.h>
#include <Processors/Executors/PushingAsyncPipelineExecutor.h>
2021-08-18 14:39:04 +00:00
#include <Storages/IStorage.h>
2021-08-30 11:04:59 +00:00
#include "Core/Protocol.h"
2021-08-17 19:59:51 +00:00
2021-09-26 21:22:04 +00:00
2021-08-17 19:59:51 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_PACKET_FROM_SERVER;
2021-08-19 20:27:40 +00:00
extern const int UNKNOWN_EXCEPTION;
2021-09-04 18:19:01 +00:00
extern const int NOT_IMPLEMENTED;
2021-08-17 19:59:51 +00:00
}
2021-09-11 17:16:37 +00:00
LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_)
2021-08-17 19:59:51 +00:00
: WithContext(context_)
2021-09-04 18:19:01 +00:00
, session(getContext(), ClientInfo::Interface::LOCAL)
2021-09-11 17:16:37 +00:00
, send_progress(send_progress_)
2021-08-17 19:59:51 +00:00
{
2021-08-20 08:38:50 +00:00
/// Authenticate and create a context to execute queries.
session.authenticate("default", "", Poco::Net::SocketAddress{});
2021-09-24 08:29:01 +00:00
session.makeSessionContext();
2021-09-11 11:34:22 +00:00
if (!CurrentThread::isInitialized())
thread_status.emplace();
2021-08-17 19:59:51 +00:00
}
2021-08-21 10:55:54 +00:00
LocalConnection::~LocalConnection()
{
try
{
state.reset();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
bool LocalConnection::hasReadPendingData() const
{
return !state->is_finished;
}
std::optional<UInt64> LocalConnection::checkPacket(size_t)
{
return next_packet_type;
}
void LocalConnection::updateProgress(const Progress & value)
{
state->progress.incrementPiecewiseAtomically(value);
}
2021-08-17 19:59:51 +00:00
void LocalConnection::sendQuery(
const ConnectionTimeouts &,
const String & query_,
const String & query_id_,
UInt64,
const Settings *,
const ClientInfo *,
bool)
{
2021-09-24 08:29:01 +00:00
query_context = session.makeQueryContext();
query_context->setCurrentQueryId(query_id_);
if (send_progress)
query_context->setProgressCallback([this] (const Progress & value) { return this->updateProgress(value); });
CurrentThread::QueryScope query_scope_holder(query_context);
2021-08-19 20:27:40 +00:00
state.reset();
state.emplace();
state->query_id = query_id_;
state->query = query_;
2021-09-11 17:16:37 +00:00
if (send_progress)
state->after_send_progress.restart();
2021-08-20 08:38:50 +00:00
2021-09-11 17:16:37 +00:00
next_packet_type.reset();
2021-09-11 12:03:28 +00:00
2021-08-19 20:27:40 +00:00
try
{
2021-09-16 16:14:17 +00:00
state->io = executeQuery(state->query, query_context, false, state->stage);
2021-08-23 07:13:27 +00:00
2021-09-29 19:03:30 +00:00
if (state->io.pipeline.pushing())
2021-08-19 20:27:40 +00:00
{
2021-09-29 19:03:30 +00:00
size_t num_threads = state->io.pipeline.getNumThreads();
if (num_threads > 1)
{
state->pushing_async_executor = std::make_unique<PushingAsyncPipelineExecutor>(state->io.pipeline);
state->pushing_async_executor->start();
state->block = state->pushing_async_executor->getHeader();
}
else
{
state->pushing_executor = std::make_unique<PushingPipelineExecutor>(state->io.pipeline);
state->pushing_executor->start();
state->block = state->pushing_executor->getHeader();
}
2021-08-19 20:27:40 +00:00
}
2021-09-29 19:03:30 +00:00
else if (state->io.pipeline.pulling())
2021-08-19 20:27:40 +00:00
{
2021-08-23 07:13:27 +00:00
state->block = state->io.pipeline.getHeader();
2021-09-19 18:24:06 +00:00
state->executor = std::make_unique<PullingAsyncPipelineExecutor>(state->io.pipeline);
2021-08-19 20:27:40 +00:00
}
2021-09-29 19:03:30 +00:00
else if (state->io.pipeline.completed())
2021-08-19 20:27:40 +00:00
{
2021-09-29 19:03:30 +00:00
CompletedPipelineExecutor executor(state->io.pipeline);
executor.execute();
2021-08-19 20:27:40 +00:00
}
2021-08-23 08:50:12 +00:00
if (state->block)
next_packet_type = Protocol::Server::Data;
2021-08-19 20:27:40 +00:00
}
catch (const Exception & e)
2021-08-17 19:59:51 +00:00
{
2021-08-19 20:27:40 +00:00
state->io.onException();
state->exception.emplace(e);
2021-08-17 19:59:51 +00:00
}
2021-08-19 20:27:40 +00:00
catch (const std::exception & e)
2021-08-17 19:59:51 +00:00
{
2021-08-19 20:27:40 +00:00
state->io.onException();
state->exception.emplace(Exception::CreateFromSTDTag{}, e);
2021-08-17 19:59:51 +00:00
}
2021-08-19 20:27:40 +00:00
catch (...)
2021-08-17 19:59:51 +00:00
{
2021-08-19 20:27:40 +00:00
state->io.onException();
state->exception.emplace("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
2021-08-17 19:59:51 +00:00
}
}
2021-08-18 14:39:04 +00:00
void LocalConnection::sendData(const Block & block, const String &, bool)
{
2021-09-29 19:17:26 +00:00
if (!block)
return;
2021-09-29 19:03:30 +00:00
if (state->pushing_async_executor)
2021-08-18 14:39:04 +00:00
{
2021-09-29 19:03:30 +00:00
state->pushing_async_executor->push(std::move(block));
2021-08-21 10:55:54 +00:00
}
2021-09-29 19:03:30 +00:00
else if (state->pushing_executor)
2021-08-21 10:55:54 +00:00
{
2021-09-29 19:03:30 +00:00
state->pushing_executor->push(std::move(block));
2021-08-18 14:39:04 +00:00
}
}
2021-08-17 19:59:51 +00:00
void LocalConnection::sendCancel()
{
2021-09-29 19:03:30 +00:00
if (state->executor)
2021-08-19 20:27:40 +00:00
state->executor->cancel();
2021-08-17 19:59:51 +00:00
}
2021-08-19 20:27:40 +00:00
bool LocalConnection::pullBlock(Block & block)
2021-08-17 19:59:51 +00:00
{
2021-09-29 19:03:30 +00:00
if (state->executor)
2021-08-21 10:55:54 +00:00
return state->executor->pull(block, query_context->getSettingsRef().interactive_delay / 1000);
2021-08-19 20:27:40 +00:00
return false;
2021-08-17 19:59:51 +00:00
}
void LocalConnection::finishQuery()
{
2021-08-21 10:55:54 +00:00
next_packet_type = Protocol::Server::EndOfStream;
if (!state)
return;
2021-09-29 19:03:30 +00:00
if (state->executor)
2021-08-17 19:59:51 +00:00
{
2021-09-29 19:03:30 +00:00
state->executor.reset();
2021-08-17 19:59:51 +00:00
}
2021-09-29 19:03:30 +00:00
else if (state->pushing_async_executor)
2021-08-17 19:59:51 +00:00
{
2021-09-29 19:03:30 +00:00
state->pushing_async_executor->finish();
}
else if (state->pushing_executor)
{
state->pushing_executor->finish();
2021-08-17 19:59:51 +00:00
}
2021-08-19 20:27:40 +00:00
state->io.onFinish();
2021-08-20 08:38:50 +00:00
state.reset();
2021-08-17 19:59:51 +00:00
}
bool LocalConnection::poll(size_t)
{
2021-08-21 10:55:54 +00:00
if (!state)
return false;
2021-09-11 17:16:37 +00:00
/// Wait for next poll to collect current packet.
if (next_packet_type)
return true;
if (send_progress && (state->after_send_progress.elapsedMicroseconds() >= query_context->getSettingsRef().interactive_delay))
2021-08-17 19:59:51 +00:00
{
2021-08-21 10:55:54 +00:00
state->after_send_progress.restart();
2021-08-17 19:59:51 +00:00
next_packet_type = Protocol::Server::Progress;
2021-08-19 20:27:40 +00:00
return true;
}
2021-08-17 19:59:51 +00:00
2021-08-21 15:29:28 +00:00
if (!state->is_finished)
2021-08-19 20:27:40 +00:00
{
2021-08-21 15:29:28 +00:00
try
{
pollImpl();
}
catch (const Exception & e)
{
state->io.onException();
state->exception.emplace(e);
}
catch (const std::exception & e)
{
state->io.onException();
state->exception.emplace(Exception::CreateFromSTDTag{}, e);
}
catch (...)
{
state->io.onException();
state->exception.emplace("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION);
}
2021-08-19 20:27:40 +00:00
}
if (state->exception)
{
next_packet_type = Protocol::Server::Exception;
2021-08-17 19:59:51 +00:00
return true;
}
2021-08-19 20:27:40 +00:00
if (state->is_finished && !state->sent_totals)
{
state->sent_totals = true;
Block totals;
2021-09-29 19:03:30 +00:00
if (state->executor)
2021-08-19 20:27:40 +00:00
totals = state->executor->getTotalsBlock();
if (totals)
{
next_packet_type = Protocol::Server::Totals;
state->block.emplace(totals);
return true;
}
}
if (state->is_finished && !state->sent_extremes)
{
state->sent_extremes = true;
Block extremes;
2021-09-29 19:03:30 +00:00
if (state->executor)
2021-08-19 20:27:40 +00:00
extremes = state->executor->getExtremesBlock();
if (extremes)
{
next_packet_type = Protocol::Server::Extremes;
state->block.emplace(extremes);
return true;
}
}
2021-09-11 17:16:37 +00:00
if (state->is_finished && send_progress && !state->sent_progress)
2021-08-23 07:13:27 +00:00
{
state->sent_progress = true;
next_packet_type = Protocol::Server::Progress;
return true;
}
2021-08-19 20:27:40 +00:00
if (state->is_finished)
{
finishQuery();
return true;
}
2021-09-19 18:24:06 +00:00
if (state->block && state->block.value())
2021-08-17 19:59:51 +00:00
{
next_packet_type = Protocol::Server::Data;
2021-08-19 20:27:40 +00:00
return true;
}
return false;
}
2021-08-17 19:59:51 +00:00
2021-08-19 20:27:40 +00:00
bool LocalConnection::pollImpl()
{
Block block;
auto next_read = pullBlock(block);
if (block)
{
2021-09-04 18:19:01 +00:00
state->block.emplace(block);
2021-08-17 19:59:51 +00:00
}
2021-08-19 20:27:40 +00:00
else if (!next_read)
2021-08-17 19:59:51 +00:00
{
2021-08-19 20:27:40 +00:00
state->is_finished = true;
2021-08-17 19:59:51 +00:00
}
2021-08-19 20:27:40 +00:00
2021-08-17 19:59:51 +00:00
return true;
}
Packet LocalConnection::receivePacket()
{
Packet packet;
2021-09-11 11:34:22 +00:00
if (!state)
{
packet.type = Protocol::Server::EndOfStream;
return packet;
}
if (!next_packet_type)
poll(0);
2021-09-11 11:34:22 +00:00
if (!next_packet_type)
2021-08-21 15:29:28 +00:00
{
packet.type = Protocol::Server::EndOfStream;
return packet;
}
2021-08-21 10:55:54 +00:00
2021-08-17 19:59:51 +00:00
packet.type = next_packet_type.value();
switch (next_packet_type.value())
{
2021-08-19 20:27:40 +00:00
case Protocol::Server::Totals: [[fallthrough]];
case Protocol::Server::Extremes: [[fallthrough]];
2021-09-11 17:16:37 +00:00
case Protocol::Server::Log: [[fallthrough]];
2021-08-17 19:59:51 +00:00
case Protocol::Server::Data:
2021-08-30 11:04:59 +00:00
case Protocol::Server::ProfileEvents:
2021-08-17 19:59:51 +00:00
{
2021-09-19 18:24:06 +00:00
if (state->block && state->block.value())
2021-08-17 19:59:51 +00:00
{
2021-09-19 18:24:06 +00:00
packet.block = std::move(state->block.value());
2021-08-19 20:27:40 +00:00
state->block.reset();
2021-08-17 19:59:51 +00:00
}
2021-08-19 20:27:40 +00:00
break;
}
case Protocol::Server::Exception:
{
packet.exception = std::make_unique<Exception>(*state->exception);
2021-08-17 19:59:51 +00:00
break;
}
case Protocol::Server::Progress:
{
2021-08-19 20:27:40 +00:00
packet.progress = std::move(state->progress);
state->progress.reset();
2021-08-17 19:59:51 +00:00
break;
}
case Protocol::Server::EndOfStream:
{
break;
}
default:
2021-09-26 21:22:04 +00:00
throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER,
"Unknown packet {} for {}", toString(packet.type), getDescription());
2021-08-17 19:59:51 +00:00
}
2021-08-21 10:55:54 +00:00
next_packet_type.reset();
return packet;
2021-08-17 19:59:51 +00:00
}
2021-08-23 08:50:12 +00:00
void LocalConnection::getServerVersion(
2021-08-27 08:46:31 +00:00
const ConnectionTimeouts & /* timeouts */, String & /* name */,
UInt64 & /* version_major */, UInt64 & /* version_minor */,
2021-09-04 18:19:01 +00:00
UInt64 & /* version_patch */, UInt64 & /* revision */)
2021-08-23 08:50:12 +00:00
{
2021-09-04 18:19:01 +00:00
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented");
2021-08-23 08:50:12 +00:00
}
2021-09-04 18:19:01 +00:00
void LocalConnection::setDefaultDatabase(const String &)
2021-08-23 08:50:12 +00:00
{
2021-09-04 18:19:01 +00:00
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented");
2021-08-23 08:50:12 +00:00
}
2021-09-04 18:19:01 +00:00
UInt64 LocalConnection::getServerRevision(const ConnectionTimeouts &)
2021-08-23 08:50:12 +00:00
{
2021-09-04 18:19:01 +00:00
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented");
2021-08-23 08:50:12 +00:00
}
const String & LocalConnection::getServerTimezone(const ConnectionTimeouts &)
{
2021-09-04 18:19:01 +00:00
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented");
2021-08-23 08:50:12 +00:00
}
const String & LocalConnection::getServerDisplayName(const ConnectionTimeouts &)
{
2021-09-04 18:19:01 +00:00
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented");
}
void LocalConnection::sendExternalTablesData(ExternalTablesData &)
{
2021-09-24 08:29:01 +00:00
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented");
2021-08-23 08:50:12 +00:00
}
2021-09-11 17:16:37 +00:00
ServerConnectionPtr LocalConnection::createConnection(const ConnectionParameters &, ContextPtr current_context, bool send_progress)
2021-09-11 11:34:22 +00:00
{
2021-09-11 17:16:37 +00:00
return std::make_unique<LocalConnection>(current_context, send_progress);
2021-09-11 11:34:22 +00:00
}
2021-08-17 19:59:51 +00:00
}