|
@@ -33,10 +33,10 @@ template <class NextLayer>
|
|
class RpcConnectionImpl : public RpcConnection {
|
|
class RpcConnectionImpl : public RpcConnection {
|
|
public:
|
|
public:
|
|
RpcConnectionImpl(RpcEngine *engine);
|
|
RpcConnectionImpl(RpcEngine *engine);
|
|
- virtual void Connect(const ::asio::ip::tcp::endpoint &server,
|
|
|
|
|
|
+ virtual void Connect(const std::vector<::asio::ip::tcp::endpoint> &server,
|
|
RpcCallback &handler);
|
|
RpcCallback &handler);
|
|
virtual void ConnectAndFlush(
|
|
virtual void ConnectAndFlush(
|
|
- const ::asio::ip::tcp::endpoint &server) override;
|
|
|
|
|
|
+ const std::vector<::asio::ip::tcp::endpoint> &server) override;
|
|
virtual void Handshake(RpcCallback &handler) override;
|
|
virtual void Handshake(RpcCallback &handler) override;
|
|
virtual void Disconnect() override;
|
|
virtual void Disconnect() override;
|
|
virtual void OnSendCompleted(const ::asio::error_code &ec,
|
|
virtual void OnSendCompleted(const ::asio::error_code &ec,
|
|
@@ -52,7 +52,11 @@ public:
|
|
|
|
|
|
private:
|
|
private:
|
|
const Options options_;
|
|
const Options options_;
|
|
|
|
+ std::vector<::asio::ip::tcp::endpoint> additional_endpoints_;
|
|
NextLayer next_layer_;
|
|
NextLayer next_layer_;
|
|
|
|
+
|
|
|
|
+ void ConnectComplete(const ::asio::error_code &ec);
|
|
|
|
+ void HandshakeComplete(const Status &s);
|
|
};
|
|
};
|
|
|
|
|
|
template <class NextLayer>
|
|
template <class NextLayer>
|
|
@@ -63,7 +67,7 @@ RpcConnectionImpl<NextLayer>::RpcConnectionImpl(RpcEngine *engine)
|
|
|
|
|
|
template <class NextLayer>
|
|
template <class NextLayer>
|
|
void RpcConnectionImpl<NextLayer>::Connect(
|
|
void RpcConnectionImpl<NextLayer>::Connect(
|
|
- const ::asio::ip::tcp::endpoint &server, RpcCallback &handler) {
|
|
|
|
|
|
+ const std::vector<::asio::ip::tcp::endpoint> &server, RpcCallback &handler) {
|
|
auto connectionSuccessfulReq = std::make_shared<Request>(
|
|
auto connectionSuccessfulReq = std::make_shared<Request>(
|
|
engine_, [handler](::google::protobuf::io::CodedInputStream *is,
|
|
engine_, [handler](::google::protobuf::io::CodedInputStream *is,
|
|
const Status &status) {
|
|
const Status &status) {
|
|
@@ -76,28 +80,65 @@ void RpcConnectionImpl<NextLayer>::Connect(
|
|
|
|
|
|
template <class NextLayer>
|
|
template <class NextLayer>
|
|
void RpcConnectionImpl<NextLayer>::ConnectAndFlush(
|
|
void RpcConnectionImpl<NextLayer>::ConnectAndFlush(
|
|
- const ::asio::ip::tcp::endpoint &server) {
|
|
|
|
- std::shared_ptr<RpcConnection> shared_this = shared_from_this();
|
|
|
|
- next_layer_.async_connect(server,
|
|
|
|
- [shared_this, this](const ::asio::error_code &ec) {
|
|
|
|
- std::lock_guard<std::mutex> state_lock(connection_state_lock_);
|
|
|
|
- Status status = ToStatus(ec);
|
|
|
|
- if (status.ok()) {
|
|
|
|
- StartReading();
|
|
|
|
- Handshake([shared_this, this](const Status &s) {
|
|
|
|
- std::lock_guard<std::mutex> state_lock(connection_state_lock_);
|
|
|
|
- if (s.ok()) {
|
|
|
|
- FlushPendingRequests();
|
|
|
|
- } else {
|
|
|
|
- CommsError(s);
|
|
|
|
- };
|
|
|
|
- });
|
|
|
|
- } else {
|
|
|
|
- CommsError(status);
|
|
|
|
- }
|
|
|
|
- });
|
|
|
|
|
|
+ const std::vector<::asio::ip::tcp::endpoint> &server) {
|
|
|
|
+ std::lock_guard<std::mutex> state_lock(connection_state_lock_);
|
|
|
|
+
|
|
|
|
+ if (server.empty()) {
|
|
|
|
+ Status s = Status::InvalidArgument("No endpoints provided");
|
|
|
|
+ CommsError(s);
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Take the first endpoint, but remember the alternatives for later
|
|
|
|
+ additional_endpoints_ = server;
|
|
|
|
+ ::asio::ip::tcp::endpoint first_endpoint = additional_endpoints_.front();
|
|
|
|
+ additional_endpoints_.erase(additional_endpoints_.begin());
|
|
|
|
+
|
|
|
|
+ auto shared_this = shared_from_this();
|
|
|
|
+ next_layer_.async_connect(first_endpoint, [shared_this, this](const ::asio::error_code &ec) {
|
|
|
|
+ ConnectComplete(ec);
|
|
|
|
+ });
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+template <class NextLayer>
|
|
|
|
+void RpcConnectionImpl<NextLayer>::ConnectComplete(const ::asio::error_code &ec) {
|
|
|
|
+ auto shared_this = RpcConnectionImpl<NextLayer>::shared_from_this();
|
|
|
|
+ std::lock_guard<std::mutex> state_lock(connection_state_lock_);
|
|
|
|
+
|
|
|
|
+ Status status = ToStatus(ec);
|
|
|
|
+ if (status.ok()) {
|
|
|
|
+ StartReading();
|
|
|
|
+ Handshake([shared_this, this](const Status & s) {
|
|
|
|
+ HandshakeComplete(s);
|
|
|
|
+ });
|
|
|
|
+ } else {
|
|
|
|
+ next_layer_.close();
|
|
|
|
+ if (!additional_endpoints_.empty()) {
|
|
|
|
+ // If we have additional endpoints, keep trying until we either run out or
|
|
|
|
+ // hit one
|
|
|
|
+ ::asio::ip::tcp::endpoint next_endpoint = additional_endpoints_.front();
|
|
|
|
+ additional_endpoints_.erase(additional_endpoints_.begin());
|
|
|
|
+
|
|
|
|
+ next_layer_.async_connect(next_endpoint, [shared_this, this](const ::asio::error_code &ec) {
|
|
|
|
+ ConnectComplete(ec);
|
|
|
|
+ });
|
|
|
|
+ } else {
|
|
|
|
+ CommsError(status);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+template <class NextLayer>
|
|
|
|
+void RpcConnectionImpl<NextLayer>::HandshakeComplete(const Status &s) {
|
|
|
|
+ std::lock_guard<std::mutex> state_lock(connection_state_lock_);
|
|
|
|
+ if (s.ok()) {
|
|
|
|
+ FlushPendingRequests();
|
|
|
|
+ } else {
|
|
|
|
+ CommsError(s);
|
|
|
|
+ };
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+
|
|
template <class NextLayer>
|
|
template <class NextLayer>
|
|
void RpcConnectionImpl<NextLayer>::Handshake(RpcCallback &handler) {
|
|
void RpcConnectionImpl<NextLayer>::Handshake(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
|