rpc_connection.h 4.8 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149
  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/util.h"
  22. #include <asio/connect.hpp>
  23. #include <asio/read.hpp>
  24. #include <asio/write.hpp>
  25. namespace hdfs {
  26. template <class NextLayer> class RpcConnectionImpl : public RpcConnection {
  27. public:
  28. RpcConnectionImpl(RpcEngine *engine);
  29. virtual void Connect(const std::vector<::asio::ip::tcp::endpoint> &server,
  30. Callback &&handler) override;
  31. virtual void Handshake(Callback &&handler) override;
  32. virtual void Shutdown() override;
  33. virtual void OnSendCompleted(const ::asio::error_code &ec,
  34. size_t transferred) override;
  35. virtual void OnRecvCompleted(const ::asio::error_code &ec,
  36. size_t transferred) override;
  37. private:
  38. NextLayer next_layer_;
  39. };
  40. template <class NextLayer>
  41. RpcConnectionImpl<NextLayer>::RpcConnectionImpl(RpcEngine *engine)
  42. : RpcConnection(engine)
  43. , next_layer_(engine->io_service())
  44. {}
  45. template <class NextLayer>
  46. void RpcConnectionImpl<NextLayer>::Connect(
  47. const std::vector<::asio::ip::tcp::endpoint> &server, Callback &&handler) {
  48. ::asio::async_connect(
  49. next_layer_, server.begin(), server.end(),
  50. [handler](const ::asio::error_code &ec,
  51. std::vector<::asio::ip::tcp::endpoint>::const_iterator) {
  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. // TODO: Current RPC has failed -- we should abandon the
  73. // connection and do proper clean up
  74. assert(false && "Unimplemented");
  75. }
  76. if (!pending_requests_.size()) {
  77. return;
  78. }
  79. std::shared_ptr<Request> req = pending_requests_.front();
  80. pending_requests_.erase(pending_requests_.begin());
  81. requests_on_fly_[req->call_id()] = req;
  82. request_over_the_wire_ = req;
  83. // TODO: set the timeout for the RPC request
  84. asio::async_write(
  85. next_layer_, asio::buffer(req->payload()),
  86. std::bind(&RpcConnectionImpl<NextLayer>::OnSendCompleted, this, _1, _2));
  87. }
  88. template <class NextLayer>
  89. void RpcConnectionImpl<NextLayer>::OnRecvCompleted(const ::asio::error_code &ec,
  90. size_t) {
  91. using std::placeholders::_1;
  92. using std::placeholders::_2;
  93. std::lock_guard<std::mutex> state_lock(engine_state_lock_);
  94. switch (ec.value()) {
  95. case 0:
  96. // No errors
  97. break;
  98. case asio::error::operation_aborted:
  99. // The event loop has been shut down. Ignore the error.
  100. return;
  101. default:
  102. assert(false && "Unimplemented");
  103. }
  104. if (resp_state_ == kReadLength) {
  105. resp_state_ = kReadContent;
  106. auto buf = ::asio::buffer(reinterpret_cast<char *>(&resp_length_),
  107. sizeof(resp_length_));
  108. asio::async_read(next_layer_, buf,
  109. std::bind(&RpcConnectionImpl<NextLayer>::OnRecvCompleted,
  110. this, _1, _2));
  111. } else if (resp_state_ == kReadContent) {
  112. resp_state_ = kParseResponse;
  113. resp_length_ = ntohl(resp_length_);
  114. resp_data_.resize(resp_length_);
  115. asio::async_read(next_layer_, ::asio::buffer(resp_data_),
  116. std::bind(&RpcConnectionImpl<NextLayer>::OnRecvCompleted, this, _1, _2));
  117. } else if (resp_state_ == kParseResponse) {
  118. resp_state_ = kReadLength;
  119. HandleRpcResponse(resp_data_);
  120. resp_data_.clear();
  121. Start();
  122. }
  123. }
  124. template <class NextLayer> void RpcConnectionImpl<NextLayer>::Shutdown() {
  125. next_layer_.close();
  126. }
  127. }
  128. #endif