Merge pull request #36781 from Avogar/fix-timeout-message

Better exception messages while socket timeouts
This commit is contained in:
Kruglov Pavel 2022-05-02 13:12:35 +02:00 committed by GitHub
commit 188aa3b694
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 21 additions and 10 deletions

View File

@ -161,16 +161,17 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
}
catch (Poco::TimeoutException & e)
{
/// disconnect() will reset the socket, get timeouts before.
const std::string & message = fmt::format("{} ({}, receive timeout {} ms, send timeout {} ms)",
e.displayText(), getDescription(),
socket->getReceiveTimeout().totalMilliseconds(),
socket->getSendTimeout().totalMilliseconds());
disconnect();
/// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost.
throw NetException(message, ErrorCodes::SOCKET_TIMEOUT);
/// This exception can only be thrown from socket->connect(), so add information about connection timeout.
const auto & connection_timeout = static_cast<bool>(secure) ? timeouts.secure_connection_timeout : timeouts.connection_timeout;
throw NetException(
ErrorCodes::SOCKET_TIMEOUT,
"{} ({}, connection timeout {} ms)",
e.displayText(),
getDescription(),
connection_timeout.totalMilliseconds());
}
}

View File

@ -197,7 +197,10 @@ bool ConnectionEstablisherAsync::checkReceiveTimeout()
destroyFiber();
/// In not async case this exception would be thrown and caught in ConnectionEstablisher::run,
/// but in async case we process timeout outside and cannot throw exception. So, we just save fail message.
fail_message = "Timeout exceeded while reading from socket (" + result.entry->getDescription() + ")";
fail_message = fmt::format(
"Timeout exceeded while reading from socket ({}, receive timeout {} ms)",
result.entry->getDescription(),
result.entry->getSocket()->getReceiveTimeout().totalMilliseconds());
epoll.remove(socket_fd);
resetResult();
return false;

View File

@ -354,11 +354,16 @@ bool HedgedConnections::resumePacketReceiver(const HedgedConnections::ReplicaLoc
}
else if (std::holds_alternative<Poco::Timespan>(res))
{
const String & description = replica_state.connection->getDescription();
finishProcessReplica(replica_state, true);
/// Check if there is no more active connections with the same offset and there is no new replica in process.
if (offset_states[location.offset].active_connection_count == 0 && !offset_states[location.offset].next_replica_in_process)
throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT);
throw NetException(
ErrorCodes::SOCKET_TIMEOUT,
"Timeout exceeded while reading from socket ({}, receive timeout {} ms)",
description,
std::get<Poco::Timespan>(res).totalMilliseconds());
}
else if (std::holds_alternative<std::exception_ptr>(res))
{

View File

@ -37,7 +37,7 @@ public:
if (!connection->hasReadPendingData() && !checkReceiveTimeout())
{
/// Receive timeout expired.
return Poco::Timespan();
return connection->getSocket()->getReceiveTimeout();
}
/// Resume fiber.

View File

@ -114,6 +114,8 @@ bool RemoteQueryExecutorReadContext::checkTimeout(bool blocking)
{
if (last_used_socket)
e.addMessage(" while reading from socket ({})", last_used_socket->peerAddress().toString());
if (e.code() == ErrorCodes::SOCKET_TIMEOUT)
e.addMessage(" (receive timeout {} ms)", receive_timeout_usec / 1000);
throw;
}
}