|
@@ -59,17 +59,20 @@ public:
|
|
|
|
|
|
private:
|
|
private:
|
|
const Options options_;
|
|
const Options options_;
|
|
|
|
+ ::asio::ip::tcp::endpoint current_endpoint_;
|
|
std::vector<::asio::ip::tcp::endpoint> additional_endpoints_;
|
|
std::vector<::asio::ip::tcp::endpoint> additional_endpoints_;
|
|
NextLayer next_layer_;
|
|
NextLayer next_layer_;
|
|
|
|
+ ::asio::deadline_timer connect_timer_;
|
|
|
|
|
|
- void ConnectComplete(const ::asio::error_code &ec);
|
|
|
|
|
|
+ void ConnectComplete(const ::asio::error_code &ec, const ::asio::ip::tcp::endpoint &remote);
|
|
};
|
|
};
|
|
|
|
|
|
template <class NextLayer>
|
|
template <class NextLayer>
|
|
RpcConnectionImpl<NextLayer>::RpcConnectionImpl(RpcEngine *engine)
|
|
RpcConnectionImpl<NextLayer>::RpcConnectionImpl(RpcEngine *engine)
|
|
: RpcConnection(engine),
|
|
: RpcConnection(engine),
|
|
options_(engine->options()),
|
|
options_(engine->options()),
|
|
- next_layer_(engine->io_service()) {
|
|
|
|
|
|
+ next_layer_(engine->io_service()),
|
|
|
|
+ connect_timer_(engine->io_service()) {
|
|
LOG_TRACE(kRPC, << "RpcConnectionImpl::RpcConnectionImpl called");
|
|
LOG_TRACE(kRPC, << "RpcConnectionImpl::RpcConnectionImpl called");
|
|
}
|
|
}
|
|
|
|
|
|
@@ -129,20 +132,43 @@ void RpcConnectionImpl<NextLayer>::ConnectAndFlush(
|
|
additional_endpoints_ = server;
|
|
additional_endpoints_ = server;
|
|
::asio::ip::tcp::endpoint first_endpoint = additional_endpoints_.front();
|
|
::asio::ip::tcp::endpoint first_endpoint = additional_endpoints_.front();
|
|
additional_endpoints_.erase(additional_endpoints_.begin());
|
|
additional_endpoints_.erase(additional_endpoints_.begin());
|
|
|
|
+ current_endpoint_ = first_endpoint;
|
|
|
|
|
|
auto shared_this = shared_from_this();
|
|
auto shared_this = shared_from_this();
|
|
- next_layer_.async_connect(first_endpoint, [shared_this, this](const ::asio::error_code &ec) {
|
|
|
|
- ConnectComplete(ec);
|
|
|
|
|
|
+ next_layer_.async_connect(first_endpoint, [shared_this, this, first_endpoint](const ::asio::error_code &ec) {
|
|
|
|
+ ConnectComplete(ec, first_endpoint);
|
|
|
|
+ });
|
|
|
|
+
|
|
|
|
+ // Prompt the timer to timeout
|
|
|
|
+ auto weak_this = std::weak_ptr<RpcConnection>(shared_this);
|
|
|
|
+ connect_timer_.expires_from_now(
|
|
|
|
+ std::chrono::milliseconds(options_.rpc_connect_timeout));
|
|
|
|
+ connect_timer_.async_wait([shared_this, this, first_endpoint](const ::asio::error_code &ec) {
|
|
|
|
+ if (ec)
|
|
|
|
+ ConnectComplete(ec, first_endpoint);
|
|
|
|
+ else
|
|
|
|
+ ConnectComplete(make_error_code(asio::error::host_unreachable), first_endpoint);
|
|
});
|
|
});
|
|
}
|
|
}
|
|
|
|
|
|
template <class NextLayer>
|
|
template <class NextLayer>
|
|
-void RpcConnectionImpl<NextLayer>::ConnectComplete(const ::asio::error_code &ec) {
|
|
|
|
|
|
+void RpcConnectionImpl<NextLayer>::ConnectComplete(const ::asio::error_code &ec, const ::asio::ip::tcp::endpoint & remote) {
|
|
auto shared_this = RpcConnectionImpl<NextLayer>::shared_from_this();
|
|
auto shared_this = RpcConnectionImpl<NextLayer>::shared_from_this();
|
|
std::lock_guard<std::mutex> state_lock(connection_state_lock_);
|
|
std::lock_guard<std::mutex> state_lock(connection_state_lock_);
|
|
|
|
+ connect_timer_.cancel();
|
|
|
|
|
|
LOG_TRACE(kRPC, << "RpcConnectionImpl::ConnectComplete called");
|
|
LOG_TRACE(kRPC, << "RpcConnectionImpl::ConnectComplete called");
|
|
|
|
|
|
|
|
+ // Could be an old async connect returning a result after we've moved on
|
|
|
|
+ if (remote != current_endpoint_) {
|
|
|
|
+ LOG_DEBUG(kRPC, << "Got ConnectComplete for " << remote << " but current_endpoint_ is " << current_endpoint_);
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ if (connected_ != kConnecting) {
|
|
|
|
+ LOG_DEBUG(kRPC, << "Got ConnectComplete but current state is " << connected_);;
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
Status status = ToStatus(ec);
|
|
Status status = ToStatus(ec);
|
|
if(event_handlers_) {
|
|
if(event_handlers_) {
|
|
auto event_resp = event_handlers_->call(FS_NN_CONNECT_EVENT, cluster_name_.c_str(), 0);
|
|
auto event_resp = event_handlers_->call(FS_NN_CONNECT_EVENT, cluster_name_.c_str(), 0);
|
|
@@ -159,6 +185,7 @@ void RpcConnectionImpl<NextLayer>::ConnectComplete(const ::asio::error_code &ec)
|
|
HandshakeComplete(s);
|
|
HandshakeComplete(s);
|
|
});
|
|
});
|
|
} else {
|
|
} else {
|
|
|
|
+ LOG_DEBUG(kRPC, << "Rpc connection failed; err=" << status.ToString());;
|
|
std::string err = SafeDisconnect(get_asio_socket_ptr(&next_layer_));
|
|
std::string err = SafeDisconnect(get_asio_socket_ptr(&next_layer_));
|
|
if(!err.empty()) {
|
|
if(!err.empty()) {
|
|
LOG_INFO(kRPC, << "Rpc connection failed to connect to endpoint, error closing connection: " << err);
|
|
LOG_INFO(kRPC, << "Rpc connection failed to connect to endpoint, error closing connection: " << err);
|
|
@@ -169,10 +196,19 @@ void RpcConnectionImpl<NextLayer>::ConnectComplete(const ::asio::error_code &ec)
|
|
// hit one
|
|
// hit one
|
|
::asio::ip::tcp::endpoint next_endpoint = additional_endpoints_.front();
|
|
::asio::ip::tcp::endpoint next_endpoint = additional_endpoints_.front();
|
|
additional_endpoints_.erase(additional_endpoints_.begin());
|
|
additional_endpoints_.erase(additional_endpoints_.begin());
|
|
|
|
+ current_endpoint_ = next_endpoint;
|
|
|
|
|
|
- next_layer_.async_connect(next_endpoint, [shared_this, this](const ::asio::error_code &ec) {
|
|
|
|
- ConnectComplete(ec);
|
|
|
|
|
|
+ next_layer_.async_connect(next_endpoint, [shared_this, this, next_endpoint](const ::asio::error_code &ec) {
|
|
|
|
+ ConnectComplete(ec, next_endpoint);
|
|
});
|
|
});
|
|
|
|
+ connect_timer_.expires_from_now(
|
|
|
|
+ std::chrono::milliseconds(options_.rpc_connect_timeout));
|
|
|
|
+ connect_timer_.async_wait([shared_this, this, next_endpoint](const ::asio::error_code &ec) {
|
|
|
|
+ if (ec)
|
|
|
|
+ ConnectComplete(ec, next_endpoint);
|
|
|
|
+ else
|
|
|
|
+ ConnectComplete(make_error_code(asio::error::host_unreachable), next_endpoint);
|
|
|
|
+ });
|
|
} else {
|
|
} else {
|
|
CommsError(status);
|
|
CommsError(status);
|
|
}
|
|
}
|
|
@@ -184,6 +220,7 @@ void RpcConnectionImpl<NextLayer>::SendHandshake(RpcCallback &handler) {
|
|
assert(lock_held(connection_state_lock_)); // Must be holding lock before calling
|
|
assert(lock_held(connection_state_lock_)); // Must be holding lock before calling
|
|
|
|
|
|
LOG_TRACE(kRPC, << "RpcConnectionImpl::SendHandshake called");
|
|
LOG_TRACE(kRPC, << "RpcConnectionImpl::SendHandshake called");
|
|
|
|
+ connected_ = kHandshaking;
|
|
|
|
|
|
auto shared_this = shared_from_this();
|
|
auto shared_this = shared_from_this();
|
|
auto handshake_packet = PrepareHandshakePacket();
|
|
auto handshake_packet = PrepareHandshakePacket();
|
|
@@ -250,6 +287,8 @@ void RpcConnectionImpl<NextLayer>::FlushPendingRequests() {
|
|
return;
|
|
return;
|
|
case kConnecting:
|
|
case kConnecting:
|
|
return;
|
|
return;
|
|
|
|
+ case kHandshaking:
|
|
|
|
+ return;
|
|
case kAuthenticating:
|
|
case kAuthenticating:
|
|
if (auth_requests_.empty()) {
|
|
if (auth_requests_.empty()) {
|
|
return;
|
|
return;
|
|
@@ -379,7 +418,7 @@ void RpcConnectionImpl<NextLayer>::Disconnect() {
|
|
LOG_INFO(kRPC, << "RpcConnectionImpl::Disconnect called");
|
|
LOG_INFO(kRPC, << "RpcConnectionImpl::Disconnect called");
|
|
|
|
|
|
request_over_the_wire_.reset();
|
|
request_over_the_wire_.reset();
|
|
- if (connected_ == kConnecting || connected_ == kAuthenticating || connected_ == kConnected) {
|
|
|
|
|
|
+ if (connected_ == kConnecting || connected_ == kHandshaking || connected_ == kAuthenticating || connected_ == kConnected) {
|
|
// Don't print out errors, we were expecting a disconnect here
|
|
// Don't print out errors, we were expecting a disconnect here
|
|
SafeDisconnect(get_asio_socket_ptr(&next_layer_));
|
|
SafeDisconnect(get_asio_socket_ptr(&next_layer_));
|
|
}
|
|
}
|