mock_connection.h 4.5 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141
  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 LIBHDFSPP_TEST_MOCK_CONNECTION_H_
  19. #define LIBHDFSPP_TEST_MOCK_CONNECTION_H_
  20. #include "common/async_stream.h"
  21. #include <boost/system/error_code.hpp>
  22. #include <boost/asio/buffer.hpp>
  23. #include <boost/asio/streambuf.hpp>
  24. #include <boost/asio/io_service.hpp>
  25. #include <gmock/gmock.h>
  26. namespace hdfs {
  27. typedef std::pair<boost::system::error_code, std::string> ProducerResult;
  28. class AsioProducer {
  29. public:
  30. /*
  31. * Return either:
  32. * (::boost::system::error_code(), <some data>) for a good result
  33. * (<an ::asio::error instance>, <anything>) to pass an error to the caller
  34. * (::asio::error::would_block, <anything>) to block the next call forever
  35. */
  36. virtual ProducerResult Produce() = 0;
  37. };
  38. class MockConnectionBase : public AsioProducer, public AsyncStream {
  39. public:
  40. MockConnectionBase(boost::asio::io_service *io_service);
  41. virtual ~MockConnectionBase();
  42. typedef std::pair<boost::system::error_code, std::string> ProducerResult;
  43. void async_read_some(const MutableBuffer &buf,
  44. std::function<void (const boost::system::error_code & error,
  45. std::size_t bytes_transferred) > handler) override {
  46. if (produced_.size() == 0) {
  47. ProducerResult r = Produce();
  48. if (r.first == boost::asio::error::would_block) {
  49. return; // No more reads to do
  50. }
  51. if (r.first) {
  52. io_service_->post(std::bind(handler, r.first, 0));
  53. return;
  54. }
  55. boost::asio::mutable_buffers_1 data = produced_.prepare(r.second.size());
  56. boost::asio::buffer_copy(data, boost::asio::buffer(r.second));
  57. produced_.commit(r.second.size());
  58. }
  59. size_t len = std::min(boost::asio::buffer_size(buf), produced_.size());
  60. boost::asio::buffer_copy(buf, produced_.data());
  61. produced_.consume(len);
  62. io_service_->post(std::bind(handler, boost::system::error_code(), len));
  63. }
  64. void async_write_some(const ConstBuffer &buf,
  65. std::function<void (const boost::system::error_code & error,
  66. std::size_t bytes_transferred) > handler) override {
  67. // CompletionResult res = OnWrite(buf);
  68. io_service_->post(std::bind(handler, boost::system::error_code(), boost::asio::buffer_size(buf)));
  69. }
  70. template <class Endpoint, class Callback>
  71. void async_connect(const Endpoint &, Callback &&handler) {
  72. io_service_->post([handler]() { handler(::boost::system::error_code()); });
  73. }
  74. virtual void cancel() {}
  75. virtual void close() {}
  76. protected:
  77. ProducerResult Produce() override = 0;
  78. boost::asio::io_service *io_service_;
  79. private:
  80. boost::asio::streambuf produced_;
  81. };
  82. class SharedConnectionData : public AsioProducer {
  83. public:
  84. bool checkProducerForConnect = false;
  85. MOCK_METHOD0(Produce, ProducerResult());
  86. };
  87. class SharedMockConnection : public MockConnectionBase {
  88. public:
  89. using MockConnectionBase::MockConnectionBase;
  90. template <class Endpoint, class Callback>
  91. void async_connect(const Endpoint &, Callback &&handler) {
  92. auto data = shared_connection_data_.lock();
  93. assert(data);
  94. if (!data->checkProducerForConnect) {
  95. io_service_->post([handler]() { handler(::boost::system::error_code()); });
  96. } else {
  97. ProducerResult result = Produce();
  98. if (result.first == boost::asio::error::would_block) {
  99. return; // Connect will hang
  100. } else {
  101. io_service_->post([handler, result]() { handler( result.first); });
  102. }
  103. }
  104. }
  105. static void SetSharedConnectionData(std::shared_ptr<SharedConnectionData> new_producer) {
  106. shared_connection_data_ = new_producer; // get a weak reference to it
  107. }
  108. protected:
  109. ProducerResult Produce() override;
  110. static std::weak_ptr<SharedConnectionData> shared_connection_data_;
  111. };
  112. }
  113. #endif