ClickHouse/src/Client/ConnectionEstablisher.h

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

129 lines
4.4 KiB
C++
Raw Normal View History

2021-02-17 17:34:52 +00:00
#pragma once
2021-02-21 14:03:24 +00:00
#include <variant>
2023-03-03 19:30:43 +00:00
#include <Common/AsyncTaskExecutor.h>
2021-02-17 17:34:52 +00:00
#include <Common/Epoll.h>
#include <Common/Fiber.h>
#include <Common/FiberStack.h>
#include <Common/TimerDescriptor.h>
#include <Common/PoolWithFailoverBase.h>
#include <Client/ConnectionPool.h>
namespace DB
{
2021-02-21 14:03:24 +00:00
/// Class for establishing connection to the replica. It supports setting up
/// an async callback that will be called when reading from socket blocks.
2021-02-17 17:34:52 +00:00
class ConnectionEstablisher
{
public:
using TryResult = PoolWithFailoverBase<IConnectionPool>::TryResult;
ConnectionEstablisher(IConnectionPool * pool_,
const ConnectionTimeouts * timeouts_,
const Settings * settings_,
2021-02-21 14:03:24 +00:00
Poco::Logger * log,
2021-02-17 17:34:52 +00:00
const QualifiedTableName * table_to_check = nullptr);
2021-02-21 14:03:24 +00:00
/// Establish connection and save it in result, write possible exception message in fail_message.
void run(TryResult & result, std::string & fail_message);
2021-02-17 17:34:52 +00:00
2021-02-21 14:03:24 +00:00
/// Set async callback that will be called when reading from socket blocks.
void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); }
2021-02-17 17:34:52 +00:00
2021-02-21 14:03:24 +00:00
bool isFinished() const { return is_finished; }
2021-02-17 17:34:52 +00:00
2021-02-21 14:03:24 +00:00
private:
IConnectionPool * pool;
const ConnectionTimeouts * timeouts;
const Settings * settings;
Poco::Logger * log;
const QualifiedTableName * table_to_check;
2021-02-17 17:34:52 +00:00
2021-02-21 14:03:24 +00:00
bool is_finished;
AsyncCallback async_callback = {};
2021-02-17 17:34:52 +00:00
2021-02-21 14:03:24 +00:00
};
2021-02-17 17:34:52 +00:00
#if defined(OS_LINUX)
2021-02-21 14:03:24 +00:00
/// Class for nonblocking establishing connection to the replica.
/// It runs establishing connection process in fiber and sets special
/// read callback which is called when reading from socket blocks.
/// When read callback is called, socket and receive timeout are added in epoll
/// and execution returns to the main program.
/// So, you can poll this epoll file descriptor to determine when to resume.
2023-03-03 19:30:43 +00:00
class ConnectionEstablisherAsync : public AsyncTaskExecutor
2021-02-21 14:03:24 +00:00
{
public:
using TryResult = PoolWithFailoverBase<IConnectionPool>::TryResult;
ConnectionEstablisherAsync(IConnectionPool * pool_,
2023-03-03 19:30:43 +00:00
const ConnectionTimeouts * timeouts_,
const Settings * settings_,
Poco::Logger * log_,
const QualifiedTableName * table_to_check_ = nullptr);
/// Get file descriptor that can be added in epoll and be polled,
/// when this fd becomes ready, you call resume establishing connection.
int getFileDescriptor() { return epoll.getFileDescriptor(); }
/// Check if the process of connection establishing was finished.
/// The process is considered finished if connection is ready,
/// some exception occurred or timeout exceeded.
bool isFinished() { return is_finished; }
2021-02-21 14:03:24 +00:00
TryResult getResult() const { return result; }
2021-02-17 17:34:52 +00:00
2021-02-21 14:03:24 +00:00
const std::string & getFailMessage() const { return fail_message; }
2021-02-17 17:34:52 +00:00
private:
2023-03-03 19:30:43 +00:00
bool checkBeforeTaskResume() override;
2021-02-17 17:34:52 +00:00
2023-03-03 19:30:43 +00:00
void afterTaskResume() override;
2021-02-17 17:34:52 +00:00
2023-03-03 19:30:43 +00:00
void processAsyncEvent(int fd, Poco::Timespan socket_timeout, AsyncEventTimeoutType type, const std::string & description, uint32_t events) override;
void clearAsyncEvent() override;
2021-02-17 17:34:52 +00:00
2023-03-03 19:30:43 +00:00
struct Task : public AsyncTask
{
Task(ConnectionEstablisherAsync & connection_establisher_async_) : connection_establisher_async(connection_establisher_async_) {}
ConnectionEstablisherAsync & connection_establisher_async;
2021-02-17 17:34:52 +00:00
2023-05-22 18:22:05 +00:00
void run(AsyncCallback async_callback, SuspendCallback suspend_callback) override;
2021-02-17 17:34:52 +00:00
};
2023-03-28 16:00:00 +00:00
void cancelAfter() override;
2023-03-03 19:30:43 +00:00
/// When epoll file descriptor is ready, check if it's an expired timeout.
/// Return false if receive timeout expired and socket is not ready, return true otherwise.
bool checkTimeout();
2021-02-17 17:34:52 +00:00
void reset();
2021-02-21 14:03:24 +00:00
void resetResult();
2021-02-17 17:34:52 +00:00
2021-02-21 14:03:24 +00:00
ConnectionEstablisher connection_establisher;
2021-02-17 17:34:52 +00:00
TryResult result;
2021-02-21 14:03:24 +00:00
std::string fail_message;
/// We use timer descriptor for checking socket receive timeout.
2023-03-03 19:30:43 +00:00
TimerDescriptor timeout_descriptor;
Poco::Timespan timeout;
AsyncEventTimeoutType timeout_type;
2021-02-21 14:03:24 +00:00
/// In read callback we add socket file descriptor and timer descriptor with receive timeout
/// in epoll, so we can return epoll file descriptor outside for polling.
2021-02-17 17:34:52 +00:00
Epoll epoll;
2021-02-21 14:03:24 +00:00
int socket_fd = -1;
std::string socket_description;
2023-03-03 19:30:43 +00:00
bool is_finished = false;
bool restarted = false;
2021-02-17 17:34:52 +00:00
};
2021-02-21 14:03:24 +00:00
#endif
2021-02-17 17:34:52 +00:00
}