rpc_connection.h 5.1 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158
  1. /**
  2. * Licensed to the Apache Software Foundation (ASF) under one
  3. * or more contributor license agreements. See the NOTICE file
  4. * distributed with this work for additional information
  5. * regarding copyright ownership. The ASF licenses this file
  6. * to you under the Apache License, Version 2.0 (the
  7. * "License"); you may not use this file except in compliance
  8. * with the License. You may obtain a copy of the License at
  9. *
  10. * http://www.apache.org/licenses/LICENSE-2.0
  11. *
  12. * Unless required by applicable law or agreed to in writing, software
  13. * distributed under the License is distributed on an "AS IS" BASIS,
  14. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  15. * See the License for the specific language governing permissions and
  16. * limitations under the License.
  17. */
  18. #ifndef LIB_RPC_RPC_CONNECTION_H_
  19. #define LIB_RPC_RPC_CONNECTION_H_
  20. #include "rpc_engine.h"
  21. #include "common/logging.h"
  22. #include "common/util.h"
  23. #include <asio/connect.hpp>
  24. #include <asio/read.hpp>
  25. #include <asio/write.hpp>
  26. namespace hdfs {
  27. template <class NextLayer> class RpcConnectionImpl : public RpcConnection {
  28. public:
  29. RpcConnectionImpl(RpcEngine *engine);
  30. virtual void Connect(const ::asio::ip::tcp::endpoint &server,
  31. Callback &&handler) override;
  32. virtual void Handshake(Callback &&handler) override;
  33. virtual void Shutdown() override;
  34. virtual void OnSendCompleted(const ::asio::error_code &ec,
  35. size_t transferred) override;
  36. virtual void OnRecvCompleted(const ::asio::error_code &ec,
  37. size_t transferred) override;
  38. NextLayer &next_layer() { return next_layer_; }
  39. private:
  40. const Options options_;
  41. NextLayer next_layer_;
  42. };
  43. template <class NextLayer>
  44. RpcConnectionImpl<NextLayer>::RpcConnectionImpl(RpcEngine *engine)
  45. : RpcConnection(engine), options_(engine->options()),
  46. next_layer_(engine->io_service()) {}
  47. template <class NextLayer>
  48. void RpcConnectionImpl<NextLayer>::Connect(
  49. const ::asio::ip::tcp::endpoint &server, Callback &&handler) {
  50. next_layer_.async_connect(server,
  51. [handler](const ::asio::error_code &ec) {
  52. handler(ToStatus(ec));
  53. });
  54. }
  55. template <class NextLayer>
  56. void RpcConnectionImpl<NextLayer>::Handshake(Callback &&handler) {
  57. auto handshake_packet = PrepareHandshakePacket();
  58. ::asio::async_write(
  59. next_layer_, asio::buffer(*handshake_packet),
  60. [handshake_packet, handler](const ::asio::error_code &ec, size_t) {
  61. handler(ToStatus(ec));
  62. });
  63. }
  64. template <class NextLayer>
  65. void RpcConnectionImpl<NextLayer>::OnSendCompleted(const ::asio::error_code &ec,
  66. size_t) {
  67. using std::placeholders::_1;
  68. using std::placeholders::_2;
  69. std::lock_guard<std::mutex> state_lock(engine_state_lock_);
  70. request_over_the_wire_.reset();
  71. if (ec) {
  72. // Current RPC has failed -- abandon the
  73. // connection and do proper clean up
  74. ClearAndDisconnect(ec);
  75. return;
  76. }
  77. if (!pending_requests_.size()) {
  78. return;
  79. }
  80. std::shared_ptr<Request> req = pending_requests_.front();
  81. pending_requests_.erase(pending_requests_.begin());
  82. requests_on_fly_[req->call_id()] = req;
  83. request_over_the_wire_ = req;
  84. req->timer().expires_from_now(
  85. std::chrono::milliseconds(options_.rpc_timeout));
  86. req->timer().async_wait(std::bind(
  87. &RpcConnectionImpl<NextLayer>::HandleRpcTimeout, this, req, _1));
  88. asio::async_write(
  89. next_layer_, asio::buffer(req->payload()),
  90. std::bind(&RpcConnectionImpl<NextLayer>::OnSendCompleted, this, _1, _2));
  91. }
  92. template <class NextLayer>
  93. void RpcConnectionImpl<NextLayer>::OnRecvCompleted(const ::asio::error_code &ec,
  94. size_t) {
  95. using std::placeholders::_1;
  96. using std::placeholders::_2;
  97. std::lock_guard<std::mutex> state_lock(engine_state_lock_);
  98. switch (ec.value()) {
  99. case 0:
  100. // No errors
  101. break;
  102. case asio::error::operation_aborted:
  103. // The event loop has been shut down. Ignore the error.
  104. return;
  105. default:
  106. LOG_WARN() << "Network error during RPC: " << ec.message();
  107. ClearAndDisconnect(ec);
  108. return;
  109. }
  110. if (resp_state_ == kReadLength) {
  111. resp_state_ = kReadContent;
  112. auto buf = ::asio::buffer(reinterpret_cast<char *>(&resp_length_),
  113. sizeof(resp_length_));
  114. asio::async_read(next_layer_, buf,
  115. std::bind(&RpcConnectionImpl<NextLayer>::OnRecvCompleted,
  116. this, _1, _2));
  117. } else if (resp_state_ == kReadContent) {
  118. resp_state_ = kParseResponse;
  119. resp_length_ = ntohl(resp_length_);
  120. resp_data_.resize(resp_length_);
  121. asio::async_read(next_layer_, ::asio::buffer(resp_data_),
  122. std::bind(&RpcConnectionImpl<NextLayer>::OnRecvCompleted,
  123. this, _1, _2));
  124. } else if (resp_state_ == kParseResponse) {
  125. resp_state_ = kReadLength;
  126. HandleRpcResponse(resp_data_);
  127. resp_data_.clear();
  128. Start();
  129. }
  130. }
  131. template <class NextLayer> void RpcConnectionImpl<NextLayer>::Shutdown() {
  132. next_layer_.cancel();
  133. next_layer_.close();
  134. }
  135. }
  136. #endif