protobuf.h 4.2 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128
  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_COMMON_CONTINUATION_PROTOBUF_H_
  19. #define LIBHDFSPP_COMMON_CONTINUATION_PROTOBUF_H_
  20. #include "common/util.h"
  21. #include <google/protobuf/message_lite.h>
  22. #include <google/protobuf/io/coded_stream.h>
  23. #include <google/protobuf/io/zero_copy_stream_impl_lite.h>
  24. #include <cassert>
  25. namespace hdfs {
  26. namespace continuation {
  27. template <class Stream, size_t MaxMessageSize = 512>
  28. struct ReadDelimitedPBMessageContinuation : public Continuation {
  29. ReadDelimitedPBMessageContinuation(Stream *stream,
  30. ::google::protobuf::MessageLite *msg)
  31. : stream_(stream), msg_(msg) {}
  32. virtual void Run(const Next &next) override {
  33. namespace pbio = google::protobuf::io;
  34. auto handler = [this, next](const asio::error_code &ec, size_t) {
  35. Status status;
  36. if (ec) {
  37. status = ToStatus(ec);
  38. } else {
  39. pbio::ArrayInputStream as(&buf_[0], buf_.size());
  40. pbio::CodedInputStream is(&as);
  41. uint32_t size = 0;
  42. bool v = is.ReadVarint32(&size);
  43. assert(v);
  44. is.PushLimit(size);
  45. msg_->Clear();
  46. v = msg_->MergeFromCodedStream(&is);
  47. assert(v);
  48. }
  49. next(status);
  50. };
  51. asio::async_read(
  52. *stream_, asio::buffer(buf_),
  53. std::bind(&ReadDelimitedPBMessageContinuation::CompletionHandler, this,
  54. std::placeholders::_1, std::placeholders::_2),
  55. handler);
  56. }
  57. private:
  58. size_t CompletionHandler(const asio::error_code &ec, size_t transferred) {
  59. if (ec) {
  60. return 0;
  61. }
  62. size_t offset = 0, len = 0;
  63. for (size_t i = 0; i + 1 < transferred && i < sizeof(int); ++i) {
  64. len = (len << 7) | (buf_[i] & 0x7f);
  65. if ((uint8_t)buf_.at(i) < 0x80) {
  66. offset = i + 1;
  67. break;
  68. }
  69. }
  70. assert(offset + len < buf_.size() && "Message is too big");
  71. return offset ? len + offset - transferred : 1;
  72. }
  73. Stream *stream_;
  74. ::google::protobuf::MessageLite *msg_;
  75. std::array<char, MaxMessageSize> buf_;
  76. };
  77. template <class Stream>
  78. struct WriteDelimitedPBMessageContinuation : Continuation {
  79. WriteDelimitedPBMessageContinuation(Stream *stream,
  80. const google::protobuf::MessageLite *msg)
  81. : stream_(stream), msg_(msg) {}
  82. virtual void Run(const Next &next) override {
  83. namespace pbio = google::protobuf::io;
  84. int size = msg_->ByteSize();
  85. buf_.reserve(pbio::CodedOutputStream::VarintSize32(size) + size);
  86. pbio::StringOutputStream ss(&buf_);
  87. pbio::CodedOutputStream os(&ss);
  88. os.WriteVarint32(size);
  89. msg_->SerializeToCodedStream(&os);
  90. write_coroutine_ =
  91. std::shared_ptr<Continuation>(Write(stream_, asio::buffer(buf_)));
  92. write_coroutine_->Run([next](const Status &stat) { next(stat); });
  93. }
  94. private:
  95. Stream *stream_;
  96. const google::protobuf::MessageLite *msg_;
  97. std::string buf_;
  98. std::shared_ptr<Continuation> write_coroutine_;
  99. };
  100. template <class Stream, size_t MaxMessageSize = 512>
  101. static inline Continuation *
  102. ReadDelimitedPBMessage(Stream *stream, ::google::protobuf::MessageLite *msg) {
  103. return new ReadDelimitedPBMessageContinuation<Stream, MaxMessageSize>(stream,
  104. msg);
  105. }
  106. template <class Stream>
  107. static inline Continuation *
  108. WriteDelimitedPBMessage(Stream *stream, ::google::protobuf::MessageLite *msg) {
  109. return new WriteDelimitedPBMessageContinuation<Stream>(stream, msg);
  110. }
  111. }
  112. }
  113. #endif