mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Poco::Timespan reference fix
This commit is contained in:
parent
b083432e6e
commit
0d0a14a925
@ -159,17 +159,12 @@ void IBridge::initialize(Application & self)
|
|||||||
if (port > 0xFFFF)
|
if (port > 0xFFFF)
|
||||||
throw Exception("Out of range 'http-port': " + std::to_string(port), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
throw Exception("Out of range 'http-port': " + std::to_string(port), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||||
|
|
||||||
http_timeout = config().getUInt("http-timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT);
|
http_timeout = config().getUInt64("http-timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT);
|
||||||
max_server_connections = config().getUInt("max-server-connections", 1024);
|
max_server_connections = config().getUInt("max-server-connections", 1024);
|
||||||
keep_alive_timeout = config().getUInt("keep-alive-timeout", 10);
|
keep_alive_timeout = config().getUInt64("keep-alive-timeout", 10);
|
||||||
|
|
||||||
initializeTerminationAndSignalProcessing();
|
initializeTerminationAndSignalProcessing();
|
||||||
|
|
||||||
#if USE_ODBC
|
|
||||||
if (bridgeName() == "ODBCBridge")
|
|
||||||
Poco::Data::ODBC::Connector::registerConnector();
|
|
||||||
#endif
|
|
||||||
|
|
||||||
ServerApplication::initialize(self); // NOLINT
|
ServerApplication::initialize(self); // NOLINT
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -55,7 +55,7 @@ protected:
|
|||||||
|
|
||||||
virtual Poco::Logger * getLog() const = 0;
|
virtual Poco::Logger * getLog() const = 0;
|
||||||
|
|
||||||
virtual const Poco::Timespan & getHTTPTimeout() const = 0;
|
virtual Poco::Timespan getHTTPTimeout() const = 0;
|
||||||
|
|
||||||
virtual Poco::URI createBaseURI() const = 0;
|
virtual Poco::URI createBaseURI() const = 0;
|
||||||
|
|
||||||
|
@ -28,7 +28,7 @@ LibraryBridgeHelper::LibraryBridgeHelper(
|
|||||||
, log(&Poco::Logger::get("LibraryBridgeHelper"))
|
, log(&Poco::Logger::get("LibraryBridgeHelper"))
|
||||||
, sample_block(sample_block_)
|
, sample_block(sample_block_)
|
||||||
, config(context_->getConfigRef())
|
, config(context_->getConfigRef())
|
||||||
, http_timeout(context_->getSettingsRef().http_receive_timeout.value.totalSeconds())
|
, http_timeout(context_->getSettingsRef().http_receive_timeout.value)
|
||||||
, dictionary_id(dictionary_id_)
|
, dictionary_id(dictionary_id_)
|
||||||
{
|
{
|
||||||
bridge_port = config.getUInt("library_bridge.port", DEFAULT_PORT);
|
bridge_port = config.getUInt("library_bridge.port", DEFAULT_PORT);
|
||||||
|
@ -57,7 +57,7 @@ protected:
|
|||||||
|
|
||||||
Poco::Logger * getLog() const override { return log; }
|
Poco::Logger * getLog() const override { return log; }
|
||||||
|
|
||||||
const Poco::Timespan & getHTTPTimeout() const override { return http_timeout; }
|
Poco::Timespan getHTTPTimeout() const override { return http_timeout; }
|
||||||
|
|
||||||
Poco::URI createBaseURI() const override;
|
Poco::URI createBaseURI() const override;
|
||||||
|
|
||||||
|
@ -63,7 +63,7 @@ public:
|
|||||||
|
|
||||||
XDBCBridgeHelper(
|
XDBCBridgeHelper(
|
||||||
ContextPtr global_context_,
|
ContextPtr global_context_,
|
||||||
const Poco::Timespan & http_timeout_,
|
Poco::Timespan http_timeout_,
|
||||||
const std::string & connection_string_)
|
const std::string & connection_string_)
|
||||||
: IXDBCBridgeHelper(global_context_)
|
: IXDBCBridgeHelper(global_context_)
|
||||||
, log(&Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper"))
|
, log(&Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper"))
|
||||||
@ -90,7 +90,7 @@ protected:
|
|||||||
|
|
||||||
String configPrefix() const override { return BridgeHelperMixin::configPrefix(); }
|
String configPrefix() const override { return BridgeHelperMixin::configPrefix(); }
|
||||||
|
|
||||||
const Poco::Timespan & getHTTPTimeout() const override { return http_timeout; }
|
Poco::Timespan getHTTPTimeout() const override { return http_timeout; }
|
||||||
|
|
||||||
const Poco::Util::AbstractConfiguration & getConfig() const override { return config; }
|
const Poco::Util::AbstractConfiguration & getConfig() const override { return config; }
|
||||||
|
|
||||||
@ -118,7 +118,7 @@ private:
|
|||||||
|
|
||||||
Poco::Logger * log;
|
Poco::Logger * log;
|
||||||
std::string connection_string;
|
std::string connection_string;
|
||||||
const Poco::Timespan & http_timeout;
|
Poco::Timespan http_timeout;
|
||||||
std::string bridge_host;
|
std::string bridge_host;
|
||||||
size_t bridge_port;
|
size_t bridge_port;
|
||||||
|
|
||||||
|
@ -116,7 +116,7 @@ ConnectionEstablisherAsync::ConnectionEstablisherAsync(
|
|||||||
epoll.add(receive_timeout.getDescriptor());
|
epoll.add(receive_timeout.getDescriptor());
|
||||||
}
|
}
|
||||||
|
|
||||||
void ConnectionEstablisherAsync::Routine::ReadCallback::operator()(int fd, const Poco::Timespan & timeout, const std::string &)
|
void ConnectionEstablisherAsync::Routine::ReadCallback::operator()(int fd, Poco::Timespan timeout, const std::string &)
|
||||||
{
|
{
|
||||||
/// Check if it's the first time and we need to add socket fd to epoll.
|
/// Check if it's the first time and we need to add socket fd to epoll.
|
||||||
if (connection_establisher_async.socket_fd == -1)
|
if (connection_establisher_async.socket_fd == -1)
|
||||||
|
@ -92,7 +92,7 @@ private:
|
|||||||
ConnectionEstablisherAsync & connection_establisher_async;
|
ConnectionEstablisherAsync & connection_establisher_async;
|
||||||
Fiber & fiber;
|
Fiber & fiber;
|
||||||
|
|
||||||
void operator()(int fd, const Poco::Timespan & timeout, const std::string &);
|
void operator()(int fd, Poco::Timespan timeout, const std::string &);
|
||||||
};
|
};
|
||||||
|
|
||||||
Fiber operator()(Fiber && sink);
|
Fiber operator()(Fiber && sink);
|
||||||
|
@ -98,7 +98,7 @@ private:
|
|||||||
PacketReceiver & receiver;
|
PacketReceiver & receiver;
|
||||||
Fiber & sink;
|
Fiber & sink;
|
||||||
|
|
||||||
void operator()(int, const Poco::Timespan & timeout, const std::string &)
|
void operator()(int, Poco::Timespan timeout, const std::string &)
|
||||||
{
|
{
|
||||||
receiver.receive_timeout.setRelative(timeout);
|
receiver.receive_timeout.setRelative(timeout);
|
||||||
receiver.is_read_in_process = true;
|
receiver.is_read_in_process = true;
|
||||||
|
@ -9,7 +9,7 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
using AsyncCallback = std::function<void(int, const Poco::Timespan &, const std::string &)>;
|
using AsyncCallback = std::function<void(int, Poco::Timespan, const std::string &)>;
|
||||||
|
|
||||||
class Epoll
|
class Epoll
|
||||||
{
|
{
|
||||||
|
@ -74,7 +74,7 @@ void TimerDescriptor::drain() const
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void TimerDescriptor::setRelative(const Poco::Timespan & timespan) const
|
void TimerDescriptor::setRelative(Poco::Timespan timespan) const
|
||||||
{
|
{
|
||||||
itimerspec spec;
|
itimerspec spec;
|
||||||
spec.it_interval.tv_nsec = 0;
|
spec.it_interval.tv_nsec = 0;
|
||||||
|
@ -24,7 +24,7 @@ public:
|
|||||||
|
|
||||||
void reset() const;
|
void reset() const;
|
||||||
void drain() const;
|
void drain() const;
|
||||||
void setRelative(const Poco::Timespan & timespan) const;
|
void setRelative(Poco::Timespan timespan) const;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -101,7 +101,7 @@ struct SettingFieldTimespan
|
|||||||
Poco::Timespan value;
|
Poco::Timespan value;
|
||||||
bool changed = false;
|
bool changed = false;
|
||||||
|
|
||||||
explicit SettingFieldTimespan(const Poco::Timespan & x = {}) : value(x) {}
|
explicit SettingFieldTimespan(Poco::Timespan x = {}) : value(x) {}
|
||||||
|
|
||||||
template <class Rep, class Period = std::ratio<1>>
|
template <class Rep, class Period = std::ratio<1>>
|
||||||
explicit SettingFieldTimespan(const std::chrono::duration<Rep, Period> & x)
|
explicit SettingFieldTimespan(const std::chrono::duration<Rep, Period> & x)
|
||||||
@ -110,7 +110,7 @@ struct SettingFieldTimespan
|
|||||||
explicit SettingFieldTimespan(UInt64 x) : SettingFieldTimespan(Poco::Timespan{static_cast<Poco::Timespan::TimeDiff>(x * microseconds_per_unit)}) {}
|
explicit SettingFieldTimespan(UInt64 x) : SettingFieldTimespan(Poco::Timespan{static_cast<Poco::Timespan::TimeDiff>(x * microseconds_per_unit)}) {}
|
||||||
explicit SettingFieldTimespan(const Field & f);
|
explicit SettingFieldTimespan(const Field & f);
|
||||||
|
|
||||||
SettingFieldTimespan & operator =(const Poco::Timespan & x) { value = x; changed = true; return *this; }
|
SettingFieldTimespan & operator =(Poco::Timespan x) { value = x; changed = true; return *this; }
|
||||||
|
|
||||||
template <class Rep, class Period = std::ratio<1>>
|
template <class Rep, class Period = std::ratio<1>>
|
||||||
SettingFieldTimespan & operator =(const std::chrono::duration<Rep, Period> & x) { *this = Poco::Timespan{static_cast<Poco::Timespan::TimeDiff>(std::chrono::duration_cast<std::chrono::microseconds>(x).count())}; return *this; }
|
SettingFieldTimespan & operator =(const std::chrono::duration<Rep, Period> & x) { *this = Poco::Timespan{static_cast<Poco::Timespan::TimeDiff>(std::chrono::duration_cast<std::chrono::microseconds>(x).count())}; return *this; }
|
||||||
|
@ -19,7 +19,7 @@ struct RemoteQueryExecutorRoutine
|
|||||||
RemoteQueryExecutorReadContext & read_context;
|
RemoteQueryExecutorReadContext & read_context;
|
||||||
Fiber & fiber;
|
Fiber & fiber;
|
||||||
|
|
||||||
void operator()(int fd, const Poco::Timespan & timeout = 0, const std::string fd_description = "")
|
void operator()(int fd, Poco::Timespan timeout = 0, const std::string fd_description = "")
|
||||||
{
|
{
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
@ -89,7 +89,7 @@ RemoteQueryExecutorReadContext::RemoteQueryExecutorReadContext(IConnections & co
|
|||||||
fiber = boost::context::fiber(std::allocator_arg_t(), stack, std::move(routine));
|
fiber = boost::context::fiber(std::allocator_arg_t(), stack, std::move(routine));
|
||||||
}
|
}
|
||||||
|
|
||||||
void RemoteQueryExecutorReadContext::setConnectionFD(int fd, const Poco::Timespan & timeout, const std::string & fd_description)
|
void RemoteQueryExecutorReadContext::setConnectionFD(int fd, Poco::Timespan timeout, const std::string & fd_description)
|
||||||
{
|
{
|
||||||
if (fd == connection_fd)
|
if (fd == connection_fd)
|
||||||
return;
|
return;
|
||||||
|
@ -58,7 +58,7 @@ public:
|
|||||||
bool checkTimeout(bool blocking = false);
|
bool checkTimeout(bool blocking = false);
|
||||||
bool checkTimeoutImpl(bool blocking);
|
bool checkTimeoutImpl(bool blocking);
|
||||||
|
|
||||||
void setConnectionFD(int fd, const Poco::Timespan & timeout = 0, const std::string & fd_description = "");
|
void setConnectionFD(int fd, Poco::Timespan timeout = 0, const std::string & fd_description = "");
|
||||||
void setTimer() const;
|
void setTimer() const;
|
||||||
|
|
||||||
bool resumeRoutine();
|
bool resumeRoutine();
|
||||||
|
@ -24,9 +24,9 @@ struct ConnectionTimeouts
|
|||||||
|
|
||||||
ConnectionTimeouts() = default;
|
ConnectionTimeouts() = default;
|
||||||
|
|
||||||
ConnectionTimeouts(const Poco::Timespan & connection_timeout_,
|
ConnectionTimeouts(Poco::Timespan connection_timeout_,
|
||||||
const Poco::Timespan & send_timeout_,
|
Poco::Timespan send_timeout_,
|
||||||
const Poco::Timespan & receive_timeout_)
|
Poco::Timespan receive_timeout_)
|
||||||
: connection_timeout(connection_timeout_),
|
: connection_timeout(connection_timeout_),
|
||||||
send_timeout(send_timeout_),
|
send_timeout(send_timeout_),
|
||||||
receive_timeout(receive_timeout_),
|
receive_timeout(receive_timeout_),
|
||||||
@ -38,10 +38,10 @@ struct ConnectionTimeouts
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
ConnectionTimeouts(const Poco::Timespan & connection_timeout_,
|
ConnectionTimeouts(Poco::Timespan connection_timeout_,
|
||||||
const Poco::Timespan & send_timeout_,
|
Poco::Timespan send_timeout_,
|
||||||
const Poco::Timespan & receive_timeout_,
|
Poco::Timespan receive_timeout_,
|
||||||
const Poco::Timespan & tcp_keep_alive_timeout_)
|
Poco::Timespan tcp_keep_alive_timeout_)
|
||||||
: connection_timeout(connection_timeout_),
|
: connection_timeout(connection_timeout_),
|
||||||
send_timeout(send_timeout_),
|
send_timeout(send_timeout_),
|
||||||
receive_timeout(receive_timeout_),
|
receive_timeout(receive_timeout_),
|
||||||
@ -52,11 +52,11 @@ struct ConnectionTimeouts
|
|||||||
receive_data_timeout(receive_timeout_)
|
receive_data_timeout(receive_timeout_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
ConnectionTimeouts(const Poco::Timespan & connection_timeout_,
|
ConnectionTimeouts(Poco::Timespan connection_timeout_,
|
||||||
const Poco::Timespan & send_timeout_,
|
Poco::Timespan send_timeout_,
|
||||||
const Poco::Timespan & receive_timeout_,
|
Poco::Timespan receive_timeout_,
|
||||||
const Poco::Timespan & tcp_keep_alive_timeout_,
|
Poco::Timespan tcp_keep_alive_timeout_,
|
||||||
const Poco::Timespan & http_keep_alive_timeout_)
|
Poco::Timespan http_keep_alive_timeout_)
|
||||||
: connection_timeout(connection_timeout_),
|
: connection_timeout(connection_timeout_),
|
||||||
send_timeout(send_timeout_),
|
send_timeout(send_timeout_),
|
||||||
receive_timeout(receive_timeout_),
|
receive_timeout(receive_timeout_),
|
||||||
@ -68,14 +68,14 @@ struct ConnectionTimeouts
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
ConnectionTimeouts(const Poco::Timespan & connection_timeout_,
|
ConnectionTimeouts(Poco::Timespan connection_timeout_,
|
||||||
const Poco::Timespan & send_timeout_,
|
Poco::Timespan send_timeout_,
|
||||||
const Poco::Timespan & receive_timeout_,
|
Poco::Timespan receive_timeout_,
|
||||||
const Poco::Timespan & tcp_keep_alive_timeout_,
|
Poco::Timespan tcp_keep_alive_timeout_,
|
||||||
const Poco::Timespan & http_keep_alive_timeout_,
|
Poco::Timespan http_keep_alive_timeout_,
|
||||||
const Poco::Timespan & secure_connection_timeout_,
|
Poco::Timespan secure_connection_timeout_,
|
||||||
const Poco::Timespan & receive_hello_timeout_,
|
Poco::Timespan receive_hello_timeout_,
|
||||||
const Poco::Timespan & receive_data_timeout_)
|
Poco::Timespan receive_data_timeout_)
|
||||||
: connection_timeout(connection_timeout_),
|
: connection_timeout(connection_timeout_),
|
||||||
send_timeout(send_timeout_),
|
send_timeout(send_timeout_),
|
||||||
receive_timeout(receive_timeout_),
|
receive_timeout(receive_timeout_),
|
||||||
@ -87,7 +87,7 @@ struct ConnectionTimeouts
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
static Poco::Timespan saturate(const Poco::Timespan & timespan, const Poco::Timespan & limit)
|
static Poco::Timespan saturate(Poco::Timespan timespan, Poco::Timespan limit)
|
||||||
{
|
{
|
||||||
if (limit.totalMicroseconds() == 0)
|
if (limit.totalMicroseconds() == 0)
|
||||||
return timespan;
|
return timespan;
|
||||||
@ -95,7 +95,7 @@ struct ConnectionTimeouts
|
|||||||
return (timespan > limit) ? limit : timespan;
|
return (timespan > limit) ? limit : timespan;
|
||||||
}
|
}
|
||||||
|
|
||||||
ConnectionTimeouts getSaturated(const Poco::Timespan & limit) const
|
ConnectionTimeouts getSaturated(Poco::Timespan limit) const
|
||||||
{
|
{
|
||||||
return ConnectionTimeouts(saturate(connection_timeout, limit),
|
return ConnectionTimeouts(saturate(connection_timeout, limit),
|
||||||
saturate(send_timeout, limit),
|
saturate(send_timeout, limit),
|
||||||
|
@ -8,7 +8,7 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
using AsyncCallback = std::function<void(int, const Poco::Timespan &, const std::string &)>;
|
using AsyncCallback = std::function<void(int, Poco::Timespan, const std::string &)>;
|
||||||
|
|
||||||
/// Works with the ready Poco::Net::Socket. Blocking operations.
|
/// Works with the ready Poco::Net::Socket. Blocking operations.
|
||||||
class ReadBufferFromPocoSocket : public BufferWithOwnMemory<ReadBuffer>
|
class ReadBufferFromPocoSocket : public BufferWithOwnMemory<ReadBuffer>
|
||||||
|
@ -7,8 +7,8 @@ namespace DB
|
|||||||
{
|
{
|
||||||
|
|
||||||
TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_,
|
TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_,
|
||||||
const Poco::Timespan & send_timeout_,
|
Poco::Timespan send_timeout_,
|
||||||
const Poco::Timespan & receive_timeout_,
|
Poco::Timespan receive_timeout_,
|
||||||
bool limit_max_timeout)
|
bool limit_max_timeout)
|
||||||
: socket(socket_), send_timeout(send_timeout_), receive_timeout(receive_timeout_)
|
: socket(socket_), send_timeout(send_timeout_), receive_timeout(receive_timeout_)
|
||||||
{
|
{
|
||||||
@ -22,7 +22,7 @@ TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_,
|
|||||||
socket.setReceiveTimeout(receive_timeout);
|
socket.setReceiveTimeout(receive_timeout);
|
||||||
}
|
}
|
||||||
|
|
||||||
TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_, bool limit_max_timeout)
|
TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_, Poco::Timespan timeout_, bool limit_max_timeout)
|
||||||
: TimeoutSetter(socket_, timeout_, timeout_, limit_max_timeout)
|
: TimeoutSetter(socket_, timeout_, timeout_, limit_max_timeout)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
@ -11,11 +11,11 @@ namespace DB
|
|||||||
struct TimeoutSetter
|
struct TimeoutSetter
|
||||||
{
|
{
|
||||||
TimeoutSetter(Poco::Net::StreamSocket & socket_,
|
TimeoutSetter(Poco::Net::StreamSocket & socket_,
|
||||||
const Poco::Timespan & send_timeout_,
|
Poco::Timespan send_timeout_,
|
||||||
const Poco::Timespan & receive_timeout_,
|
Poco::Timespan receive_timeout_,
|
||||||
bool limit_max_timeout = false);
|
bool limit_max_timeout = false);
|
||||||
|
|
||||||
TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_, bool limit_max_timeout = false);
|
TimeoutSetter(Poco::Net::StreamSocket & socket_, Poco::Timespan timeout_, bool limit_max_timeout = false);
|
||||||
|
|
||||||
~TimeoutSetter();
|
~TimeoutSetter();
|
||||||
|
|
||||||
|
@ -541,7 +541,7 @@ Cluster::Cluster(const Settings & settings, const std::vector<std::vector<String
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
Poco::Timespan Cluster::saturate(const Poco::Timespan & v, const Poco::Timespan & limit)
|
Poco::Timespan Cluster::saturate(Poco::Timespan v, Poco::Timespan limit)
|
||||||
{
|
{
|
||||||
if (limit.totalMicroseconds() == 0)
|
if (limit.totalMicroseconds() == 0)
|
||||||
return v;
|
return v;
|
||||||
|
@ -52,7 +52,7 @@ public:
|
|||||||
Cluster & operator=(const Cluster &) = delete;
|
Cluster & operator=(const Cluster &) = delete;
|
||||||
|
|
||||||
/// is used to set a limit on the size of the timeout
|
/// is used to set a limit on the size of the timeout
|
||||||
static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit);
|
static Poco::Timespan saturate(Poco::Timespan v, Poco::Timespan limit);
|
||||||
|
|
||||||
public:
|
public:
|
||||||
using SlotToShard = std::vector<UInt64>;
|
using SlotToShard = std::vector<UInt64>;
|
||||||
|
@ -475,7 +475,7 @@ void TCPHandler::runImpl()
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
bool TCPHandler::readDataNext(const size_t & poll_interval, const int & receive_timeout)
|
bool TCPHandler::readDataNext(size_t poll_interval, time_t receive_timeout)
|
||||||
{
|
{
|
||||||
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
|
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
|
||||||
|
|
||||||
@ -493,8 +493,8 @@ bool TCPHandler::readDataNext(const size_t & poll_interval, const int & receive_
|
|||||||
* If we periodically poll, the receive_timeout of the socket itself does not work.
|
* If we periodically poll, the receive_timeout of the socket itself does not work.
|
||||||
* Therefore, an additional check is added.
|
* Therefore, an additional check is added.
|
||||||
*/
|
*/
|
||||||
double elapsed = watch.elapsedSeconds();
|
Float64 elapsed = watch.elapsedSeconds();
|
||||||
if (elapsed > receive_timeout)
|
if (elapsed > static_cast<Float64>(receive_timeout))
|
||||||
{
|
{
|
||||||
throw Exception(ErrorCodes::SOCKET_TIMEOUT,
|
throw Exception(ErrorCodes::SOCKET_TIMEOUT,
|
||||||
"Timeout exceeded while receiving data from client. Waited for {} seconds, timeout is {} seconds.",
|
"Timeout exceeded while receiving data from client. Waited for {} seconds, timeout is {} seconds.",
|
||||||
@ -535,10 +535,7 @@ std::tuple<size_t, int> TCPHandler::getReadTimeouts(const Settings & connection_
|
|||||||
|
|
||||||
void TCPHandler::readData(const Settings & connection_settings)
|
void TCPHandler::readData(const Settings & connection_settings)
|
||||||
{
|
{
|
||||||
size_t poll_interval;
|
auto [poll_interval, receive_timeout] = getReadTimeouts(connection_settings);
|
||||||
int receive_timeout;
|
|
||||||
|
|
||||||
std::tie(poll_interval, receive_timeout) = getReadTimeouts(connection_settings);
|
|
||||||
sendLogs();
|
sendLogs();
|
||||||
|
|
||||||
while (readDataNext(poll_interval, receive_timeout))
|
while (readDataNext(poll_interval, receive_timeout))
|
||||||
|
@ -174,7 +174,7 @@ private:
|
|||||||
void receiveIgnoredPartUUIDs();
|
void receiveIgnoredPartUUIDs();
|
||||||
String receiveReadTaskResponseAssumeLocked();
|
String receiveReadTaskResponseAssumeLocked();
|
||||||
bool receiveData(bool scalar);
|
bool receiveData(bool scalar);
|
||||||
bool readDataNext(const size_t & poll_interval, const int & receive_timeout);
|
bool readDataNext(size_t poll_interval, time_t receive_timeout);
|
||||||
void readData(const Settings & connection_settings);
|
void readData(const Settings & connection_settings);
|
||||||
void receiveClusterNameAndSalt();
|
void receiveClusterNameAndSalt();
|
||||||
std::tuple<size_t, int> getReadTimeouts(const Settings & connection_settings);
|
std::tuple<size_t, int> getReadTimeouts(const Settings & connection_settings);
|
||||||
|
@ -22,7 +22,7 @@ private:
|
|||||||
|
|
||||||
/* A factory method to create bridge helper, that will assist in remote interaction */
|
/* A factory method to create bridge helper, that will assist in remote interaction */
|
||||||
virtual BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
virtual BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
||||||
const Poco::Timespan & http_timeout_,
|
Poco::Timespan http_timeout_,
|
||||||
const std::string & connection_string_) const = 0;
|
const std::string & connection_string_) const = 0;
|
||||||
|
|
||||||
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
|
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
|
||||||
@ -48,7 +48,7 @@ public:
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
||||||
const Poco::Timespan & http_timeout_,
|
Poco::Timespan http_timeout_,
|
||||||
const std::string & connection_string_) const override
|
const std::string & connection_string_) const override
|
||||||
{
|
{
|
||||||
return std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(context, http_timeout_, connection_string_);
|
return std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(context, http_timeout_, connection_string_);
|
||||||
@ -68,7 +68,7 @@ public:
|
|||||||
|
|
||||||
private:
|
private:
|
||||||
BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
BridgeHelperPtr createBridgeHelper(ContextPtr context,
|
||||||
const Poco::Timespan & http_timeout_,
|
Poco::Timespan http_timeout_,
|
||||||
const std::string & connection_string_) const override
|
const std::string & connection_string_) const override
|
||||||
{
|
{
|
||||||
return std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(context, http_timeout_, connection_string_);
|
return std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(context, http_timeout_, connection_string_);
|
||||||
|
Loading…
Reference in New Issue
Block a user