]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
msgr/async: huge refactoring of protocol V1 23415/head
authorRicardo Dias <rdias@suse.com>
Thu, 28 Jun 2018 16:16:30 +0000 (17:16 +0100)
committerRicardo Dias <rdias@suse.com>
Wed, 19 Sep 2018 12:47:48 +0000 (13:47 +0100)
This huge commit extracts the protocol V1 logic from the AsyncConnection
class into its own class called ProtocolV1.
The new protocol class extends from a Protocol base class that defines
an interface to allow the implementation of different wire protocols
in the future, such as the msgr V2 protocol.

Signed-off-by: Ricardo Dias <rdias@suse.com>
src/msg/CMakeLists.txt
src/msg/async/AsyncConnection.cc
src/msg/async/AsyncConnection.h
src/msg/async/AsyncMessenger.cc
src/msg/async/AsyncMessenger.h
src/msg/async/Protocol.cc [new file with mode: 0644]
src/msg/async/Protocol.h [new file with mode: 0644]

index 28c9ad7840011ffdceab7848cc1c8867aa7d46dd..099888a2ee17027fcc9161b6dd9efd20a2a90334 100644 (file)
@@ -21,6 +21,7 @@ endif(HAVE_XIO)
 list(APPEND msg_srcs
   async/AsyncConnection.cc
   async/AsyncMessenger.cc
+  async/Protocol.cc
   async/Event.cc
   async/EventSelect.cc
   async/PosixStack.cc
index e631ff1033f037ff3e9c565c18ca58f67390eda2..cf2110d92a541a681085d00f787bc7f3a537b4f6 100644 (file)
@@ -1,4 +1,4 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
 // vim: ts=8 sw=2 smarttab
 /*
  * Ceph - scalable distributed file system
 #include "AsyncMessenger.h"
 #include "AsyncConnection.h"
 
+#include "Protocol.h"
+
 #include "messages/MOSDOp.h"
 #include "messages/MOSDOpReply.h"
 #include "common/EventTrace.h"
 
 // Constant to limit starting sequence number to 2^31.  Nothing special about it, just a big number.  PLR
-#define SEQ_MASK  0x7fffffff 
+#define SEQ_MASK  0x7fffffff
 
 #define dout_subsys ceph_subsys_ms
 #undef dout_prefix
@@ -38,8 +40,6 @@ ostream& AsyncConnection::_conn_prefix(std::ostream *_dout) {
                << (msgr2 ? " msgr2" : " legacy")
                 << " :" << port
                 << " s=" << get_state_name(state)
-                << " pgs=" << peer_global_seq
-                << " cs=" << connect_seq
                 << " l=" << policy.lossy
                 << ").";
 }
@@ -48,7 +48,6 @@ ostream& AsyncConnection::_conn_prefix(std::ostream *_dout) {
 // 1. Don't dispatch any event when closed! It may cause AsyncConnection alive even if AsyncMessenger dead
 
 const uint32_t AsyncConnection::TCP_PREFETCH_MIN_SIZE = 512;
-const int ASYNC_COALESCE_THRESHOLD = 256;
 
 class C_time_wakeup : public EventCallback {
   AsyncConnectionRef conn;
@@ -80,6 +79,14 @@ class C_handle_write : public EventCallback {
   }
 };
 
+class C_handle_write_callback : public EventCallback {
+  AsyncConnectionRef conn;
+
+public:
+  explicit C_handle_write_callback(AsyncConnectionRef c) : conn(c) {}
+  void do_request(uint64_t fd) override { conn->handle_write_callback(); }
+};
+
 class C_clean_handler : public EventCallback {
   AsyncConnectionRef conn;
  public:
@@ -100,62 +107,39 @@ class C_tick_wakeup : public EventCallback {
   }
 };
 
-static void alloc_aligned_buffer(bufferlist& data, unsigned len, unsigned off)
-{
-  // create a buffer to read into that matches the data alignment
-  unsigned alloc_len = 0;
-  unsigned left = len;
-  unsigned head = 0;
-  if (off & ~CEPH_PAGE_MASK) {
-    // head
-    alloc_len += CEPH_PAGE_SIZE;
-    head = std::min<uint64_t>(CEPH_PAGE_SIZE - (off & ~CEPH_PAGE_MASK), left);
-    left -= head;
-  }
-  alloc_len += left;
-  bufferptr ptr(buffer::create_small_page_aligned(alloc_len));
-  if (head)
-    ptr.set_offset(CEPH_PAGE_SIZE - head);
-  data.push_back(std::move(ptr));
-}
 
-AsyncConnection::AsyncConnection(
-  CephContext *cct, AsyncMessenger *m, DispatchQueue *q,
-  Worker *w, bool m2)
+AsyncConnection::AsyncConnection(CephContext *cct, AsyncMessenger *m, DispatchQueue *q,
+                                 Worker *w, bool m2, bool local)
   : Connection(cct, m), delay_state(NULL), async_msgr(m), conn_id(q->get_id()),
-    logger(w->get_perf_counter()), global_seq(0), connect_seq(0), peer_global_seq(0),
-    state(STATE_NONE), state_after_send(STATE_NONE), port(-1),
-    dispatch_queue(q), can_write(WriteStatus::NOWRITE),
-    keepalive(false), recv_buf(NULL),
+    logger(w->get_perf_counter()),
+    state(STATE_NONE), port(-1),
+    dispatch_queue(q), recv_buf(NULL),
     recv_max_prefetch(std::max<int64_t>(msgr->cct->_conf->ms_tcp_prefetch_max_size, TCP_PREFETCH_MIN_SIZE)),
     recv_start(0), recv_end(0),
     last_active(ceph::coarse_mono_clock::now()),
     inactive_timeout_us(cct->_conf->ms_tcp_read_timeout*1000*1000),
-    msg_left(0), cur_msg_size(0), got_bad_auth(false),
-    authorizer(NULL),
-    msgr2(m2), replacing(false),
-    is_reset_from_peer(false), once_ready(false), state_buffer(NULL), state_offset(0),
-    worker(w), center(&w->center)
+    msgr2(m2), state_offset(0),
+    worker(w), center(&w->center),read_buffer(nullptr)
 {
   read_handler = new C_handle_read(this);
   write_handler = new C_handle_write(this);
+  write_callback_handler = new C_handle_write_callback(this);
   wakeup_handler = new C_time_wakeup(this);
   tick_handler = new C_tick_wakeup(this);
   // double recv_max_prefetch see "read_until"
   recv_buf = new char[2*recv_max_prefetch];
-  state_buffer = new char[4096];
+  if (local) {
+    protocol = std::unique_ptr<Protocol>(new LoopbackProtocolV1(this));
+  } else {
+    protocol = std::unique_ptr<Protocol>(new ProtocolV1(this));
+  }
   logger->inc(l_msgr_created_connections);
 }
 
 AsyncConnection::~AsyncConnection()
 {
-  ceph_assert(out_q.empty());
-  ceph_assert(sent.empty());
-  delete authorizer;
   if (recv_buf)
     delete[] recv_buf;
-  if (state_buffer)
-    delete[] state_buffer;
   ceph_assert(!delay_state);
 }
 
@@ -175,65 +159,22 @@ void AsyncConnection::maybe_start_delay_thread()
   }
 }
 
-/* return -1 means `fd` occurs error or closed, it should be closed
- * return 0 means EAGAIN or EINTR */
-ssize_t AsyncConnection::read_bulk(char *buf, unsigned len)
-{
-  ssize_t nread;
- again:
-  nread = cs.read(buf, len);
-  if (nread < 0) {
-    if (nread == -EAGAIN) {
-      nread = 0;
-    } else if (nread == -EINTR) {
-      goto again;
-    } else {
-      ldout(async_msgr->cct, 1) << __func__ << " reading from fd=" << cs.fd()
-                          << " : "<< strerror(nread) << dendl;
-      return -1;
-    }
-  } else if (nread == 0) {
-    ldout(async_msgr->cct, 1) << __func__ << " peer close file descriptor "
-                              << cs.fd() << dendl;
-    return -1;
-  }
-  return nread;
-}
-
-// return the remaining bytes, it may larger than the length of ptr
-// else return < 0 means error
-ssize_t AsyncConnection::_try_send(bool more)
-{
-  if (async_msgr->cct->_conf->ms_inject_socket_failures && cs) {
-    if (rand() % async_msgr->cct->_conf->ms_inject_socket_failures == 0) {
-      ldout(async_msgr->cct, 0) << __func__ << " injecting socket failure" << dendl;
-      cs.shutdown();
-    }
-  }
-
-  ceph_assert(center->in_thread());
-  ssize_t r = cs.send(outcoming_bl, more);
-  if (r < 0) {
-    ldout(async_msgr->cct, 1) << __func__ << " send error: " << cpp_strerror(r) << dendl;
-    return r;
-  }
 
-  ldout(async_msgr->cct, 10) << __func__ << " sent bytes " << r
-                             << " remaining bytes " << outcoming_bl.length() << dendl;
-
-  if (!open_write && is_queued()) {
-    center->create_file_event(cs.fd(), EVENT_WRITABLE, write_handler);
-    open_write = true;
-  }
-
-  if (open_write && !is_queued()) {
-    center->delete_file_event(cs.fd(), EVENT_WRITABLE);
-    open_write = false;
-    if (state_after_send != STATE_NONE)
-      center->dispatch_event_external(read_handler);
+void AsyncConnection::read(unsigned len, char *buffer,
+                           std::function<void(char *, ssize_t)> callback) {
+  ldout(async_msgr->cct, 20) << __func__
+                             << (pendingReadLen ? " continue" : " start")
+                             << " len=" << len << dendl;
+  readCallback = callback;
+  pendingReadLen = len;
+  read_buffer = buffer;
+  ssize_t r = read_until(len, buffer);
+  if (r <= 0) {
+    // read all bytes, or an error occured
+    pendingReadLen.reset();
+    read_buffer = nullptr;
+    callback(buffer, r);
   }
-
-  return outcoming_bl.length();
 }
 
 // Because this func will be called multi times to populate
@@ -317,1602 +258,205 @@ ssize_t AsyncConnection::read_until(unsigned len, char *p)
   return len - state_offset;
 }
 
-void AsyncConnection::inject_delay() {
-  if (async_msgr->cct->_conf->ms_inject_internal_delays) {
-    ldout(async_msgr->cct, 10) << __func__ << " sleep for " << 
-      async_msgr->cct->_conf->ms_inject_internal_delays << dendl;
-    utime_t t;
-    t.set_from_double(async_msgr->cct->_conf->ms_inject_internal_delays);
-    t.sleep();
-  }
-}
-
-void AsyncConnection::process()
+/* return -1 means `fd` occurs error or closed, it should be closed
+ * return 0 means EAGAIN or EINTR */
+ssize_t AsyncConnection::read_bulk(char *buf, unsigned len)
 {
-  ssize_t r = 0;
-  int prev_state = state;
-#if defined(WITH_LTTNG) && defined(WITH_EVENTTRACE)
-  utime_t ltt_recv_stamp = ceph_clock_now();
-#endif
-  bool need_dispatch_writer = false;
-  std::lock_guard<std::mutex> l(lock);
-  last_active = ceph::coarse_mono_clock::now();
-  auto recv_start_time = ceph::mono_clock::now();
-  do {
-    ldout(async_msgr->cct, 20) << __func__ << " prev state is " << get_state_name(prev_state) << dendl;
-    prev_state = state;
-    switch (state) {
-      case STATE_OPEN:
-        {
-          char tag = -1;
-          r = read_until(sizeof(tag), &tag);
-          if (r < 0) {
-            ldout(async_msgr->cct, 1) << __func__ << " read tag failed" << dendl;
-            goto fail;
-          } else if (r > 0) {
-            break;
-          }
-
-          if (tag == CEPH_MSGR_TAG_KEEPALIVE) {
-            ldout(async_msgr->cct, 20) << __func__ << " got KEEPALIVE" << dendl;
-           set_last_keepalive(ceph_clock_now());
-          } else if (tag == CEPH_MSGR_TAG_KEEPALIVE2) {
-            state = STATE_OPEN_KEEPALIVE2;
-          } else if (tag == CEPH_MSGR_TAG_KEEPALIVE2_ACK) {
-            state = STATE_OPEN_KEEPALIVE2_ACK;
-          } else if (tag == CEPH_MSGR_TAG_ACK) {
-            state = STATE_OPEN_TAG_ACK;
-          } else if (tag == CEPH_MSGR_TAG_MSG) {
-            state = STATE_OPEN_MESSAGE_HEADER;
-          } else if (tag == CEPH_MSGR_TAG_CLOSE) {
-            state = STATE_OPEN_TAG_CLOSE;
-          } else {
-            ldout(async_msgr->cct, 0) << __func__ << " bad tag " << (int)tag << dendl;
-            goto fail;
-          }
-
-          break;
-        }
-
-      case STATE_OPEN_KEEPALIVE2:
-        {
-          ceph_timespec *t;
-          r = read_until(sizeof(*t), state_buffer);
-          if (r < 0) {
-            ldout(async_msgr->cct, 1) << __func__ << " read keeplive timespec failed" << dendl;
-            goto fail;
-          } else if (r > 0) {
-            break;
-          }
-
-          ldout(async_msgr->cct, 30) << __func__ << " got KEEPALIVE2 tag ..." << dendl;
-          t = (ceph_timespec*)state_buffer;
-          utime_t kp_t = utime_t(*t);
-          write_lock.lock();
-          _append_keepalive_or_ack(true, &kp_t);
-         write_lock.unlock();
-          ldout(async_msgr->cct, 20) << __func__ << " got KEEPALIVE2 " << kp_t << dendl;
-         set_last_keepalive(ceph_clock_now());
-          need_dispatch_writer = true;
-          state = STATE_OPEN;
-          break;
-        }
-
-      case STATE_OPEN_KEEPALIVE2_ACK:
-        {
-          ceph_timespec *t;
-          r = read_until(sizeof(*t), state_buffer);
-          if (r < 0) {
-            ldout(async_msgr->cct, 1) << __func__ << " read keeplive timespec failed" << dendl;
-            goto fail;
-          } else if (r > 0) {
-            break;
-          }
-
-          t = (ceph_timespec*)state_buffer;
-          set_last_keepalive_ack(utime_t(*t));
-          ldout(async_msgr->cct, 20) << __func__ << " got KEEPALIVE_ACK" << dendl;
-          state = STATE_OPEN;
-          break;
-        }
-
-      case STATE_OPEN_TAG_ACK:
-        {
-          ceph_le64 *seq;
-          r = read_until(sizeof(*seq), state_buffer);
-          if (r < 0) {
-            ldout(async_msgr->cct, 1) << __func__ << " read ack seq failed" << dendl;
-            goto fail;
-          } else if (r > 0) {
-            break;
-          }
-
-          seq = (ceph_le64*)state_buffer;
-          ldout(async_msgr->cct, 20) << __func__ << " got ACK" << dendl;
-          handle_ack(*seq);
-          state = STATE_OPEN;
-          break;
-        }
-
-      case STATE_OPEN_MESSAGE_HEADER:
-        {
-#if defined(WITH_LTTNG) && defined(WITH_EVENTTRACE)
-          ltt_recv_stamp = ceph_clock_now();
-#endif
-          recv_stamp = ceph_clock_now();
-          ldout(async_msgr->cct, 20) << __func__ << " begin MSG" << dendl;
-          ceph_msg_header header;
-          __u32 header_crc = 0;
-
-          r = read_until(sizeof(header), state_buffer);
-          if (r < 0) {
-            ldout(async_msgr->cct, 1) << __func__ << " read message header failed" << dendl;
-            goto fail;
-          } else if (r > 0) {
-            break;
-          }
-
-          ldout(async_msgr->cct, 20) << __func__ << " got MSG header" << dendl;
-
-          header = *((ceph_msg_header*)state_buffer);
-
-         ldout(async_msgr->cct, 20) << __func__ << " got envelope type=" << header.type
-                              << " src " << entity_name_t(header.src)
-                              << " front=" << header.front_len
-                              << " data=" << header.data_len
-                              << " off " << header.data_off << dendl;
-
-         if (msgr->crcflags & MSG_CRC_HEADER) {
-           header_crc = ceph_crc32c(0, (unsigned char *)&header,
-               sizeof(header) - sizeof(header.crc));
-           // verify header crc
-           if (header_crc != header.crc) {
-             ldout(async_msgr->cct,0) << __func__ << " got bad header crc "
-               << header_crc << " != " << header.crc << dendl;
-             goto fail;
-           }
-         }
-
-          // Reset state
-          data_buf.clear();
-          front.clear();
-          middle.clear();
-          data.clear();
-          current_header = header;
-          state = STATE_OPEN_MESSAGE_THROTTLE_MESSAGE;
-          break;
-        }
-
-      case STATE_OPEN_MESSAGE_THROTTLE_MESSAGE:
-        {
-          if (policy.throttler_messages) {
-            ldout(async_msgr->cct, 10) << __func__ << " wants " << 1 << " message from policy throttler "
-                                       << policy.throttler_messages->get_current() << "/"
-                                       << policy.throttler_messages->get_max() << dendl;
-            if (!policy.throttler_messages->get_or_fail()) {
-              ldout(async_msgr->cct, 10) << __func__ << " wants 1 message from policy throttle "
-                                        << policy.throttler_messages->get_current() << "/"
-                                        << policy.throttler_messages->get_max() << " failed, just wait." << dendl;
-              // following thread pool deal with th full message queue isn't a
-              // short time, so we can wait a ms.
-              if (register_time_events.empty())
-                register_time_events.insert(center->create_time_event(1000, wakeup_handler));
-              break;
-            }
-          }
-
-          state = STATE_OPEN_MESSAGE_THROTTLE_BYTES;
-          break;
-        }
-
-      case STATE_OPEN_MESSAGE_THROTTLE_BYTES:
-        {
-          cur_msg_size = current_header.front_len + current_header.middle_len + current_header.data_len;
-          if (cur_msg_size) {
-            if (policy.throttler_bytes) {
-              ldout(async_msgr->cct, 10) << __func__ << " wants " << cur_msg_size << " bytes from policy throttler "
-                                         << policy.throttler_bytes->get_current() << "/"
-                                         << policy.throttler_bytes->get_max() << dendl;
-              if (!policy.throttler_bytes->get_or_fail(cur_msg_size)) {
-                ldout(async_msgr->cct, 10) << __func__ << " wants " << cur_msg_size << " bytes from policy throttler "
-                                           << policy.throttler_bytes->get_current() << "/"
-                                           << policy.throttler_bytes->get_max() << " failed, just wait." << dendl;
-                // following thread pool deal with th full message queue isn't a
-                // short time, so we can wait a ms.
-                if (register_time_events.empty())
-                  register_time_events.insert(center->create_time_event(1000, wakeup_handler));
-                break;
-              }
-            }
-          }
-
-          state = STATE_OPEN_MESSAGE_THROTTLE_DISPATCH_QUEUE;
-          break;
-        }
-
-      case STATE_OPEN_MESSAGE_THROTTLE_DISPATCH_QUEUE:
-        {
-          if (cur_msg_size) {
-            if (!dispatch_queue->dispatch_throttler.get_or_fail(cur_msg_size)) {
-              ldout(async_msgr->cct, 10) << __func__ << " wants " << cur_msg_size << " bytes from dispatch throttle "
-                                         << dispatch_queue->dispatch_throttler.get_current() << "/"
-                                         << dispatch_queue->dispatch_throttler.get_max() << " failed, just wait." << dendl;
-              // following thread pool deal with th full message queue isn't a
-              // short time, so we can wait a ms.
-              if (register_time_events.empty())
-                register_time_events.insert(center->create_time_event(1000, wakeup_handler));
-              break;
-            }
-          }
-
-          throttle_stamp = ceph_clock_now();
-          state = STATE_OPEN_MESSAGE_READ_FRONT;
-          break;
-        }
-
-      case STATE_OPEN_MESSAGE_READ_FRONT:
-        {
-          // read front
-          unsigned front_len = current_header.front_len;
-          if (front_len) {
-            if (!front.length())
-              front.push_back(buffer::create(front_len));
-
-            r = read_until(front_len, front.c_str());
-            if (r < 0) {
-              ldout(async_msgr->cct, 1) << __func__ << " read message front failed" << dendl;
-              goto fail;
-            } else if (r > 0) {
-              break;
-            }
-
-            ldout(async_msgr->cct, 20) << __func__ << " got front " << front.length() << dendl;
-          }
-          state = STATE_OPEN_MESSAGE_READ_MIDDLE;
-        }
-
-      case STATE_OPEN_MESSAGE_READ_MIDDLE:
-        {
-          // read middle
-          unsigned middle_len = current_header.middle_len;
-          if (middle_len) {
-            if (!middle.length())
-              middle.push_back(buffer::create(middle_len));
-
-            r = read_until(middle_len, middle.c_str());
-            if (r < 0) {
-              ldout(async_msgr->cct, 1) << __func__ << " read message middle failed" << dendl;
-              goto fail;
-            } else if (r > 0) {
-              break;
-            }
-            ldout(async_msgr->cct, 20) << __func__ << " got middle " << middle.length() << dendl;
-          }
-
-          state = STATE_OPEN_MESSAGE_READ_DATA_PREPARE;
-        }
-
-      case STATE_OPEN_MESSAGE_READ_DATA_PREPARE:
-        {
-          // read data
-          unsigned data_len = le32_to_cpu(current_header.data_len);
-          unsigned data_off = le32_to_cpu(current_header.data_off);
-          if (data_len) {
-            // get a buffer
-            map<ceph_tid_t,pair<bufferlist,int> >::iterator p = rx_buffers.find(current_header.tid);
-            if (p != rx_buffers.end()) {
-              ldout(async_msgr->cct,10) << __func__ << " seleting rx buffer v " << p->second.second
-                                  << " at offset " << data_off
-                                  << " len " << p->second.first.length() << dendl;
-              data_buf = p->second.first;
-              // make sure it's big enough
-              if (data_buf.length() < data_len)
-                data_buf.push_back(buffer::create(data_len - data_buf.length()));
-              data_blp = data_buf.begin();
-            } else {
-              ldout(async_msgr->cct,20) << __func__ << " allocating new rx buffer at offset " << data_off << dendl;
-              alloc_aligned_buffer(data_buf, data_len, data_off);
-              data_blp = data_buf.begin();
-            }
-          }
-
-          msg_left = data_len;
-          state = STATE_OPEN_MESSAGE_READ_DATA;
-        }
-
-      case STATE_OPEN_MESSAGE_READ_DATA:
-        {
-          while (msg_left > 0) {
-            bufferptr bp = data_blp.get_current_ptr();
-            unsigned read = std::min(bp.length(), msg_left);
-            r = read_until(read, bp.c_str());
-            if (r < 0) {
-              ldout(async_msgr->cct, 1) << __func__ << " read data error " << dendl;
-              goto fail;
-            } else if (r > 0) {
-              break;
-            }
-
-            data_blp.advance(read);
-            data.append(bp, 0, read);
-            msg_left -= read;
-          }
-
-          if (msg_left > 0)
-            break;
-
-          state = STATE_OPEN_MESSAGE_READ_FOOTER_AND_DISPATCH;
-        }
-
-      case STATE_OPEN_MESSAGE_READ_FOOTER_AND_DISPATCH:
-        {
-          ceph_msg_footer footer;
-          ceph_msg_footer_old old_footer;
-          unsigned len;
-          // footer
-          if (has_feature(CEPH_FEATURE_MSG_AUTH))
-            len = sizeof(footer);
-          else
-            len = sizeof(old_footer);
-
-          r = read_until(len, state_buffer);
-          if (r < 0) {
-            ldout(async_msgr->cct, 1) << __func__ << " read footer data error " << dendl;
-            goto fail;
-          } else if (r > 0) {
-            break;
-          }
-
-          if (has_feature(CEPH_FEATURE_MSG_AUTH)) {
-            footer = *((ceph_msg_footer*)state_buffer);
-          } else {
-            old_footer = *((ceph_msg_footer_old*)state_buffer);
-            footer.front_crc = old_footer.front_crc;
-            footer.middle_crc = old_footer.middle_crc;
-            footer.data_crc = old_footer.data_crc;
-            footer.sig = 0;
-            footer.flags = old_footer.flags;
-          }
-          int aborted = (footer.flags & CEPH_MSG_FOOTER_COMPLETE) == 0;
-          ldout(async_msgr->cct, 10) << __func__ << " aborted = " << aborted << dendl;
-          if (aborted) {
-            ldout(async_msgr->cct, 0) << __func__ << " got " << front.length() << " + " << middle.length() << " + " << data.length()
-                                << " byte message.. ABORTED" << dendl;
-            goto fail;
-          }
-
-          ldout(async_msgr->cct, 20) << __func__ << " got " << front.length() << " + " << middle.length()
-                              << " + " << data.length() << " byte message" << dendl;
-          Message *message = decode_message(async_msgr->cct, async_msgr->crcflags, current_header, footer,
-                                            front, middle, data, this);
-          if (!message) {
-            ldout(async_msgr->cct, 1) << __func__ << " decode message failed " << dendl;
-            goto fail;
-          }
-
-          //
-          //  Check the signature if one should be present.  A zero return indicates success. PLR
-          //
-
-          if (session_security.get() == NULL) {
-            ldout(async_msgr->cct, 10) << __func__ << " no session security set" << dendl;
-          } else {
-            if (session_security->check_message_signature(message)) {
-              ldout(async_msgr->cct, 0) << __func__ << " Signature check failed" << dendl;
-              message->put();
-              goto fail;
-            }
-          }
-          message->set_byte_throttler(policy.throttler_bytes);
-          message->set_message_throttler(policy.throttler_messages);
-
-          // store reservation size in message, so we don't get confused
-          // by messages entering the dispatch queue through other paths.
-          message->set_dispatch_throttle_size(cur_msg_size);
-
-          message->set_recv_stamp(recv_stamp);
-          message->set_throttle_stamp(throttle_stamp);
-          message->set_recv_complete_stamp(ceph_clock_now());
-
-          // check received seq#.  if it is old, drop the message.  
-          // note that incoming messages may skip ahead.  this is convenient for the client
-          // side queueing because messages can't be renumbered, but the (kernel) client will
-          // occasionally pull a message out of the sent queue to send elsewhere.  in that case
-          // it doesn't matter if we "got" it or not.
-          uint64_t cur_seq = in_seq;
-          if (message->get_seq() <= cur_seq) {
-            ldout(async_msgr->cct,0) << __func__ << " got old message "
-                    << message->get_seq() << " <= " << cur_seq << " " << message << " " << *message
-                    << ", discarding" << dendl;
-            message->put();
-            if (has_feature(CEPH_FEATURE_RECONNECT_SEQ) && async_msgr->cct->_conf->ms_die_on_old_message)
-              ceph_abort_msg("old msgs despite reconnect_seq feature");
-            break;
-          }
-          if (message->get_seq() > cur_seq + 1) {
-            ldout(async_msgr->cct, 0) << __func__ << " missed message?  skipped from seq "
-                                      << cur_seq << " to " << message->get_seq() << dendl;
-            if (async_msgr->cct->_conf->ms_die_on_skipped_message)
-              ceph_abort_msg("skipped incoming seq");
-          }
-
-          message->set_connection(this);
-
-#if defined(WITH_LTTNG) && defined(WITH_EVENTTRACE)
-          if (message->get_type() == CEPH_MSG_OSD_OP || message->get_type() == CEPH_MSG_OSD_OPREPLY) {
-            utime_t ltt_processed_stamp = ceph_clock_now();
-            double usecs_elapsed = (ltt_processed_stamp.to_nsec()-ltt_recv_stamp.to_nsec())/1000;
-            ostringstream buf;
-            if (message->get_type() == CEPH_MSG_OSD_OP)
-              OID_ELAPSED_WITH_MSG(message, usecs_elapsed, "TIME_TO_DECODE_OSD_OP", false);
-            else
-              OID_ELAPSED_WITH_MSG(message, usecs_elapsed, "TIME_TO_DECODE_OSD_OPREPLY", false);
-          }
-#endif
-
-          // note last received message.
-          in_seq = message->get_seq();
-         ldout(async_msgr->cct, 5) << " rx " << message->get_source() << " seq "
-                                    << message->get_seq() << " " << message
-                                   << " " << *message << dendl;
-
-          if (!policy.lossy) {
-            ack_left++;
-            need_dispatch_writer = true;
-          }
-          state = STATE_OPEN;
-
-          logger->inc(l_msgr_recv_messages);
-          logger->inc(l_msgr_recv_bytes, cur_msg_size + sizeof(ceph_msg_header) + sizeof(ceph_msg_footer));
-
-          async_msgr->ms_fast_preprocess(message);
-          auto fast_dispatch_time = ceph::mono_clock::now();
-          logger->tinc(l_msgr_running_recv_time, fast_dispatch_time - recv_start_time);
-          if (delay_state) {
-            double delay_period = 0;
-            if (rand() % 10000 < async_msgr->cct->_conf->ms_inject_delay_probability * 10000.0) {
-              delay_period = async_msgr->cct->_conf->ms_inject_delay_max * (double)(rand() % 10000) / 10000.0;
-              ldout(async_msgr->cct, 1) << "queue_received will delay after " << (ceph_clock_now() + delay_period)
-                                       << " on " << message << " " << *message << dendl;
-            }
-            delay_state->queue(delay_period, message);
-          } else if (async_msgr->ms_can_fast_dispatch(message)) {
-            lock.unlock();
-            dispatch_queue->fast_dispatch(message);
-            recv_start_time = ceph::mono_clock::now();
-            logger->tinc(l_msgr_running_fast_dispatch_time,
-                         recv_start_time - fast_dispatch_time);
-            lock.lock();
-          } else {
-            dispatch_queue->enqueue(message, message->get_priority(), conn_id);
-          }
-
-         // clean up local buffer references
-          data_buf.clear();
-          front.clear();
-          middle.clear();
-          data.clear();
-
-          break;
-        }
-
-      case STATE_OPEN_TAG_CLOSE:
-        {
-          ldout(async_msgr->cct, 20) << __func__ << " got CLOSE" << dendl;
-          _stop();
-          return ;
-        }
-
-      case STATE_STANDBY:
-        {
-          ldout(async_msgr->cct, 20) << __func__ << " enter STANDY" << dendl;
-
-          break;
-        }
-
-      case STATE_NONE:
-        {
-          ldout(async_msgr->cct, 20) << __func__ << " enter none state" << dendl;
-          break;
-        }
-
-      case STATE_CLOSED:
-        {
-          ldout(async_msgr->cct, 20) << __func__ << " socket closed" << dendl;
-          break;
-        }
-
-      case STATE_WAIT:
-        {
-          ldout(async_msgr->cct, 1) << __func__ << " enter wait state, failing" << dendl;
-          goto fail;
-        }
-
-      default:
-        {
-          if (_process_connection() < 0)
-            goto fail;
-          break;
-        }
+  ssize_t nread;
+ again:
+  nread = cs.read(buf, len);
+  if (nread < 0) {
+    if (nread == -EAGAIN) {
+      nread = 0;
+    } else if (nread == -EINTR) {
+      goto again;
+    } else {
+      ldout(async_msgr->cct, 1) << __func__ << " reading from fd=" << cs.fd()
+                          << " : "<< strerror(nread) << dendl;
+      return -1;
     }
-  } while (prev_state != state);
-
-  if (need_dispatch_writer && is_connected())
-    center->dispatch_event_external(write_handler);
-
-  logger->tinc(l_msgr_running_recv_time, ceph::mono_clock::now() - recv_start_time);
-  return;
-
- fail:
-  fault();
-}
-
-ssize_t AsyncConnection::_process_connection()
-{
-  ssize_t r = 0;
-
-  switch(state) {
-    case STATE_WAIT_SEND:
-      {
-        std::lock_guard<std::mutex> l(write_lock);
-        if (!outcoming_bl.length()) {
-          ceph_assert(state_after_send);
-          state = state_after_send;
-          state_after_send = STATE_NONE;
-        }
-        break;
-      }
-
-    case STATE_CONNECTING:
-      {
-        ceph_assert(!policy.server);
-
-        // reset connect state variables
-        got_bad_auth = false;
-        delete authorizer;
-        authorizer = NULL;
-        authorizer_buf.clear();
-        memset(&connect_msg, 0, sizeof(connect_msg));
-        memset(&connect_reply, 0, sizeof(connect_reply));
-
-        global_seq = async_msgr->get_global_seq();
-        // close old socket.  this is safe because we stopped the reader thread above.
-        if (cs) {
-          center->delete_file_event(cs.fd(), EVENT_READABLE|EVENT_WRITABLE);
-          cs.close();
-        }
-
-        SocketOptions opts;
-        opts.priority = async_msgr->get_socket_priority();
-        opts.connect_bind_addr = msgr->get_myaddr();
-        r = worker->connect(get_peer_addr(), opts, &cs);
-        if (r < 0)
-          goto fail;
-
-        center->create_file_event(cs.fd(), EVENT_READABLE, read_handler);
-        state = STATE_CONNECTING_RE;
-        break;
-      }
-
-    case STATE_CONNECTING_RE:
-      {
-        r = cs.is_connected();
-        if (r < 0) {
-          ldout(async_msgr->cct, 1) << __func__ << " reconnect failed " << dendl;
-          if (r == -ECONNREFUSED) {
-            ldout(async_msgr->cct, 2) << __func__ << " connection refused!" << dendl;
-            dispatch_queue->queue_refused(this);
-          }
-          goto fail;
-        } else if (r == 0) {
-          ldout(async_msgr->cct, 10) << __func__ << " nonblock connect inprogress" << dendl;
-          if (async_msgr->get_stack()->nonblock_connect_need_writable_event())
-            center->create_file_event(cs.fd(), EVENT_WRITABLE, read_handler);
-          break;
-        }
-
-        center->delete_file_event(cs.fd(), EVENT_WRITABLE);
-        ldout(async_msgr->cct, 10) << __func__ << " connect successfully, ready to send banner" << dendl;
-
-        bufferlist bl;
-        bl.append(CEPH_BANNER, strlen(CEPH_BANNER));
-        r = try_send(bl);
-        if (r == 0) {
-          state = STATE_CONNECTING_WAIT_BANNER_AND_IDENTIFY;
-          ldout(async_msgr->cct, 10) << __func__ << " connect write banner done: "
-                                     << get_peer_addr() << dendl;
-        } else if (r > 0) {
-          state = STATE_WAIT_SEND;
-          state_after_send = STATE_CONNECTING_WAIT_BANNER_AND_IDENTIFY;
-          ldout(async_msgr->cct, 10) << __func__ << " connect wait for write banner: "
-                               << get_peer_addr() << dendl;
-        } else {
-          goto fail;
-        }
-
-        break;
-      }
-
-    case STATE_CONNECTING_WAIT_BANNER_AND_IDENTIFY:
-      {
-        entity_addr_t paddr, peer_addr_for_me;
-        bufferlist myaddrbl;
-        unsigned banner_len = strlen(CEPH_BANNER);
-        unsigned need_len = banner_len + sizeof(ceph_entity_addr)*2;
-        r = read_until(need_len, state_buffer);
-        if (r < 0) {
-          ldout(async_msgr->cct, 1) << __func__ << " read banner and identify addresses failed" << dendl;
-          goto fail;
-        } else if (r > 0) {
-          break;
-        }
-
-        if (memcmp(state_buffer, CEPH_BANNER, banner_len)) {
-          ldout(async_msgr->cct, 0) << __func__ << " connect protocol error (bad banner) on peer "
-                                    << get_peer_addr() << dendl;
-          goto fail;
-        }
-
-        bufferlist bl;
-        bl.append(state_buffer+banner_len, sizeof(ceph_entity_addr)*2);
-        auto p = bl.cbegin();
-        try {
-          decode(paddr, p);
-          decode(peer_addr_for_me, p);
-        } catch (const buffer::error& e) {
-          lderr(async_msgr->cct) << __func__ <<  " decode peer addr failed " << dendl;
-          goto fail;
-        }
-        ldout(async_msgr->cct, 20) << __func__ <<  " connect read peer addr "
-                             << paddr << " on socket " << cs.fd() << dendl;
-        if (peer_addrs.legacy_addr() != paddr) {
-          if (paddr.is_blank_ip() &&
-             peer_addrs.legacy_addr().get_port() == paddr.get_port() &&
-              peer_addrs.legacy_addr().get_nonce() == paddr.get_nonce()) {
-            ldout(async_msgr->cct, 0) << __func__ <<  " connect claims to be " << paddr
-                                << " not " << peer_addrs
-                                << " - presumably this is the same node!" << dendl;
-          } else {
-            ldout(async_msgr->cct, 10) << __func__ << " connect claims to be "
-                                      << paddr << " not " << peer_addrs << dendl;
-           goto fail;
-          }
-        }
-
-        ldout(async_msgr->cct, 20) << __func__ << " connect peer addr for me is " << peer_addr_for_me << dendl;
-        lock.unlock();
-        async_msgr->learned_addr(peer_addr_for_me);
-        if (async_msgr->cct->_conf->ms_inject_internal_delays
-            && async_msgr->cct->_conf->ms_inject_socket_failures) {
-          if (rand() % async_msgr->cct->_conf->ms_inject_socket_failures == 0) {
-            ldout(msgr->cct, 10) << __func__ << " sleep for "
-                                 << async_msgr->cct->_conf->ms_inject_internal_delays << dendl;
-            utime_t t;
-            t.set_from_double(async_msgr->cct->_conf->ms_inject_internal_delays);
-            t.sleep();
-          }
-        }
-
-        lock.lock();
-        if (state != STATE_CONNECTING_WAIT_BANNER_AND_IDENTIFY) {
-          ldout(async_msgr->cct, 1) << __func__ << " state changed while learned_addr, mark_down or "
-                                    << " replacing must be happened just now" << dendl;
-          return 0;
-        }
-
-        encode(async_msgr->get_myaddrs().legacy_addr(), myaddrbl, 0); // legacy
-        r = try_send(myaddrbl);
-        if (r == 0) {
-          state = STATE_CONNECTING_SEND_CONNECT_MSG;
-          ldout(async_msgr->cct, 10) << __func__ << " connect sent my addr "
-                                    << async_msgr->get_myaddrs().legacy_addr()
-                                    << dendl;
-        } else if (r > 0) {
-          state = STATE_WAIT_SEND;
-          state_after_send = STATE_CONNECTING_SEND_CONNECT_MSG;
-          ldout(async_msgr->cct, 10) << __func__ << " connect send my addr done: "
-                                    << async_msgr->get_myaddrs().legacy_addr()
-                                    << dendl;
-        } else {
-          ldout(async_msgr->cct, 2) << __func__ << " connect couldn't write my addr, "
-              << cpp_strerror(r) << dendl;
-          goto fail;
-        }
-
-        break;
-      }
-
-    case STATE_CONNECTING_SEND_CONNECT_MSG:
-      {
-        if (!authorizer) {
-          authorizer = async_msgr->get_authorizer(peer_type, false);
-        }
-        bufferlist bl;
-
-        connect_msg.features = policy.features_supported;
-        connect_msg.host_type = async_msgr->get_myname().type();
-        connect_msg.global_seq = global_seq;
-        connect_msg.connect_seq = connect_seq;
-        connect_msg.protocol_version = async_msgr->get_proto_version(peer_type, true);
-        connect_msg.authorizer_protocol = authorizer ? authorizer->protocol : 0;
-        connect_msg.authorizer_len = authorizer ? authorizer->bl.length() : 0;
-        if (authorizer)
-          ldout(async_msgr->cct, 10) << __func__ <<  " connect_msg.authorizer_len="
-                                     << connect_msg.authorizer_len << " protocol="
-                                     << connect_msg.authorizer_protocol << dendl;
-        connect_msg.flags = 0;
-        if (policy.lossy)
-          connect_msg.flags |= CEPH_MSG_CONNECT_LOSSY;  // this is fyi, actually, server decides!
-        bl.append((char*)&connect_msg, sizeof(connect_msg));
-        if (authorizer) {
-          bl.append(authorizer->bl.c_str(), authorizer->bl.length());
-        }
-        ldout(async_msgr->cct, 10) << __func__ << " connect sending gseq=" << global_seq << " cseq="
-            << connect_seq << " proto=" << connect_msg.protocol_version << dendl;
-
-        r = try_send(bl);
-        if (r == 0) {
-          state = STATE_CONNECTING_WAIT_CONNECT_REPLY;
-          ldout(async_msgr->cct,20) << __func__ << " connect wrote (self +) cseq, waiting for reply" << dendl;
-        } else if (r > 0) {
-          state = STATE_WAIT_SEND;
-          state_after_send = STATE_CONNECTING_WAIT_CONNECT_REPLY;
-          ldout(async_msgr->cct, 10) << __func__ << " continue send reply " << dendl;
-        } else {
-          ldout(async_msgr->cct, 2) << __func__ << " connect couldn't send reply "
-              << cpp_strerror(r) << dendl;
-          goto fail;
-        }
-
-        break;
-      }
-
-    case STATE_CONNECTING_WAIT_CONNECT_REPLY:
-      {
-        r = read_until(sizeof(connect_reply), state_buffer);
-        if (r < 0) {
-          ldout(async_msgr->cct, 1) << __func__ << " read connect reply failed" << dendl;
-          goto fail;
-        } else if (r > 0) {
-          break;
-        }
-
-        connect_reply = *((ceph_msg_connect_reply*)state_buffer);
-
-        ldout(async_msgr->cct, 20) << __func__ << " connect got reply tag " << (int)connect_reply.tag
-                             << " connect_seq " << connect_reply.connect_seq << " global_seq "
-                             << connect_reply.global_seq << " proto " << connect_reply.protocol_version
-                             << " flags " << (int)connect_reply.flags << " features "
-                             << connect_reply.features << dendl;
-        state = STATE_CONNECTING_WAIT_CONNECT_REPLY_AUTH;
-
-        break;
-      }
-
-    case STATE_CONNECTING_WAIT_CONNECT_REPLY_AUTH:
-      {
-        bufferlist authorizer_reply;
-        if (connect_reply.authorizer_len) {
-          ldout(async_msgr->cct, 10) << __func__ << " reply.authorizer_len=" << connect_reply.authorizer_len << dendl;
-          ceph_assert(connect_reply.authorizer_len < 4096);
-          r = read_until(connect_reply.authorizer_len, state_buffer);
-          if (r < 0) {
-            ldout(async_msgr->cct, 1) << __func__ << " read connect reply authorizer failed" << dendl;
-            goto fail;
-          } else if (r > 0) {
-            break;
-          }
-
-          authorizer_reply.append(state_buffer, connect_reply.authorizer_len);
-
-         if (connect_reply.tag == CEPH_MSGR_TAG_CHALLENGE_AUTHORIZER) {
-           ldout(async_msgr->cct,10) << __func__ << " connect got auth challenge" << dendl;
-           authorizer->add_challenge(async_msgr->cct, authorizer_reply);
-           state = STATE_CONNECTING_SEND_CONNECT_MSG;
-           break;
-         }
-
-          auto iter = authorizer_reply.cbegin();
-          if (authorizer && !authorizer->verify_reply(iter)) {
-            ldout(async_msgr->cct, 0) << __func__ << " failed verifying authorize reply" << dendl;
-            goto fail;
-          }
-        }
-        r = handle_connect_reply(connect_msg, connect_reply);
-        if (r < 0)
-          goto fail;
-
-        // state must be changed!
-        ceph_assert(state != STATE_CONNECTING_WAIT_CONNECT_REPLY_AUTH);
-        break;
-      }
-
-    case STATE_CONNECTING_WAIT_ACK_SEQ:
-      {
-        uint64_t newly_acked_seq = 0;
-
-        r = read_until(sizeof(newly_acked_seq), state_buffer);
-        if (r < 0) {
-          ldout(async_msgr->cct, 1) << __func__ << " read connect ack seq failed" << dendl;
-          goto fail;
-        } else if (r > 0) {
-          break;
-        }
-
-        newly_acked_seq = *((uint64_t*)state_buffer);
-        ldout(async_msgr->cct, 2) << __func__ << " got newly_acked_seq " << newly_acked_seq
-                            << " vs out_seq " << out_seq << dendl;
-        discard_requeued_up_to(newly_acked_seq);
-        //while (newly_acked_seq > out_seq.read()) {
-        //  Message *m = _get_next_outgoing(NULL);
-        //  ceph_assert(m);
-        //  ldout(async_msgr->cct, 2) << __func__ << " discarding previously sent " << m->get_seq()
-        //                      << " " << *m << dendl;
-        //  ceph_assert(m->get_seq() <= newly_acked_seq);
-        //  m->put();
-        //  out_seq.inc();
-        //}
-
-        bufferlist bl;
-        uint64_t s = in_seq;
-        bl.append((char*)&s, sizeof(s));
-        r = try_send(bl);
-        if (r == 0) {
-          state = STATE_CONNECTING_READY;
-          ldout(async_msgr->cct, 10) << __func__ << " send in_seq done " << dendl;
-        } else if (r > 0) {
-          state_after_send = STATE_CONNECTING_READY;
-          state = STATE_WAIT_SEND;
-          ldout(async_msgr->cct, 10) << __func__ << " continue send in_seq " << dendl;
-        } else {
-          goto fail;
-        }
-        break;
-      }
-
-    case STATE_CONNECTING_READY:
-      {
-        // hooray!
-        peer_global_seq = connect_reply.global_seq;
-        policy.lossy = connect_reply.flags & CEPH_MSG_CONNECT_LOSSY;
-        state = STATE_OPEN;
-        once_ready = true;
-        connect_seq += 1;
-        ceph_assert(connect_seq == connect_reply.connect_seq);
-        backoff = utime_t();
-        set_features((uint64_t)connect_reply.features & (uint64_t)connect_msg.features);
-        ldout(async_msgr->cct, 10) << __func__ << " connect success " << connect_seq
-                                   << ", lossy = " << policy.lossy << ", features "
-                                   << get_features() << dendl;
-
-        // If we have an authorizer, get a new AuthSessionHandler to deal with ongoing security of the
-        // connection.  PLR
-        if (authorizer != NULL) {
-          session_security.reset(
-              get_auth_session_handler(async_msgr->cct,
-                                       authorizer->protocol,
-                                       authorizer->session_key,
-                                       get_features()));
-        } else {
-          // We have no authorizer, so we shouldn't be applying security to messages in this AsyncConnection.  PLR
-          session_security.reset();
-        }
-
-        if (delay_state)
-          ceph_assert(delay_state->ready());
-        dispatch_queue->queue_connect(this);
-        async_msgr->ms_deliver_handle_fast_connect(this);
-
-        // make sure no pending tick timer
-        if (last_tick_id)
-          center->delete_time_event(last_tick_id);
-        last_tick_id = center->create_time_event(inactive_timeout_us, tick_handler);
-
-        // message may in queue between last _try_send and connection ready
-        // write event may already notify and we need to force scheduler again
-        write_lock.lock();
-        can_write = WriteStatus::CANWRITE;
-        if (is_queued())
-          center->dispatch_event_external(write_handler);
-        write_lock.unlock();
-        maybe_start_delay_thread();
-        break;
-      }
-
-    case STATE_ACCEPTING:
-      {
-        bufferlist bl;
-        center->create_file_event(cs.fd(), EVENT_READABLE, read_handler);
-
-        bl.append(CEPH_BANNER, strlen(CEPH_BANNER));
-
-       auto legacy = async_msgr->get_myaddrs().legacy_addr();
-        encode(legacy, bl, 0); // legacy
-        port = legacy.get_port();
-        encode(socket_addr, bl, 0); // legacy
-        ldout(async_msgr->cct, 1) << __func__ << " sd=" << cs.fd() << " " << socket_addr << dendl;
-
-        r = try_send(bl);
-        if (r == 0) {
-          state = STATE_ACCEPTING_WAIT_BANNER_ADDR;
-          ldout(async_msgr->cct, 10) << __func__ << " write banner and addr done: "
-            << get_peer_addr() << dendl;
-        } else if (r > 0) {
-          state = STATE_WAIT_SEND;
-          state_after_send = STATE_ACCEPTING_WAIT_BANNER_ADDR;
-          ldout(async_msgr->cct, 10) << __func__ << " wait for write banner and addr: "
-                              << get_peer_addr() << dendl;
-        } else {
-          goto fail;
-        }
-
-        break;
-      }
-    case STATE_ACCEPTING_WAIT_BANNER_ADDR:
-      {
-        bufferlist addr_bl;
-        entity_addr_t peer_addr;
-
-        r = read_until(strlen(CEPH_BANNER) + sizeof(ceph_entity_addr), state_buffer);
-        if (r < 0) {
-          ldout(async_msgr->cct, 1) << __func__ << " read peer banner and addr failed" << dendl;
-          goto fail;
-        } else if (r > 0) {
-          break;
-        }
-
-        if (memcmp(state_buffer, CEPH_BANNER, strlen(CEPH_BANNER))) {
-          ldout(async_msgr->cct, 1) << __func__ << " accept peer sent bad banner '" << state_buffer
-                                    << "' (should be '" << CEPH_BANNER << "')" << dendl;
-          goto fail;
-        }
-
-        addr_bl.append(state_buffer+strlen(CEPH_BANNER), sizeof(ceph_entity_addr));
-        try {
-          auto ti = addr_bl.cbegin();
-          decode(peer_addr, ti);
-        } catch (const buffer::error& e) {
-         lderr(async_msgr->cct) << __func__ <<  " decode peer_addr failed " << dendl;
-          goto fail;
-       }
-
-        ldout(async_msgr->cct, 10) << __func__ << " accept peer addr is " << peer_addr << dendl;
-        if (peer_addr.is_blank_ip()) {
-          // peer apparently doesn't know what ip they have; figure it out for them.
-          int port = peer_addr.get_port();
-          peer_addr.u = socket_addr.u;
-          peer_addr.set_port(port);
-          ldout(async_msgr->cct, 0) << __func__ << " accept peer addr is really " << peer_addr
-                             << " (socket is " << socket_addr << ")" << dendl;
-        }
-        set_peer_addr(peer_addr);  // so that connection_state gets set up
-       target_addr = peer_addr;
-        state = STATE_ACCEPTING_WAIT_CONNECT_MSG;
-        break;
-      }
-
-    case STATE_ACCEPTING_WAIT_CONNECT_MSG:
-      {
-        r = read_until(sizeof(connect_msg), state_buffer);
-        if (r < 0) {
-          ldout(async_msgr->cct, 1) << __func__ << " read connect msg failed" << dendl;
-          goto fail;
-        } else if (r > 0) {
-          break;
-        }
-
-        connect_msg = *((ceph_msg_connect*)state_buffer);
-        state = STATE_ACCEPTING_WAIT_CONNECT_MSG_AUTH;
-        break;
-      }
-
-    case STATE_ACCEPTING_WAIT_CONNECT_MSG_AUTH:
-      {
-        bufferlist authorizer_reply;
-
-        if (connect_msg.authorizer_len) {
-          if (!authorizer_buf.length())
-            authorizer_buf.push_back(buffer::create(connect_msg.authorizer_len));
-
-          r = read_until(connect_msg.authorizer_len, authorizer_buf.c_str());
-          if (r < 0) {
-            ldout(async_msgr->cct, 1) << __func__ << " read connect authorizer failed" << dendl;
-            goto fail;
-          } else if (r > 0) {
-            break;
-          }
-        }
-
-        ldout(async_msgr->cct, 20) << __func__ << " accept got peer connect_seq "
-                             << connect_msg.connect_seq << " global_seq "
-                             << connect_msg.global_seq << dendl;
-        set_peer_type(connect_msg.host_type);
-        policy = async_msgr->get_policy(connect_msg.host_type);
-        ldout(async_msgr->cct, 10) << __func__ << " accept of host_type " << connect_msg.host_type
-                                   << ", policy.lossy=" << policy.lossy << " policy.server="
-                                   << policy.server << " policy.standby=" << policy.standby
-                                   << " policy.resetcheck=" << policy.resetcheck << dendl;
-
-        r = handle_connect_msg(connect_msg, authorizer_buf, authorizer_reply);
-        if (r < 0)
-          goto fail;
-
-        // state is changed by "handle_connect_msg"
-        ceph_assert(state != STATE_ACCEPTING_WAIT_CONNECT_MSG_AUTH);
-        break;
-      }
-
-    case STATE_ACCEPTING_WAIT_SEQ:
-      {
-        uint64_t newly_acked_seq;
-        r = read_until(sizeof(newly_acked_seq), state_buffer);
-        if (r < 0) {
-          ldout(async_msgr->cct, 1) << __func__ << " read ack seq failed" << dendl;
-          goto fail_registered;
-        } else if (r > 0) {
-          break;
-        }
-
-        newly_acked_seq = *((uint64_t*)state_buffer);
-        ldout(async_msgr->cct, 2) << __func__ << " accept get newly_acked_seq " << newly_acked_seq << dendl;
-        discard_requeued_up_to(newly_acked_seq);
-        state = STATE_ACCEPTING_READY;
-        break;
-      }
-
-    case STATE_ACCEPTING_READY:
-      {
-        ldout(async_msgr->cct, 20) << __func__ << " accept done" << dendl;
-        state = STATE_OPEN;
-        memset(&connect_msg, 0, sizeof(connect_msg));
-
-        if (delay_state)
-          ceph_assert(delay_state->ready());
-        // make sure no pending tick timer
-        if (last_tick_id)
-          center->delete_time_event(last_tick_id);
-        last_tick_id = center->create_time_event(inactive_timeout_us, tick_handler);
-
-        write_lock.lock();
-        can_write = WriteStatus::CANWRITE;
-        if (is_queued())
-          center->dispatch_event_external(write_handler);
-        write_lock.unlock();
-        maybe_start_delay_thread();
-        break;
-      }
-
-    default:
-      {
-        lderr(async_msgr->cct) << __func__ << " bad state: " << state << dendl;
-        ceph_abort();
-      }
+  } else if (nread == 0) {
+    ldout(async_msgr->cct, 1) << __func__ << " peer close file descriptor "
+                              << cs.fd() << dendl;
+    return -1;
   }
-
-  return 0;
-
-fail_registered:
-  ldout(async_msgr->cct, 10) << "accept fault after register" << dendl;
-  inject_delay();
-
-fail:
-  return -1;
+  return nread;
 }
 
-int AsyncConnection::handle_connect_reply(ceph_msg_connect &connect, ceph_msg_connect_reply &reply)
-{
-  uint64_t feat_missing;
-  if (reply.tag == CEPH_MSGR_TAG_FEATURES) {
-    ldout(async_msgr->cct, 0) << __func__ << " connect protocol feature mismatch, my "
-                        << std::hex << connect.features << " < peer "
-                        << reply.features << " missing "
-                        << (reply.features & ~policy.features_supported)
-                        << std::dec << dendl;
-    goto fail;
-  }
-
-  if (reply.tag == CEPH_MSGR_TAG_BADPROTOVER) {
-    ldout(async_msgr->cct, 0) << __func__ << " connect protocol version mismatch, my "
-                        << connect.protocol_version << " != " << reply.protocol_version
-                        << dendl;
-    goto fail;
-  }
-
-  if (reply.tag == CEPH_MSGR_TAG_BADAUTHORIZER) {
-    ldout(async_msgr->cct,0) << __func__ << " connect got BADAUTHORIZER" << dendl;
-    if (got_bad_auth)
-      goto fail;
-    got_bad_auth = true;
-    delete authorizer;
-    authorizer = async_msgr->get_authorizer(peer_type, true);  // try harder
-    state = STATE_CONNECTING_SEND_CONNECT_MSG;
-  }
-  if (reply.tag == CEPH_MSGR_TAG_RESETSESSION) {
-    ldout(async_msgr->cct, 0) << __func__ << " connect got RESETSESSION" << dendl;
-    was_session_reset();
-    // see was_session_reset
-    outcoming_bl.clear();
-    state = STATE_CONNECTING_SEND_CONNECT_MSG;
-  }
-  if (reply.tag == CEPH_MSGR_TAG_RETRY_GLOBAL) {
-    global_seq = async_msgr->get_global_seq(reply.global_seq);
-    ldout(async_msgr->cct, 5) << __func__ << " connect got RETRY_GLOBAL "
-                              << reply.global_seq << " chose new "
-                              << global_seq << dendl;
-    state = STATE_CONNECTING_SEND_CONNECT_MSG;
-  }
-  if (reply.tag == CEPH_MSGR_TAG_RETRY_SESSION) {
-    ceph_assert(reply.connect_seq > connect_seq);
-    ldout(async_msgr->cct, 5) << __func__ << " connect got RETRY_SESSION "
-                              << connect_seq << " -> "
-                              << reply.connect_seq << dendl;
-    connect_seq = reply.connect_seq;
-    state = STATE_CONNECTING_SEND_CONNECT_MSG;
-  }
-  if (reply.tag == CEPH_MSGR_TAG_WAIT) {
-    ldout(async_msgr->cct, 1) << __func__ << " connect got WAIT (connection race)" << dendl;
-    state = STATE_WAIT;
-  }
-
-  feat_missing = policy.features_required & ~(uint64_t)connect_reply.features;
-  if (feat_missing) {
-    ldout(async_msgr->cct, 1) << __func__ << " missing required features " << std::hex
-                              << feat_missing << std::dec << dendl;
-    goto fail;
-  }
-
-  if (reply.tag == CEPH_MSGR_TAG_SEQ) {
-    ldout(async_msgr->cct, 10) << __func__ << " got CEPH_MSGR_TAG_SEQ, reading acked_seq and writing in_seq" << dendl;
-    state = STATE_CONNECTING_WAIT_ACK_SEQ;
-  }
-  if (reply.tag == CEPH_MSGR_TAG_READY) {
-    ldout(async_msgr->cct, 10) << __func__ << " got CEPH_MSGR_TAG_READY " << dendl;
-    state = STATE_CONNECTING_READY;
-  }
-
-  return 0;
+void AsyncConnection::write(bufferlist &bl,
+                            std::function<void(ssize_t)> callback, bool more) {
 
- fail:
-  return -1;
+    std::unique_lock<std::mutex> l(write_lock);
+    writeCallback = callback;
+    outcoming_bl.claim_append(bl);
+    ssize_t r = _try_send(more);
+    if (r <= 0) {
+      // either finish writting, or returned an error
+      writeCallback.reset();
+      l.unlock();
+      callback(r);
+      return;
+    }
 }
 
-ssize_t AsyncConnection::handle_connect_msg(ceph_msg_connect &connect, bufferlist &authorizer_bl,
-                                            bufferlist &authorizer_reply)
+// return the remaining bytes, it may larger than the length of ptr
+// else return < 0 means error
+ssize_t AsyncConnection::_try_send(bool more)
 {
-  ssize_t r = 0;
-  ceph_msg_connect_reply reply;
-  bufferlist reply_bl;
-
-  memset(&reply, 0, sizeof(reply));
-  reply.protocol_version = async_msgr->get_proto_version(peer_type, false);
-
-  // mismatch?
-  ldout(async_msgr->cct, 10) << __func__ << " accept my proto " << reply.protocol_version
-                      << ", their proto " << connect.protocol_version << dendl;
-  if (connect.protocol_version != reply.protocol_version) {
-    return _reply_accept(CEPH_MSGR_TAG_BADPROTOVER, connect, reply, authorizer_reply);
-  }
-  // require signatures for cephx?
-  if (connect.authorizer_protocol == CEPH_AUTH_CEPHX) {
-    if (peer_type == CEPH_ENTITY_TYPE_OSD ||
-        peer_type == CEPH_ENTITY_TYPE_MDS ||
-       peer_type == CEPH_ENTITY_TYPE_MGR) {
-      if (async_msgr->cct->_conf->cephx_require_signatures ||
-          async_msgr->cct->_conf->cephx_cluster_require_signatures) {
-        ldout(async_msgr->cct, 10) << __func__ << " using cephx, requiring MSG_AUTH feature bit for cluster" << dendl;
-        policy.features_required |= CEPH_FEATURE_MSG_AUTH;
-      }
-      if (async_msgr->cct->_conf->cephx_require_version >= 2 ||
-         async_msgr->cct->_conf->cephx_cluster_require_version >= 2) {
-        ldout(async_msgr->cct, 10) << __func__ << " using cephx, requiring cephx v2 feature bit for cluster" << dendl;
-        policy.features_required |= CEPH_FEATUREMASK_CEPHX_V2;
-      }
-    } else {
-      if (async_msgr->cct->_conf->cephx_require_signatures ||
-          async_msgr->cct->_conf->cephx_service_require_signatures) {
-        ldout(async_msgr->cct, 10) << __func__ << " using cephx, requiring MSG_AUTH feature bit for service" << dendl;
-        policy.features_required |= CEPH_FEATURE_MSG_AUTH;
-      }
-      if (async_msgr->cct->_conf->cephx_require_version >= 2 ||
-         async_msgr->cct->_conf->cephx_service_require_version >= 2) {
-        ldout(async_msgr->cct, 10) << __func__ << " using cephx, requiring cephx v2 feature bit for service" << dendl;
-        policy.features_required |= CEPH_FEATUREMASK_CEPHX_V2;
-      }
+  if (async_msgr->cct->_conf->ms_inject_socket_failures && cs) {
+    if (rand() % async_msgr->cct->_conf->ms_inject_socket_failures == 0) {
+      ldout(async_msgr->cct, 0) << __func__ << " injecting socket failure" << dendl;
+      cs.shutdown();
     }
   }
 
-  uint64_t feat_missing = policy.features_required & ~(uint64_t)connect.features;
-  if (feat_missing) {
-    ldout(async_msgr->cct, 1) << __func__ << " peer missing required features "
-                        << std::hex << feat_missing << std::dec << dendl;
-    return _reply_accept(CEPH_MSGR_TAG_FEATURES, connect, reply, authorizer_reply);
+  ceph_assert(center->in_thread());
+  ldout(async_msgr->cct, 25) << __func__ << " cs.send " << outcoming_bl.length()
+                             << " bytes" << dendl;
+  ssize_t r = cs.send(outcoming_bl, more);
+  if (r < 0) {
+    ldout(async_msgr->cct, 1) << __func__ << " send error: " << cpp_strerror(r) << dendl;
+    return r;
   }
 
-  lock.unlock();
+  ldout(async_msgr->cct, 10) << __func__ << " sent bytes " << r
+                             << " remaining bytes " << outcoming_bl.length() << dendl;
 
-  bool authorizer_valid;
-  bool need_challenge = HAVE_FEATURE(connect.features, CEPHX_V2);
-  bool had_challenge = (bool)authorizer_challenge;
-  if (!async_msgr->verify_authorizer(
-       this, peer_type, connect.authorizer_protocol, authorizer_bl,
-       authorizer_reply, authorizer_valid, session_key,
-       need_challenge ? &authorizer_challenge : nullptr) ||
-      !authorizer_valid) {
-    lock.lock();
-    char tag;
-    if (need_challenge && !had_challenge && authorizer_challenge) {
-      ldout(async_msgr->cct,10) << __func__ << ": challenging authorizer"
-                               << dendl;
-      ceph_assert(authorizer_reply.length());
-      tag = CEPH_MSGR_TAG_CHALLENGE_AUTHORIZER;
-    } else {
-      ldout(async_msgr->cct,0) << __func__ << ": got bad authorizer" << dendl;
-      tag = CEPH_MSGR_TAG_BADAUTHORIZER;
-    }
-    session_security.reset();
-    return _reply_accept(tag, connect, reply, authorizer_reply);
+  if (!open_write && is_queued()) {
+    center->create_file_event(cs.fd(), EVENT_WRITABLE, write_handler);
+    open_write = true;
   }
 
-  // We've verified the authorizer for this AsyncConnection, so set up the session security structure.  PLR
-  ldout(async_msgr->cct, 10) << __func__ << " accept setting up session_security." << dendl;
-
-  // existing?
-  AsyncConnectionRef existing = async_msgr->lookup_conn(peer_addrs);
-
-  inject_delay();
-
-  lock.lock();
-  if (state != STATE_ACCEPTING_WAIT_CONNECT_MSG_AUTH) {
-    ldout(async_msgr->cct, 1) << __func__ << " state changed while accept, it must be mark_down" << dendl;
-    ceph_assert(state == STATE_CLOSED);
-    goto fail;
+  if (open_write && !is_queued()) {
+    center->delete_file_event(cs.fd(), EVENT_WRITABLE);
+    open_write = false;
+    if (writeCallback) {
+      center->dispatch_event_external(write_callback_handler);
+    }
   }
 
-  if (existing == this)
-    existing = NULL;
-  if (existing) {
-    // There is no possible that existing connection will acquire this
-    // connection's lock
-    existing->lock.lock();  // skip lockdep check (we are locking a second AsyncConnection here)
-
-    if (existing->state == STATE_CLOSED) {
-      ldout(async_msgr->cct, 1) << __func__ << " existing already closed." << dendl;
-      existing->lock.unlock();
-      existing = NULL;
-      goto open;
-    }
+  return outcoming_bl.length();
+}
 
-    if (existing->replacing) {
-      ldout(async_msgr->cct, 1) << __func__ << " existing racing replace happened while replacing."
-                                << " existing_state=" << get_state_name(existing->state) << dendl;
-      reply.global_seq = existing->peer_global_seq;
-      r = _reply_accept(CEPH_MSGR_TAG_RETRY_GLOBAL, connect, reply, authorizer_reply);
-      existing->lock.unlock();
-      if (r < 0)
-        goto fail;
-      return 0;
-    }
+void AsyncConnection::inject_delay() {
+  if (async_msgr->cct->_conf->ms_inject_internal_delays) {
+    ldout(async_msgr->cct, 10) << __func__ << " sleep for " <<
+      async_msgr->cct->_conf->ms_inject_internal_delays << dendl;
+    utime_t t;
+    t.set_from_double(async_msgr->cct->_conf->ms_inject_internal_delays);
+    t.sleep();
+  }
+}
 
-    if (connect.global_seq < existing->peer_global_seq) {
-      ldout(async_msgr->cct, 10) << __func__ << " accept existing " << existing
-                           << ".gseq " << existing->peer_global_seq << " > "
-                           << connect.global_seq << ", RETRY_GLOBAL" << dendl;
-      reply.global_seq = existing->peer_global_seq;  // so we can send it below..
-      existing->lock.unlock();
-      return _reply_accept(CEPH_MSGR_TAG_RETRY_GLOBAL, connect, reply, authorizer_reply);
-    } else {
-      ldout(async_msgr->cct, 10) << __func__ << " accept existing " << existing
-                           << ".gseq " << existing->peer_global_seq
-                           << " <= " << connect.global_seq << ", looks ok" << dendl;
-    }
+void AsyncConnection::process() {
+  std::lock_guard<std::mutex> l(lock);
+  last_active = ceph::coarse_mono_clock::now();
+  recv_start_time = ceph::mono_clock::now();
 
-    if (existing->policy.lossy) {
-      ldout(async_msgr->cct, 0) << __func__ << " accept replacing existing (lossy) channel (new one lossy="
-                          << policy.lossy << ")" << dendl;
-      existing->was_session_reset();
-      goto replace;
-    }
+  ldout(async_msgr->cct, 20) << __func__ << dendl;
 
-    ldout(async_msgr->cct, 1) << __func__ << " accept connect_seq " << connect.connect_seq
-                              << " vs existing csq=" << existing->connect_seq << " existing_state="
-                              << get_state_name(existing->state) << dendl;
-
-    if (connect.connect_seq == 0 && existing->connect_seq > 0) {
-      ldout(async_msgr->cct,0) << __func__ << " accept peer reset, then tried to connect to us, replacing" << dendl;
-      // this is a hard reset from peer
-      is_reset_from_peer = true;
-      if (policy.resetcheck)
-        existing->was_session_reset(); // this resets out_queue, msg_ and connect_seq #'s
-      goto replace;
+  switch (state) {
+    case STATE_NONE: {
+      ldout(async_msgr->cct, 20) << __func__ << " enter none state" << dendl;
+      return;
     }
-
-    if (connect.connect_seq < existing->connect_seq) {
-      // old attempt, or we sent READY but they didn't get it.
-      ldout(async_msgr->cct, 10) << __func__ << " accept existing " << existing << ".cseq "
-                           << existing->connect_seq << " > " << connect.connect_seq
-                           << ", RETRY_SESSION" << dendl;
-      reply.connect_seq = existing->connect_seq + 1;
-      existing->lock.unlock();
-      return _reply_accept(CEPH_MSGR_TAG_RETRY_SESSION, connect, reply, authorizer_reply);
+    case STATE_CLOSED: {
+      ldout(async_msgr->cct, 20) << __func__ << " socket closed" << dendl;
+      return;
     }
+    case STATE_CONNECTING: {
+      ceph_assert(!policy.server);
 
-    if (connect.connect_seq == existing->connect_seq) {
-      // if the existing connection successfully opened, and/or
-      // subsequently went to standby, then the peer should bump
-      // their connect_seq and retry: this is not a connection race
-      // we need to resolve here.
-      if (existing->state == STATE_OPEN ||
-          existing->state == STATE_STANDBY) {
-        ldout(async_msgr->cct, 10) << __func__ << " accept connection race, existing " << existing
-                             << ".cseq " << existing->connect_seq << " == "
-                             << connect.connect_seq << ", OPEN|STANDBY, RETRY_SESSION" << dendl;
-        // if connect_seq both zero, dont stuck into dead lock. it's ok to replace
-        if (policy.resetcheck && existing->connect_seq == 0) {
-          goto replace;
-        }
-
-        reply.connect_seq = existing->connect_seq + 1;
-        existing->lock.unlock();
-        return _reply_accept(CEPH_MSGR_TAG_RETRY_SESSION, connect, reply, authorizer_reply);
+      if (cs) {
+        center->delete_file_event(cs.fd(), EVENT_READABLE | EVENT_WRITABLE);
+        cs.close();
       }
 
-      // connection race?
-      if (peer_addrs.legacy_addr() < async_msgr->get_myaddrs().legacy_addr() ||
-         existing->policy.server) {
-        // incoming wins
-        ldout(async_msgr->cct, 10) << __func__ << " accept connection race, existing " << existing
-                             << ".cseq " << existing->connect_seq << " == " << connect.connect_seq
-                             << ", or we are server, replacing my attempt" << dendl;
-        goto replace;
-      } else {
-        // our existing outgoing wins
-        ldout(async_msgr->cct,10) << __func__ << " accept connection race, existing "
-                            << existing << ".cseq " << existing->connect_seq
-                            << " == " << connect.connect_seq << ", sending WAIT" << dendl;
-        ceph_assert(peer_addrs.legacy_addr() > async_msgr->get_myaddrs().legacy_addr());
-        existing->lock.unlock();
-        return _reply_accept(CEPH_MSGR_TAG_WAIT, connect, reply, authorizer_reply);
+      SocketOptions opts;
+      opts.priority = async_msgr->get_socket_priority();
+      opts.connect_bind_addr = msgr->get_myaddr();
+      ssize_t r = worker->connect(get_peer_addr(), opts, &cs);
+      if (r < 0) {
+        protocol->fault();
+        return;
       }
-    }
 
-    ceph_assert(connect.connect_seq > existing->connect_seq);
-    ceph_assert(connect.global_seq >= existing->peer_global_seq);
-    if (policy.resetcheck &&   // RESETSESSION only used by servers; peers do not reset each other
-        existing->connect_seq == 0) {
-      ldout(async_msgr->cct, 0) << __func__ << " accept we reset (peer sent cseq "
-                          << connect.connect_seq << ", " << existing << ".cseq = "
-                          << existing->connect_seq << "), sending RESETSESSION" << dendl;
-      existing->lock.unlock();
-      return _reply_accept(CEPH_MSGR_TAG_RESETSESSION, connect, reply, authorizer_reply);
+      center->create_file_event(cs.fd(), EVENT_READABLE, read_handler);
+      state = STATE_CONNECTING_RE;
     }
+    case STATE_CONNECTING_RE: {
+      ssize_t r = cs.is_connected();
+      if (r < 0) {
+        ldout(async_msgr->cct, 1) << __func__ << " reconnect failed " << dendl;
+        if (r == -ECONNREFUSED) {
+          ldout(async_msgr->cct, 2)
+              << __func__ << " connection refused!" << dendl;
+          dispatch_queue->queue_refused(this);
+        }
+        protocol->fault();
+        return;
+      } else if (r == 0) {
+        ldout(async_msgr->cct, 10)
+            << __func__ << " nonblock connect inprogress" << dendl;
+        if (async_msgr->get_stack()->nonblock_connect_need_writable_event()) {
+          center->create_file_event(cs.fd(), EVENT_WRITABLE,
+                                    read_handler);
+        }
+        logger->tinc(l_msgr_running_recv_time,
+               ceph::mono_clock::now() - recv_start_time);
+        return;
+      }
 
-    // reconnect
-    ldout(async_msgr->cct, 10) << __func__ << " accept peer sent cseq " << connect.connect_seq
-                         << " > " << existing->connect_seq << dendl;
-    goto replace;
-  } // existing
-  else if (!replacing && connect.connect_seq > 0) {
-    // we reset, and they are opening a new session
-    ldout(async_msgr->cct, 0) << __func__ << " accept we reset (peer sent cseq "
-                        << connect.connect_seq << "), sending RESETSESSION" << dendl;
-    return _reply_accept(CEPH_MSGR_TAG_RESETSESSION, connect, reply, authorizer_reply);
-  } else {
-    // new session
-    ldout(async_msgr->cct, 10) << __func__ << " accept new session" << dendl;
-    existing = NULL;
-    goto open;
-  }
-
- replace:
-  ldout(async_msgr->cct, 10) << __func__ << " accept replacing " << existing << dendl;
-
-  inject_delay();
-  if (existing->policy.lossy) {
-    // disconnect from the Connection
-    ldout(async_msgr->cct, 1) << __func__ << " replacing on lossy channel, failing existing" << dendl;
-    existing->_stop();
-    existing->dispatch_queue->queue_reset(existing.get());
-  } else {
-    ceph_assert(can_write == WriteStatus::NOWRITE);
-    existing->write_lock.lock();
-
-    // reset the in_seq if this is a hard reset from peer,
-    // otherwise we respect our original connection's value
-    if (is_reset_from_peer) {
-      existing->is_reset_from_peer = true;
+      center->delete_file_event(cs.fd(), EVENT_WRITABLE);
+      ldout(async_msgr->cct, 10)
+          << __func__ << " connect successfully, ready to send banner" << dendl;
+      state = STATE_CONNECTION_ESTABLISHED;
+      break;
     }
 
-    center->delete_file_event(cs.fd(), EVENT_READABLE|EVENT_WRITABLE);
+    case STATE_ACCEPTING: {
+      center->create_file_event(cs.fd(), EVENT_READABLE, read_handler);
+      state = STATE_CONNECTION_ESTABLISHED;
 
-    if (existing->delay_state) {
-      existing->delay_state->flush();
-      ceph_assert(!delay_state);
+      break;
     }
-    existing->reset_recv_state();
-
-    auto temp_cs = std::move(cs);
-    EventCenter *new_center = center;
-    Worker *new_worker = worker;
-    // avoid _stop shutdown replacing socket
-    // queue a reset on the new connection, which we're dumping for the old
-    _stop();
-
-    dispatch_queue->queue_reset(this);
-    ldout(async_msgr->cct, 1) << __func__ << " stop myself to swap existing" << dendl;
-    existing->can_write = WriteStatus::REPLACING;
-    existing->replacing = true;
-    existing->state_offset = 0;
-    // avoid previous thread modify event
-    existing->state = STATE_NONE;
-    // Discard existing prefetch buffer in `recv_buf`
-    existing->recv_start = existing->recv_end = 0;
-    // there shouldn't exist any buffer
-    ceph_assert(recv_start == recv_end);
-
-    existing->authorizer_challenge.reset();
-
-    auto deactivate_existing = std::bind(
-        [existing, new_worker, new_center, connect, reply, authorizer_reply](ConnectedSocket &cs) mutable {
-      // we need to delete time event in original thread
-      {
-        std::lock_guard<std::mutex> l(existing->lock);
-        existing->write_lock.lock();
-        existing->requeue_sent();
-        existing->outcoming_bl.clear();
-        existing->open_write = false;
-        existing->write_lock.unlock();
-        if (existing->state == STATE_NONE) {
-          existing->shutdown_socket();
-          existing->cs = std::move(cs);
-          existing->worker->references--;
-          new_worker->references++;
-          existing->logger = new_worker->get_perf_counter();
-          existing->worker = new_worker;
-          existing->center = new_center;
-          if (existing->delay_state)
-            existing->delay_state->set_center(new_center);
-        } else if (existing->state == STATE_CLOSED) {
-          auto back_to_close = std::bind(
-            [](ConnectedSocket &cs) mutable { cs.close(); }, std::move(cs));
-          new_center->submit_to(
-              new_center->get_id(), std::move(back_to_close), true);
-          return ;
-        } else {
-          ceph_abort();
-        }
-      }
-
-      // Before changing existing->center, it may already exists some events in existing->center's queue.
-      // Then if we mark down `existing`, it will execute in another thread and clean up connection.
-      // Previous event will result in segment fault
-      auto transfer_existing = [existing, connect, reply, authorizer_reply]() mutable {
-        std::lock_guard<std::mutex> l(existing->lock);
-        if (existing->state == STATE_CLOSED)
-          return ;
-        ceph_assert(existing->state == STATE_NONE);
-  
-        existing->state = STATE_ACCEPTING_WAIT_CONNECT_MSG;
-        existing->center->create_file_event(existing->cs.fd(), EVENT_READABLE, existing->read_handler);
-        reply.global_seq = existing->peer_global_seq;
-        if (existing->_reply_accept(CEPH_MSGR_TAG_RETRY_GLOBAL, connect, reply, authorizer_reply) < 0) {
-          // handle error
-          existing->fault();
-        }
-      };
-      if (existing->center->in_thread())
-        transfer_existing();
-      else
-        existing->center->submit_to(
-            existing->center->get_id(), std::move(transfer_existing), true);
-    }, std::move(temp_cs));
-
-    existing->center->submit_to(
-        existing->center->get_id(), std::move(deactivate_existing), true);
-    existing->write_lock.unlock();
-    existing->lock.unlock();
-    return 0;
-  }
-  existing->lock.unlock();
-
- open:
-  connect_seq = connect.connect_seq + 1;
-  peer_global_seq = connect.global_seq;
-  ldout(async_msgr->cct, 10) << __func__ << " accept success, connect_seq = "
-                             << connect_seq << " in_seq=" << in_seq << ", sending READY" << dendl;
-
-  int next_state;
-
-  // if it is a hard reset from peer, we don't need a round-trip to negotiate in/out sequence
-  if ((connect.features & CEPH_FEATURE_RECONNECT_SEQ) && !is_reset_from_peer) {
-    reply.tag = CEPH_MSGR_TAG_SEQ;
-    next_state = STATE_ACCEPTING_WAIT_SEQ;
-  } else {
-    reply.tag = CEPH_MSGR_TAG_READY;
-    next_state = STATE_ACCEPTING_READY;
-    discard_requeued_up_to(0);
-    is_reset_from_peer = false;
-    in_seq = 0;
-  }
-
-  // send READY reply
-  reply.features = policy.features_supported;
-  reply.global_seq = async_msgr->get_global_seq();
-  reply.connect_seq = connect_seq;
-  reply.flags = 0;
-  reply.authorizer_len = authorizer_reply.length();
-  if (policy.lossy)
-    reply.flags = reply.flags | CEPH_MSG_CONNECT_LOSSY;
-
-  set_features((uint64_t)reply.features & (uint64_t)connect.features);
-  ldout(async_msgr->cct, 10) << __func__ << " accept features " << get_features() << dendl;
-
-  session_security.reset(
-      get_auth_session_handler(async_msgr->cct, connect.authorizer_protocol,
-                               session_key, get_features()));
-
-  reply_bl.append((char*)&reply, sizeof(reply));
-
-  if (reply.authorizer_len)
-    reply_bl.append(authorizer_reply.c_str(), authorizer_reply.length());
-
-  if (reply.tag == CEPH_MSGR_TAG_SEQ) {
-    uint64_t s = in_seq;
-    reply_bl.append((char*)&s, sizeof(s));
-  }
 
-  lock.unlock();
-  // Because "replacing" will prevent other connections preempt this addr,
-  // it's safe that here we don't acquire Connection's lock
-  r = async_msgr->accept_conn(this);
-
-  inject_delay();
-  
-  lock.lock();
-  replacing = false;
-  if (r < 0) {
-    ldout(async_msgr->cct, 1) << __func__ << " existing race replacing process for addr=" << peer_addrs
-                              << " just fail later one(this)" << dendl;
-    goto fail_registered;
-  }
-  if (state != STATE_ACCEPTING_WAIT_CONNECT_MSG_AUTH) {
-    ldout(async_msgr->cct, 1) << __func__ << " state changed while accept_conn, it must be mark_down" << dendl;
-    ceph_assert(state == STATE_CLOSED || state == STATE_NONE);
-    goto fail_registered;
+    case STATE_CONNECTION_ESTABLISHED: {
+      if (pendingReadLen) {
+        read(*pendingReadLen, read_buffer, readCallback);
+        return;
+      }
+      break;
+    }
   }
 
-  r = try_send(reply_bl);
-  if (r < 0)
-    goto fail_registered;
+  protocol->read_event();
 
-  // notify
-  dispatch_queue->queue_accept(this);
-  async_msgr->ms_deliver_handle_fast_accept(this);
-  once_ready = true;
-
-  if (r == 0) {
-    state = next_state;
-    ldout(async_msgr->cct, 2) << __func__ << " accept write reply msg done" << dendl;
-  } else {
-    state = STATE_WAIT_SEND;
-    state_after_send = next_state;
-  }
+  logger->tinc(l_msgr_running_recv_time,
+               ceph::mono_clock::now() - recv_start_time);
+}
 
-  return 0;
+bool AsyncConnection::is_connected() {
+  return protocol->is_connected();
+}
 
- fail_registered:
-  ldout(async_msgr->cct, 10) << __func__ << " accept fault after register" << dendl;
-  inject_delay();
+void AsyncConnection::connect(const entity_addrvec_t &addrs, int type,
+                              entity_addr_t &target) {
 
- fail:
-  ldout(async_msgr->cct, 10) << __func__ << " failed to accept." << dendl;
-  return -1;
+  std::lock_guard<std::mutex> l(lock);
+  set_peer_type(type);
+  set_peer_addrs(addrs);
+  policy = msgr->get_policy(type);
+  target_addr = target;
+  _connect();
 }
 
 void AsyncConnection::_connect()
 {
-  ldout(async_msgr->cct, 10) << __func__ << " csq=" << connect_seq << dendl;
+  ldout(async_msgr->cct, 10) << __func__ << dendl;
 
   state = STATE_CONNECTING;
+  protocol->connect();
   // rescheduler connection in order to avoid lock dep
   // may called by external thread(send_message)
   center->dispatch_event_external(read_handler);
@@ -1929,6 +473,7 @@ void AsyncConnection::accept(ConnectedSocket socket, entity_addr_t &addr)
   socket_addr = addr;
   target_addr = addr; // until we know better
   state = STATE_ACCEPTING;
+  protocol->accept();
   // rescheduler connection in order to avoid lock dep
   center->dispatch_event_external(read_handler);
 }
@@ -1958,7 +503,7 @@ int AsyncConnection::send_message(Message *m)
   if (async_msgr->get_myaddrs() == get_peer_addrs()) { //loopback connection
     ldout(async_msgr->cct, 20) << __func__ << " " << *m << " local" << dendl;
     std::lock_guard<std::mutex> l(write_lock);
-    if (can_write != WriteStatus::CLOSED) {
+    if (protocol->is_connected()) {
       dispatch_queue->local_delivery(m, m->get_priority());
     } else {
       ldout(async_msgr->cct, 10) << __func__ << " loopback connection closed."
@@ -1973,403 +518,55 @@ int AsyncConnection::send_message(Message *m)
   // may disturb users
   logger->inc(l_msgr_send_messages);
 
-  bufferlist bl;
-  uint64_t f = get_features();
-
-  // TODO: Currently not all messages supports reencode like MOSDMap, so here
-  // only let fast dispatch support messages prepare message
-  bool can_fast_prepare = async_msgr->ms_can_fast_dispatch(m);
-  if (can_fast_prepare)
-    prepare_send_message(f, m, bl);
-
-  std::lock_guard<std::mutex> l(write_lock);
-  // "features" changes will change the payload encoding
-  if (can_fast_prepare && (can_write == WriteStatus::NOWRITE || get_features() != f)) {
-    // ensure the correctness of message encoding
-    bl.clear();
-    m->get_payload().clear();
-    ldout(async_msgr->cct, 5) << __func__ << " clear encoded buffer previous "
-                              << f << " != " << get_features() << dendl;
-  }
-  if (can_write == WriteStatus::CLOSED) {
-    ldout(async_msgr->cct, 10) << __func__ << " connection closed."
-                               << " Drop message " << m << dendl;
-    m->put();
-  } else {
-    m->trace.event("async enqueueing message");
-    out_q[m->get_priority()].emplace_back(std::move(bl), m);
-    ldout(async_msgr->cct, 15) << __func__ << " inline write is denied, reschedule m=" << m << dendl;
-    if (can_write != WriteStatus::REPLACING)
-      center->dispatch_event_external(write_handler);
-  }
+  protocol->send_message(m);
   return 0;
 }
 
-void AsyncConnection::requeue_sent()
-{
-  if (sent.empty())
-    return;
-
-  list<pair<bufferlist, Message*> >& rq = out_q[CEPH_MSG_PRIO_HIGHEST];
-  out_seq -= sent.size();
-  while (!sent.empty()) {
-    Message* m = sent.back();
-    sent.pop_back();
-    ldout(async_msgr->cct, 10) << __func__ << " " << *m << " for resend "
-                               << " (" << m->get_seq() << ")" << dendl;
-    rq.push_front(make_pair(bufferlist(), m));
-  }
-}
-
-void AsyncConnection::discard_requeued_up_to(uint64_t seq)
-{
-  ldout(async_msgr->cct, 10) << __func__ << " " << seq << dendl;
-  std::lock_guard<std::mutex> l(write_lock);
-  if (out_q.count(CEPH_MSG_PRIO_HIGHEST) == 0) {
-    out_seq = seq;
-    return;
-  }
-  list<pair<bufferlist, Message*> >& rq = out_q[CEPH_MSG_PRIO_HIGHEST];
-  while (!rq.empty()) {
-    pair<bufferlist, Message*> p = rq.front();
-    if (p.second->get_seq() == 0 || p.second->get_seq() > seq)
-      break;
-    ldout(async_msgr->cct, 10) << __func__ << " " << *(p.second) << " for resend seq " << p.second->get_seq()
-                         << " <= " << seq << ", discarding" << dendl;
-    p.second->put();
-    rq.pop_front();
-    out_seq++;
-  }
-  if (rq.empty())
-    out_q.erase(CEPH_MSG_PRIO_HIGHEST);
-}
-
-/*
- * Tears down the AsyncConnection's message queues, and removes them from the DispatchQueue
- * Must hold write_lock prior to calling.
- */
-void AsyncConnection::discard_out_queue()
-{
-  ldout(async_msgr->cct, 10) << __func__ << " started" << dendl;
-
-  for (list<Message*>::iterator p = sent.begin(); p != sent.end(); ++p) {
-    ldout(async_msgr->cct, 20) << __func__ << " discard " << *p << dendl;
-    (*p)->put();
-  }
-  sent.clear();
-  for (map<int, list<pair<bufferlist, Message*> > >::iterator p = out_q.begin(); p != out_q.end(); ++p)
-    for (list<pair<bufferlist, Message*> >::iterator r = p->second.begin(); r != p->second.end(); ++r) {
-      ldout(async_msgr->cct, 20) << __func__ << " discard " << r->second << dendl;
-      r->second->put();
-    }
-  out_q.clear();
-}
-
-void AsyncConnection::randomize_out_seq()
-{
-  if (get_features() & CEPH_FEATURE_MSG_AUTH) {
-    // Set out_seq to a random value, so CRC won't be predictable.
-    auto rand_seq = ceph::util::generate_random_number<uint64_t>(0, SEQ_MASK);
-    lsubdout(async_msgr->cct, ms, 10) << __func__ << " randomize_out_seq " << rand_seq << dendl;
-    out_seq = rand_seq;
-  } else {
-    // previously, seq #'s always started at 0.
-    out_seq = 0;
-  }
-}
 
 void AsyncConnection::fault()
 {
-  if (state == STATE_CLOSED || state == STATE_NONE) {
-    ldout(async_msgr->cct, 10) << __func__ << " connection is already closed" << dendl;
-    return ;
-  }
-
-  if (policy.lossy && !(state >= STATE_CONNECTING && state < STATE_CONNECTING_READY)) {
-    ldout(async_msgr->cct, 1) << __func__ << " on lossy channel, failing" << dendl;
-    _stop();
-    dispatch_queue->queue_reset(this);
-    return ;
-  }
-
-  write_lock.lock();
-  can_write = WriteStatus::NOWRITE;
   shutdown_socket();
   open_write = false;
 
   // queue delayed items immediately
   if (delay_state)
     delay_state->flush();
-  // requeue sent items
-  requeue_sent();
+
   recv_start = recv_end = 0;
   state_offset = 0;
-  is_reset_from_peer = false;
   outcoming_bl.clear();
-  if (!once_ready && !is_queued() &&
-      state >=STATE_ACCEPTING && state <= STATE_ACCEPTING_WAIT_CONNECT_MSG_AUTH &&
-      !replacing) {
-    ldout(async_msgr->cct, 10) << __func__ << " with nothing to send and in the half "
-                              << " accept state just closed" << dendl;
-    write_lock.unlock();
-    _stop();
-    dispatch_queue->queue_reset(this);
-    return ;
-  }
-  replacing = false;
-  reset_recv_state();
-  if (policy.standby && !is_queued() && state != STATE_WAIT) {
-    ldout(async_msgr->cct, 10) << __func__ << " with nothing to send, going to standby" << dendl;
-    state = STATE_STANDBY;
-    write_lock.unlock();
-    return;
-  }
-
-  write_lock.unlock();
-  if (!(state >= STATE_CONNECTING && state < STATE_CONNECTING_READY) &&
-      state != STATE_WAIT) { // STATE_WAIT is coming from STATE_CONNECTING_*
-    // policy maybe empty when state is in accept
-    if (policy.server) {
-      ldout(async_msgr->cct, 0) << __func__ << " server, going to standby" << dendl;
-      state = STATE_STANDBY;
-    } else {
-      ldout(async_msgr->cct, 0) << __func__ << " initiating reconnect" << dendl;
-      connect_seq++;
-      state = STATE_CONNECTING;
-    }
-    backoff = utime_t();
-    center->dispatch_event_external(read_handler);
-  } else {
-    if (state == STATE_WAIT) {
-      backoff.set_from_double(async_msgr->cct->_conf->ms_max_backoff);
-    } else if (backoff == utime_t()) {
-      backoff.set_from_double(async_msgr->cct->_conf->ms_initial_backoff);
-    } else {
-      backoff += backoff;
-      if (backoff > async_msgr->cct->_conf->ms_max_backoff)
-        backoff.set_from_double(async_msgr->cct->_conf->ms_max_backoff);
-    }
-
-    state = STATE_CONNECTING;
-    ldout(async_msgr->cct, 10) << __func__ << " waiting " << backoff << dendl;
-    // woke up again;
-    register_time_events.insert(center->create_time_event(
-            backoff.to_nsec()/1000, wakeup_handler));
-  }
 }
 
-void AsyncConnection::was_session_reset()
-{
-  ldout(async_msgr->cct,10) << __func__ << " started" << dendl;
-  std::lock_guard<std::mutex> l(write_lock);
-  if (delay_state)
-    delay_state->discard();
-  dispatch_queue->discard_queue(conn_id);
-  discard_out_queue();
-  // note: we need to clear outcoming_bl here, but was_session_reset may be
-  // called by other thread, so let caller clear this itself!
-  // outcoming_bl.clear();
-
-  dispatch_queue->queue_remote_reset(this);
-
-  randomize_out_seq();
-
-  in_seq = 0;
-  connect_seq = 0;
-  // it's safe to directly set 0, double locked
-  ack_left = 0;
-  once_ready = false;
-  can_write = WriteStatus::NOWRITE;
-}
-
-void AsyncConnection::_stop()
-{
-  if (state == STATE_CLOSED)
-    return ;
-
-  if (delay_state)
-    delay_state->flush();
-
-  ldout(async_msgr->cct, 2) << __func__ << dendl;
-  std::lock_guard<std::mutex> l(write_lock);
-
-  reset_recv_state();
+void AsyncConnection::_stop() {
+  writeCallback.reset();
   dispatch_queue->discard_queue(conn_id);
-  discard_out_queue();
   async_msgr->unregister_conn(this);
   worker->release_worker();
 
   state = STATE_CLOSED;
   open_write = false;
-  can_write = WriteStatus::CLOSED;
+
   state_offset = 0;
   // Make sure in-queue events will been processed
   center->dispatch_event_external(EventCallbackRef(new C_clean_handler(this)));
 }
 
-void AsyncConnection::prepare_send_message(uint64_t features, Message *m, bufferlist &bl)
-{
-  ldout(async_msgr->cct, 20) << __func__ << " m" << " " << *m << dendl;
-
-  // associate message with Connection (for benefit of encode_payload)
-  if (m->empty_payload())
-    ldout(async_msgr->cct, 20) << __func__ << " encoding features "
-                               << features << " " << m << " " << *m << dendl;
-  else
-    ldout(async_msgr->cct, 20) << __func__ << " half-reencoding features "
-                               << features << " " << m << " " << *m << dendl;
-
-  // encode and copy out of *m
-  m->encode(features, msgr->crcflags);
-
-  bl.append(m->get_payload());
-  bl.append(m->get_middle());
-  bl.append(m->get_data());
-}
-
-ssize_t AsyncConnection::write_message(Message *m, bufferlist& bl, bool more)
-{
-  FUNCTRACE(async_msgr->cct);
-  ceph_assert(center->in_thread());
-  m->set_seq(++out_seq);
-
-  if (msgr->crcflags & MSG_CRC_HEADER)
-    m->calc_header_crc();
-
-  ceph_msg_header& header = m->get_header();
-  ceph_msg_footer& footer = m->get_footer();
-
-  // TODO: let sign_message could be reentry?
-  // Now that we have all the crcs calculated, handle the
-  // digital signature for the message, if the AsyncConnection has session
-  // security set up.  Some session security options do not
-  // actually calculate and check the signature, but they should
-  // handle the calls to sign_message and check_signature.  PLR
-  if (session_security.get() == NULL) {
-    ldout(async_msgr->cct, 20) << __func__ << " no session security" << dendl;
-  } else {
-    if (session_security->sign_message(m)) {
-      ldout(async_msgr->cct, 20) << __func__ << " failed to sign m="
-                                 << m << "): sig = " << footer.sig << dendl;
-    } else {
-      ldout(async_msgr->cct, 20) << __func__ << " signed m=" << m
-                                 << "): sig = " << footer.sig << dendl;
-    }
-  }
-  
-  outcoming_bl.append(CEPH_MSGR_TAG_MSG);
-  outcoming_bl.append((char*)&header, sizeof(header));
-
-  ldout(async_msgr->cct, 20) << __func__ << " sending message type=" << header.type
-                             << " src " << entity_name_t(header.src)
-                             << " front=" << header.front_len
-                             << " data=" << header.data_len
-                             << " off " << header.data_off << dendl;
-
-  if ((bl.length() <= ASYNC_COALESCE_THRESHOLD) && (bl.buffers().size() > 1)) {
-    for (const auto &pb : bl.buffers()) {
-      outcoming_bl.append((char*)pb.c_str(), pb.length());
-    }
-  } else {
-    outcoming_bl.claim_append(bl);  
-  }
-
-  // send footer; if receiver doesn't support signatures, use the old footer format
-  ceph_msg_footer_old old_footer;
-  if (has_feature(CEPH_FEATURE_MSG_AUTH)) {
-    outcoming_bl.append((char*)&footer, sizeof(footer));
-  } else {
-    if (msgr->crcflags & MSG_CRC_HEADER) {
-      old_footer.front_crc = footer.front_crc;
-      old_footer.middle_crc = footer.middle_crc;
-      old_footer.data_crc = footer.data_crc;
-    } else {
-       old_footer.front_crc = old_footer.middle_crc = 0;
-    }
-    old_footer.data_crc = msgr->crcflags & MSG_CRC_DATA ? footer.data_crc : 0;
-    old_footer.flags = footer.flags;
-    outcoming_bl.append((char*)&old_footer, sizeof(old_footer));
-  }
-
-  m->trace.event("async writing message");
-  ldout(async_msgr->cct, 20) << __func__ << " sending " << m->get_seq()
-                             << " " << m << dendl;
-  ssize_t total_send_size = outcoming_bl.length();
-  ssize_t rc = _try_send(more);
-  if (rc < 0) {
-    ldout(async_msgr->cct, 1) << __func__ << " error sending " << m << ", "
-                              << cpp_strerror(rc) << dendl;
-  } else {
-    logger->inc(l_msgr_send_bytes, total_send_size - outcoming_bl.length());
-    ldout(async_msgr->cct, 10) << __func__ << " sending " << m << (rc ? " continuely." :" done.") << dendl;
-  }
-  if (m->get_type() == CEPH_MSG_OSD_OP)
-    OID_EVENT_TRACE_WITH_MSG(m, "SEND_MSG_OSD_OP_END", false);
-  else if (m->get_type() == CEPH_MSG_OSD_OPREPLY)
-    OID_EVENT_TRACE_WITH_MSG(m, "SEND_MSG_OSD_OPREPLY_END", false);
-  m->put();
-
-  return rc;
+bool AsyncConnection::is_queued() const {
+  return protocol->is_queued() || outcoming_bl.length();
 }
 
-void AsyncConnection::reset_recv_state()
-{
-  // clean up state internal variables and states
-  if (state >= STATE_CONNECTING_SEND_CONNECT_MSG &&
-      state <= STATE_CONNECTING_READY) {
-    delete authorizer;
-    authorizer = NULL;
-    got_bad_auth = false;
-  }
-
-  if (state > STATE_OPEN_MESSAGE_THROTTLE_MESSAGE &&
-      state <= STATE_OPEN_MESSAGE_READ_FOOTER_AND_DISPATCH
-      && policy.throttler_messages) {
-    ldout(async_msgr->cct, 10) << __func__ << " releasing " << 1
-                               << " message to policy throttler "
-                               << policy.throttler_messages->get_current() << "/"
-                               << policy.throttler_messages->get_max() << dendl;
-    policy.throttler_messages->put();
+void AsyncConnection::shutdown_socket() {
+  for (auto &&t : register_time_events) center->delete_time_event(t);
+  register_time_events.clear();
+  if (last_tick_id) {
+    center->delete_time_event(last_tick_id);
+    last_tick_id = 0;
   }
-  if (state > STATE_OPEN_MESSAGE_THROTTLE_BYTES &&
-      state <= STATE_OPEN_MESSAGE_READ_FOOTER_AND_DISPATCH) {
-    if (policy.throttler_bytes) {
-      ldout(async_msgr->cct, 10) << __func__ << " releasing " << cur_msg_size
-                                 << " bytes to policy throttler "
-                                 << policy.throttler_bytes->get_current() << "/"
-                                 << policy.throttler_bytes->get_max() << dendl;
-      policy.throttler_bytes->put(cur_msg_size);
-    }
+  if (cs) {
+    center->delete_file_event(cs.fd(), EVENT_READABLE | EVENT_WRITABLE);
+    cs.shutdown();
+    cs.close();
   }
-  if (state > STATE_OPEN_MESSAGE_THROTTLE_DISPATCH_QUEUE &&
-      state <= STATE_OPEN_MESSAGE_READ_FOOTER_AND_DISPATCH) {
-    ldout(async_msgr->cct, 10) << __func__ << " releasing " << cur_msg_size
-                               << " bytes to dispatch_queue throttler "
-                               << dispatch_queue->dispatch_throttler.get_current() << "/"
-                               << dispatch_queue->dispatch_throttler.get_max() << dendl;
-    dispatch_queue->dispatch_throttle_release(cur_msg_size);
-  }
-}
-
-void AsyncConnection::handle_ack(uint64_t seq)
-{
-  ldout(async_msgr->cct, 15) << __func__ << " got ack seq " << seq << dendl;
-  // trim sent list
-  static const int max_pending = 128;
-  int i = 0;
-  Message *pending[max_pending];
-  write_lock.lock();
-  while (!sent.empty() && sent.front()->get_seq() <= seq && i < max_pending) {
-    Message* m = sent.front();
-    sent.pop_front();
-    pending[i++] = m;
-    ldout(async_msgr->cct, 10) << __func__ << " got ack seq "
-                               << seq << " >= " << m->get_seq() << " on "
-                               << m << " " << *m << dendl;
-  }
-  write_lock.unlock();
-  for (int k = 0; k < i; k++)
-    pending[k]->put();
 }
 
 void AsyncConnection::DelayedDelivery::do_request(uint64_t id)
@@ -2435,145 +632,58 @@ void AsyncConnection::DelayedDelivery::flush() {
 
 void AsyncConnection::send_keepalive()
 {
-  ldout(async_msgr->cct, 10) << __func__ << dendl;
-  std::lock_guard<std::mutex> l(write_lock);
-  if (can_write != WriteStatus::CLOSED) {
-    keepalive = true;
-    center->dispatch_event_external(write_handler);
-  }
+  protocol->send_keepalive();
 }
 
 void AsyncConnection::mark_down()
 {
   ldout(async_msgr->cct, 1) << __func__ << dendl;
   std::lock_guard<std::mutex> l(lock);
-  _stop();
-}
-
-void AsyncConnection::_append_keepalive_or_ack(bool ack, utime_t *tp)
-{
-  ldout(async_msgr->cct, 10) << __func__ << dendl;
-  if (ack) {
-    ceph_assert(tp);
-    struct ceph_timespec ts;
-    tp->encode_timeval(&ts);
-    outcoming_bl.append(CEPH_MSGR_TAG_KEEPALIVE2_ACK);
-    outcoming_bl.append((char*)&ts, sizeof(ts));
-  } else if (has_feature(CEPH_FEATURE_MSGR_KEEPALIVE2)) {
-    struct ceph_timespec ts;
-    utime_t t = ceph_clock_now();
-    t.encode_timeval(&ts);
-    outcoming_bl.append(CEPH_MSGR_TAG_KEEPALIVE2);
-    outcoming_bl.append((char*)&ts, sizeof(ts));
-  } else {
-    outcoming_bl.append(CEPH_MSGR_TAG_KEEPALIVE);
-  }
+  protocol->stop();
 }
 
 void AsyncConnection::handle_write()
 {
-  ldout(async_msgr->cct, 10) << __func__ << dendl;
-  ssize_t r = 0;
+  ldout(async_msgr->cct, 4) << __func__ << dendl;
+  protocol->write_event();
+}
 
+void AsyncConnection::handle_write_callback() {
+  std::lock_guard<std::mutex> l(lock);
+  last_active = ceph::coarse_mono_clock::now();
+  recv_start_time = ceph::mono_clock::now();
   write_lock.lock();
-  if (can_write == WriteStatus::CANWRITE) {
-    if (keepalive) {
-      _append_keepalive_or_ack();
-      keepalive = false;
-    }
-
-    auto start = ceph::mono_clock::now();
-    bool more;
-    do {
-      bufferlist data;
-      Message *m = _get_next_outgoing(&data);
-      if (!m)
-        break;
-
-      if (!policy.lossy) {
-        // put on sent list
-        sent.push_back(m);
-        m->get();
-      }
-      more = _has_next_outgoing();
-      write_lock.unlock();
-
-      // send_message or requeue messages may not encode message
-      if (!data.length())
-        prepare_send_message(get_features(), m, data);
-
-      r = write_message(m, data, more);
-
-      write_lock.lock();
-      if (r == 0) {
-       ;
-      } else if (r < 0) {
-       ldout(async_msgr->cct, 1) << __func__ << " send msg failed" << dendl;
-       break;
-      } else if (r > 0)
-       break;
-    } while (can_write == WriteStatus::CANWRITE);
-    write_lock.unlock();
-
-    // if r > 0 mean data still lefted, so no need _try_send.
-    if (r == 0) {
-      uint64_t left = ack_left;
-      if (left) {
-       ceph_le64 s;
-       s = in_seq;
-       outcoming_bl.append(CEPH_MSGR_TAG_ACK);
-       outcoming_bl.append((char*)&s, sizeof(s));
-       ldout(async_msgr->cct, 10) << __func__ << " try send msg ack, acked " << left << " messages" << dendl;
-       ack_left -= left;
-       left = ack_left;
-       r = _try_send(left);
-      } else if (is_queued()) {
-       r = _try_send();
-      }
-    }
-
-    logger->tinc(l_msgr_running_send_time, ceph::mono_clock::now() - start);
-    if (r < 0) {
-      ldout(async_msgr->cct, 1) << __func__ << " send msg failed" << dendl;
-      goto fail;
-    }
-  } else {
-    write_lock.unlock();
-    lock.lock();
-    write_lock.lock();
-    if (state == STATE_STANDBY && !policy.server && is_queued()) {
-      ldout(async_msgr->cct, 10) << __func__ << " policy.server is false" << dendl;
-      _connect();
-    } else if (cs && state != STATE_NONE && state != STATE_CONNECTING && state != STATE_CONNECTING_RE && state != STATE_CLOSED) {
-      r = _try_send();
-      if (r < 0) {
-        ldout(async_msgr->cct, 1) << __func__ << " send outcoming bl failed" << dendl;
-        write_lock.unlock();
-        fault();
-        lock.unlock();
-        return ;
-      }
-    }
+  if (writeCallback) {
+    auto callback = *writeCallback;
+    writeCallback.reset();
     write_lock.unlock();
-    lock.unlock();
+    callback(0);
+    return;
   }
-
-  return ;
-
- fail:
-  lock.lock();
-  fault();
-  lock.unlock();
+  write_lock.unlock();
 }
 
 void AsyncConnection::stop(bool queue_reset) {
   lock.lock();
   bool need_queue_reset = (state != STATE_CLOSED) && queue_reset;
-  _stop();
+  protocol->stop();
   lock.unlock();
   if (need_queue_reset) dispatch_queue->queue_reset(this);
 }
 
+void AsyncConnection::cleanup() {
+  shutdown_socket();
+  delete read_handler;
+  delete write_handler;
+  delete write_callback_handler;
+  delete wakeup_handler;
+  delete tick_handler;
+  if (delay_state) {
+    delete delay_state;
+    delay_state = NULL;
+  }
+}
+
 void AsyncConnection::wakeup_from(uint64_t id)
 {
   lock.lock();
@@ -2586,7 +696,7 @@ void AsyncConnection::tick(uint64_t id)
 {
   auto now = ceph::coarse_mono_clock::now();
   ldout(async_msgr->cct, 20) << __func__ << " last_id=" << last_tick_id
-                             << " last_active" << last_active << dendl;
+                             << " last_active=" << last_active << dendl;
   std::lock_guard<std::mutex> l(lock);
   last_tick_id = 0;
   auto idle_period = std::chrono::duration_cast<std::chrono::microseconds>(now - last_active).count();
@@ -2594,7 +704,7 @@ void AsyncConnection::tick(uint64_t id)
     ldout(async_msgr->cct, 1) << __func__ << " idle(" << idle_period << ") more than "
                               << inactive_timeout_us
                               << " us, mark self fault." << dendl;
-    fault();
+    protocol->fault();
   } else if (is_connected()) {
     last_tick_id = center->create_time_event(inactive_timeout_us, tick_handler);
   }
index 29db0a412f319a117c224ef4f21b711d5f0bfdde..78025d8342ba353f6e76f77e3110a8ff80b0b9e7 100644 (file)
@@ -1,4 +1,4 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
 // vim: ts=8 sw=2 smarttab
 /*
  * Ceph - scalable distributed file system
@@ -23,6 +23,8 @@
 #include <list>
 #include <mutex>
 #include <map>
+#include <functional>
+#include <optional>
 
 #include "auth/AuthSessionHandler.h"
 #include "common/ceph_time.h"
@@ -37,6 +39,7 @@
 class AsyncMessenger;
 class DispatchQueue;
 class Worker;
+class Protocol;
 
 static const int ASYNC_IOV_MAX = (IOV_MAX >= 1024 ? IOV_MAX / 4 : IOV_MAX);
 
@@ -49,87 +52,22 @@ static const int ASYNC_IOV_MAX = (IOV_MAX >= 1024 ? IOV_MAX / 4 : IOV_MAX);
  */
 class AsyncConnection : public Connection {
 
+  void read(unsigned len, char *buffer,
+            std::function<void(char *, ssize_t)> callback);
+  ssize_t read_until(unsigned needed, char *p);
   ssize_t read_bulk(char *buf, unsigned len);
-  ssize_t do_sendmsg(struct msghdr &msg, unsigned len, bool more);
-  ssize_t try_send(bufferlist &bl, bool more=false) {
-    std::lock_guard<std::mutex> l(write_lock);
-    outcoming_bl.claim_append(bl);
-    return _try_send(more);
-  }
+
+  void write(bufferlist &bl, std::function<void(ssize_t)> callback,
+             bool more=false);
   ssize_t _try_send(bool more=false);
-  ssize_t _send(Message *m);
-  void prepare_send_message(uint64_t features, Message *m, bufferlist &bl);
-  ssize_t read_until(unsigned needed, char *p);
-  ssize_t _process_connection();
+
   void _connect();
   void _stop();
-  int handle_connect_reply(ceph_msg_connect &connect, ceph_msg_connect_reply &r);
-  ssize_t handle_connect_msg(ceph_msg_connect &m, bufferlist &aubl, bufferlist &bl);
-  void was_session_reset();
   void fault();
-  void discard_out_queue();
-  void discard_requeued_up_to(uint64_t seq);
-  void requeue_sent();
-  void randomize_out_seq();
-  void handle_ack(uint64_t seq);
-  void _append_keepalive_or_ack(bool ack=false, utime_t *t=NULL);
-  ssize_t write_message(Message *m, bufferlist& bl, bool more);
   void inject_delay();
-  ssize_t _reply_accept(char tag, ceph_msg_connect &connect, ceph_msg_connect_reply &reply,
-                    bufferlist &authorizer_reply) {
-    bufferlist reply_bl;
-    reply.tag = tag;
-    reply.features = ((uint64_t)connect.features & policy.features_supported) | policy.features_required;
-    reply.authorizer_len = authorizer_reply.length();
-    reply_bl.append((char*)&reply, sizeof(reply));
-    if (reply.authorizer_len) {
-      reply_bl.append(authorizer_reply.c_str(), authorizer_reply.length());
-    }
-    ssize_t r = try_send(reply_bl);
-    if (r < 0) {
-      inject_delay();
-      return -1;
-    }
 
-    state = STATE_ACCEPTING_WAIT_CONNECT_MSG;
-    return 0;
-  }
-  bool is_queued() const {
-    return !out_q.empty() || outcoming_bl.length();
-  }
-  void shutdown_socket() {
-    for (auto &&t : register_time_events)
-      center->delete_time_event(t);
-    register_time_events.clear();
-    if (last_tick_id) {
-      center->delete_time_event(last_tick_id);
-      last_tick_id = 0;
-    }
-    if (cs) {
-      center->delete_file_event(cs.fd(), EVENT_READABLE|EVENT_WRITABLE);
-      cs.shutdown();
-      cs.close();
-    }
-  }
-  Message *_get_next_outgoing(bufferlist *bl) {
-    Message *m = 0;
-    if (!out_q.empty()) {
-      map<int, list<pair<bufferlist, Message*> > >::reverse_iterator it = out_q.rbegin();
-      ceph_assert(!it->second.empty());
-      list<pair<bufferlist, Message*> >::iterator p = it->second.begin();
-      m = p->second;
-      if (bl)
-       bl->swap(p->first);
-      it->second.erase(p);
-      if (it->second.empty())
-       out_q.erase(it->first);
-    }
-    return m;
-  }
-  bool _has_next_outgoing() const {
-    return !out_q.empty();
-  }
-  void reset_recv_state();
+  bool is_queued() const;
+  void shutdown_socket();
 
    /**
    * The DelayedDelivery is for injecting delays into Message delivery off
@@ -170,24 +108,17 @@ class AsyncConnection : public Connection {
 
  public:
   AsyncConnection(CephContext *cct, AsyncMessenger *m, DispatchQueue *q,
-                 Worker *w, bool is_msgr2);
+                 Worker *w, bool is_msgr2, bool local);
   ~AsyncConnection() override;
   void maybe_start_delay_thread();
 
   ostream& _conn_prefix(std::ostream *_dout);
 
-  bool is_connected() override {
-    return can_write.load() == WriteStatus::CANWRITE;
-  }
+  bool is_connected() override;
 
   // Only call when AsyncConnection first construct
-  void connect(const entity_addrvec_t& addrs, int type, entity_addr_t& target) {
-    set_peer_type(type);
-    set_peer_addrs(addrs);
-    policy = msgr->get_policy(type);
-    target_addr = target;
-    _connect();
-  }
+  void connect(const entity_addrvec_t& addrs, int type, entity_addr_t& target);
+
   // Only call when AsyncConnection first construct
   void accept(ConnectedSocket socket, entity_addr_t &addr);
   int send_message(Message *m) override;
@@ -198,94 +129,36 @@ class AsyncConnection : public Connection {
     std::lock_guard<std::mutex> l(lock);
     policy.lossy = true;
   }
-  entity_addr_t get_peer_socket_addr() const override {
-    return target_addr;
-  }
-  
+
+ entity_addr_t get_peer_socket_addr() const override {
+   return target_addr;
+ }
+
  private:
   enum {
     STATE_NONE,
-    STATE_OPEN,
-    STATE_OPEN_KEEPALIVE2,
-    STATE_OPEN_KEEPALIVE2_ACK,
-    STATE_OPEN_TAG_ACK,
-    STATE_OPEN_MESSAGE_HEADER,
-    STATE_OPEN_MESSAGE_THROTTLE_MESSAGE,
-    STATE_OPEN_MESSAGE_THROTTLE_BYTES,
-    STATE_OPEN_MESSAGE_THROTTLE_DISPATCH_QUEUE,
-    STATE_OPEN_MESSAGE_READ_FRONT,
-    STATE_OPEN_MESSAGE_READ_MIDDLE,
-    STATE_OPEN_MESSAGE_READ_DATA_PREPARE,
-    STATE_OPEN_MESSAGE_READ_DATA,
-    STATE_OPEN_MESSAGE_READ_FOOTER_AND_DISPATCH,
-    STATE_OPEN_TAG_CLOSE,
-    STATE_WAIT_SEND,
     STATE_CONNECTING,
     STATE_CONNECTING_RE,
-    STATE_CONNECTING_WAIT_BANNER_AND_IDENTIFY,
-    STATE_CONNECTING_SEND_CONNECT_MSG,
-    STATE_CONNECTING_WAIT_CONNECT_REPLY,
-    STATE_CONNECTING_WAIT_CONNECT_REPLY_AUTH,
-    STATE_CONNECTING_WAIT_ACK_SEQ,
-    STATE_CONNECTING_READY,
     STATE_ACCEPTING,
-    STATE_ACCEPTING_WAIT_BANNER_ADDR,
-    STATE_ACCEPTING_WAIT_CONNECT_MSG,
-    STATE_ACCEPTING_WAIT_CONNECT_MSG_AUTH,
-    STATE_ACCEPTING_WAIT_SEQ,
-    STATE_ACCEPTING_READY,
-    STATE_STANDBY,
-    STATE_CLOSED,
-    STATE_WAIT,       // just wait for racing connection
+    STATE_CONNECTION_ESTABLISHED,
+    STATE_CLOSED
   };
 
   static const uint32_t TCP_PREFETCH_MIN_SIZE;
   static const char *get_state_name(int state) {
       const char* const statenames[] = {"STATE_NONE",
-                                        "STATE_OPEN",
-                                        "STATE_OPEN_KEEPALIVE2",
-                                        "STATE_OPEN_KEEPALIVE2_ACK",
-                                        "STATE_OPEN_TAG_ACK",
-                                        "STATE_OPEN_MESSAGE_HEADER",
-                                        "STATE_OPEN_MESSAGE_THROTTLE_MESSAGE",
-                                        "STATE_OPEN_MESSAGE_THROTTLE_BYTES",
-                                        "STATE_OPEN_MESSAGE_THROTTLE_DISPATCH_QUEUE",
-                                        "STATE_OPEN_MESSAGE_READ_FRONT",
-                                        "STATE_OPEN_MESSAGE_READ_MIDDLE",
-                                        "STATE_OPEN_MESSAGE_READ_DATA_PREPARE",
-                                        "STATE_OPEN_MESSAGE_READ_DATA",
-                                        "STATE_OPEN_MESSAGE_READ_FOOTER_AND_DISPATCH",
-                                        "STATE_OPEN_TAG_CLOSE",
-                                        "STATE_WAIT_SEND",
                                         "STATE_CONNECTING",
                                         "STATE_CONNECTING_RE",
-                                        "STATE_CONNECTING_WAIT_BANNER_AND_IDENTIFY",
-                                        "STATE_CONNECTING_SEND_CONNECT_MSG",
-                                        "STATE_CONNECTING_WAIT_CONNECT_REPLY",
-                                        "STATE_CONNECTING_WAIT_CONNECT_REPLY_AUTH",
-                                        "STATE_CONNECTING_WAIT_ACK_SEQ",
-                                        "STATE_CONNECTING_READY",
                                         "STATE_ACCEPTING",
-                                        "STATE_ACCEPTING_WAIT_BANNER_ADDR",
-                                        "STATE_ACCEPTING_WAIT_CONNECT_MSG",
-                                        "STATE_ACCEPTING_WAIT_CONNECT_MSG_AUTH",
-                                        "STATE_ACCEPTING_WAIT_SEQ",
-                                        "STATE_ACCEPTING_READY",
-                                        "STATE_STANDBY",
-                                        "STATE_CLOSED",
-                                        "STATE_WAIT"};
+                                        "STATE_CONNECTION_ESTABLISHED",
+                                        "STATE_CLOSED"};
       return statenames[state];
   }
 
   AsyncMessenger *async_msgr;
   uint64_t conn_id;
   PerfCounters *logger;
-  int global_seq;
-  __u32 connect_seq, peer_global_seq;
-  std::atomic<uint64_t> out_seq{0};
-  std::atomic<uint64_t> ack_left{0}, in_seq{0};
   int state;
-  int state_after_send;
   ConnectedSocket cs;
   int port;
   Messenger::Policy policy;
@@ -297,21 +170,11 @@ class AsyncConnection : public Connection {
   bool open_write = false;
 
   std::mutex write_lock;
-  enum class WriteStatus {
-    NOWRITE,
-    REPLACING,
-    CANWRITE,
-    CLOSED
-  };
-  std::atomic<WriteStatus> can_write;
-  list<Message*> sent; // the first bufferlist need to inject seq
-  map<int, list<pair<bufferlist, Message*> > > out_q;  // priority queue for outbound msgs
-  bool keepalive;
 
   std::mutex lock;
-  utime_t backoff;         // backoff time
   EventCallbackRef read_handler;
   EventCallbackRef write_handler;
+  EventCallbackRef write_callback_handler;
   EventCallbackRef wakeup_handler;
   EventCallbackRef tick_handler;
   char *recv_buf;
@@ -320,71 +183,45 @@ class AsyncConnection : public Connection {
   uint32_t recv_end;
   set<uint64_t> register_time_events; // need to delete it if stop
   ceph::coarse_mono_clock::time_point last_active;
+  ceph::mono_clock::time_point recv_start_time;
   uint64_t last_tick_id = 0;
   const uint64_t inactive_timeout_us;
 
   // Tis section are temp variables used by state transition
 
-  // Open state
-  utime_t recv_stamp;
-  utime_t throttle_stamp;
-  unsigned msg_left;
-  uint64_t cur_msg_size;
-  ceph_msg_header current_header;
-  bufferlist data_buf;
-  bufferlist::iterator data_blp;
-  bufferlist front, middle, data;
-  ceph_msg_connect connect_msg;
-  // Connecting state
-  bool got_bad_auth;
-  AuthAuthorizer *authorizer;
-  bufferlist authorizer_buf;
-  ceph_msg_connect_reply connect_reply;
   // Accepting state
   bool msgr2 = false;
   entity_addr_t socket_addr;
   entity_addr_t target_addr;  // which of the peer_addrs we're using
-  CryptoKey session_key;
-  bool replacing;    // when replacing process happened, we will reply connect
-                     // side with RETRY tag and accept side will clear replaced
-                     // connection. So when connect side reissue connect_msg,
-                     // there won't exists conflicting connection so we use
-                     // "replacing" to skip RESETSESSION to avoid detect wrong
-                     // presentation
-  bool is_reset_from_peer;
-  bool once_ready;
-
-  // used only for local state, it will be overwrite when state transition
-  char *state_buffer;
+
   // used only by "read_until"
   uint64_t state_offset;
   Worker *worker;
   EventCenter *center;
-  std::shared_ptr<AuthSessionHandler> session_security;
-  std::unique_ptr<AuthAuthorizerChallenge> authorizer_challenge; // accept side
+
+  std::unique_ptr<Protocol> protocol;
+
+  std::optional<std::function<void(ssize_t)>> writeCallback;
+  std::function<void(char *, ssize_t)> readCallback;
+  std::optional<unsigned> pendingReadLen;
+  char *read_buffer;
 
  public:
   // used by eventcallback
   void handle_write();
+  void handle_write_callback();
   void process();
   void wakeup_from(uint64_t id);
   void tick(uint64_t id);
   void local_deliver();
   void stop(bool queue_reset);
-  void cleanup() {
-    shutdown_socket();
-    delete read_handler;
-    delete write_handler;
-    delete wakeup_handler;
-    delete tick_handler;
-    if (delay_state) {
-      delete delay_state;
-      delay_state = NULL;
-    }
-  }
+  void cleanup();
   PerfCounters *get_perf_counter() {
     return logger;
   }
+
+  friend class Protocol;
+  friend class ProtocolV1;
 }; /* AsyncConnection */
 
 typedef boost::intrusive_ptr<AsyncConnection> AsyncConnectionRef;
index accbafed69e469e0938ce1978613581a29e88bcc..959e76f77787e80eb4449b2fa615c9e4b9fdcc26 100644 (file)
@@ -1,4 +1,4 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
 // vim: ts=8 sw=2 smarttab
 /*
  * Ceph - scalable distributed file system
@@ -287,7 +287,7 @@ AsyncMessenger::AsyncMessenger(CephContext *cct, entity_name_t name,
   stack->start();
   local_worker = stack->get_worker();
   local_connection = new AsyncConnection(cct, this, &dispatch_queue,
-                                        local_worker, true);
+                                        local_worker, true, true);
   init_local_connection();
   reap_handler = new C_handle_reap(this);
   unsigned processor_num = 1;
@@ -553,7 +553,7 @@ void AsyncMessenger::add_accept(Worker *w, ConnectedSocket cli_socket, entity_ad
 {
   lock.Lock();
   AsyncConnectionRef conn = new AsyncConnection(cct, this, &dispatch_queue, w,
-                                               addr.is_msgr2());
+                                               addr.is_msgr2(), false);
   conn->accept(std::move(cli_socket), addr);
   accepting_conns.insert(conn);
   lock.Unlock();
@@ -583,7 +583,7 @@ AsyncConnectionRef AsyncMessenger::create_connect(
   // create connection
   Worker *w = stack->get_worker();
   AsyncConnectionRef conn = new AsyncConnection(cct, this, &dispatch_queue, w,
-                                               target.is_msgr2());
+                                               target.is_msgr2(), false);
   conn->connect(addrs, type, target);
   ceph_assert(!conns.count(addrs));
   conns[addrs] = conn;
index 27c228ed9955fc3442a25fa682c7e6bb1aaf0051..e53df24fb572b2f02b6c4f0750cc56d6d8df07d6 100644 (file)
@@ -1,4 +1,4 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
 // vim: ts=8 sw=2 smarttab
 /*
  * Ceph - scalable distributed file system
@@ -336,7 +336,7 @@ private:
 public:
 
   /// con used for sending messages to ourselves
-  ConnectionRef local_connection;
+  AsyncConnectionRef local_connection;
 
   /**
    * @defgroup AsyncMessenger internals
diff --git a/src/msg/async/Protocol.cc b/src/msg/async/Protocol.cc
new file mode 100644 (file)
index 0000000..bcddc69
--- /dev/null
@@ -0,0 +1,2456 @@
+#include "Protocol.h"
+
+#include "common/errno.h"
+
+#include "AsyncConnection.h"
+#include "AsyncMessenger.h"
+#include "common/EventTrace.h"
+#include "include/random.h"
+
+#define dout_subsys ceph_subsys_ms
+#undef dout_prefix
+#define dout_prefix _conn_prefix(_dout)
+ostream &ProtocolV1::_conn_prefix(std::ostream *_dout) {
+  return *_dout << "-- " << messenger->get_myaddrs().legacy_addr() << " >> "
+                << connection->peer_addrs.legacy_addr() << " conn("
+                << connection << (connection->msgr2 ? " msgr2" : " legacy")
+                << " :" << connection->port << " s=" << get_state_name(state)
+                << " pgs=" << peer_global_seq << " cs=" << connect_seq
+                << " l=" << connection->policy.lossy << ").";
+}
+
+#define WRITE(B, F) \
+  connection->write(B, std::bind(F, this, std::placeholders::_1))
+
+#define READ(L, F)    \
+  connection->read(   \
+      L, temp_buffer, \
+      std::bind(F, this, std::placeholders::_1, std::placeholders::_2))
+
+#define READB(L, B, F) \
+  connection->read(    \
+      L, B, std::bind(F, this, std::placeholders::_1, std::placeholders::_2))
+
+// Constant to limit starting sequence number to 2^31.  Nothing special about
+// it, just a big number.  PLR
+#define SEQ_MASK 0x7fffffff
+
+const int ASYNC_COALESCE_THRESHOLD = 256;
+
+using namespace std;
+
+static void alloc_aligned_buffer(bufferlist &data, unsigned len, unsigned off) {
+  // create a buffer to read into that matches the data alignment
+  unsigned alloc_len = 0;
+  unsigned left = len;
+  unsigned head = 0;
+  if (off & ~CEPH_PAGE_MASK) {
+    // head
+    alloc_len += CEPH_PAGE_SIZE;
+    head = std::min<uint64_t>(CEPH_PAGE_SIZE - (off & ~CEPH_PAGE_MASK), left);
+    left -= head;
+  }
+  alloc_len += left;
+  bufferptr ptr(buffer::create_small_page_aligned(alloc_len));
+  if (head) ptr.set_offset(CEPH_PAGE_SIZE - head);
+  data.push_back(std::move(ptr));
+}
+
+Protocol::Protocol(AsyncConnection *connection)
+    : connection(connection),
+      messenger(connection->async_msgr),
+      cct(connection->async_msgr->cct) {}
+
+Protocol::~Protocol() {}
+
+/**
+ * Protocol V1
+ **/
+
+ProtocolV1::ProtocolV1(AsyncConnection *connection)
+    : Protocol(connection),
+      temp_buffer(nullptr),
+      can_write(WriteStatus::NOWRITE),
+      keepalive(false),
+      connect_seq(0),
+      peer_global_seq(0),
+      msg_left(0),
+      cur_msg_size(0),
+      replacing(false),
+      is_reset_from_peer(false),
+      once_ready(false),
+      state(NONE),
+      global_seq(0),
+      got_bad_auth(false),
+      authorizer(nullptr),
+      wait_for_seq(false) {
+  temp_buffer = new char[4096];
+}
+
+ProtocolV1::~ProtocolV1() {
+  ceph_assert(out_q.empty());
+  ceph_assert(sent.empty());
+
+  delete[] temp_buffer;
+
+  if (authorizer) {
+    delete authorizer;
+  }
+}
+
+void ProtocolV1::connect() {
+  this->state = START_CONNECT;
+
+  // reset connect state variables
+  got_bad_auth = false;
+  if (authorizer) {
+    delete authorizer;
+    authorizer = nullptr;
+  }
+  authorizer_buf.clear();
+  memset(&connect_msg, 0, sizeof(connect_msg));
+  memset(&connect_reply, 0, sizeof(connect_reply));
+
+  global_seq = messenger->get_global_seq();
+}
+
+void ProtocolV1::accept() { this->state = START_ACCEPT; }
+
+bool ProtocolV1::is_connected() {
+  return can_write.load() == WriteStatus::CANWRITE;
+}
+
+void ProtocolV1::stop() {
+  ldout(cct, 20) << __func__ << dendl;
+  if (state == CLOSED) {
+    return;
+  }
+
+  if (connection->delay_state) connection->delay_state->flush();
+
+  ldout(cct, 2) << __func__ << dendl;
+  std::lock_guard<std::mutex> l(connection->write_lock);
+
+  reset_recv_state();
+  discard_out_queue();
+
+  connection->_stop();
+
+  can_write = WriteStatus::CLOSED;
+  state = CLOSED;
+}
+
+void ProtocolV1::fault() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  if (state == CLOSED || state == NONE) {
+    ldout(cct, 10) << __func__ << " connection is already closed" << dendl;
+    return;
+  }
+
+  if (connection->policy.lossy && state != START_CONNECT &&
+      state != CONNECTING) {
+    ldout(cct, 1) << __func__ << " on lossy channel, failing" << dendl;
+    stop();
+    connection->dispatch_queue->queue_reset(connection);
+    return;
+  }
+
+  connection->write_lock.lock();
+  can_write = WriteStatus::NOWRITE;
+  is_reset_from_peer = false;
+
+  // requeue sent items
+  requeue_sent();
+
+  if (!once_ready && out_q.empty() && state >= START_ACCEPT &&
+      state <= ACCEPTING_WAIT_CONNECT_MSG_AUTH && !replacing) {
+    ldout(cct, 10) << __func__ << " with nothing to send and in the half "
+                   << " accept state just closed" << dendl;
+    connection->write_lock.unlock();
+    stop();
+    connection->dispatch_queue->queue_reset(connection);
+    return;
+  }
+  replacing = false;
+
+  connection->fault();
+
+  reset_recv_state();
+
+  if (connection->policy.standby && out_q.empty() && state != WAIT) {
+    ldout(cct, 10) << __func__ << " with nothing to send, going to standby"
+                   << dendl;
+    state = STANDBY;
+    connection->write_lock.unlock();
+    return;
+  }
+
+  connection->write_lock.unlock();
+
+  if (state != START_CONNECT && state != CONNECTING && state != WAIT) {
+    // policy maybe empty when state is in accept
+    if (connection->policy.server) {
+      ldout(cct, 0) << __func__ << " server, going to standby" << dendl;
+      state = STANDBY;
+    } else {
+      ldout(cct, 0) << __func__ << " initiating reconnect" << dendl;
+      connect_seq++;
+      state = START_CONNECT;
+      connection->state = AsyncConnection::STATE_CONNECTING;
+    }
+    backoff = utime_t();
+    connection->center->dispatch_event_external(connection->read_handler);
+  } else {
+    if (state == WAIT) {
+      backoff.set_from_double(cct->_conf->ms_max_backoff);
+    } else if (backoff == utime_t()) {
+      backoff.set_from_double(cct->_conf->ms_initial_backoff);
+    } else {
+      backoff += backoff;
+      if (backoff > cct->_conf->ms_max_backoff)
+        backoff.set_from_double(cct->_conf->ms_max_backoff);
+    }
+
+    state = START_CONNECT;
+    connection->state = AsyncConnection::STATE_CONNECTING;
+    ldout(cct, 10) << __func__ << " waiting " << backoff << dendl;
+    // woke up again;
+    connection->register_time_events.insert(
+        connection->center->create_time_event(backoff.to_nsec() / 1000,
+                                              connection->wakeup_handler));
+  }
+}
+
+void ProtocolV1::send_message(Message *m) {
+  bufferlist bl;
+  uint64_t f = connection->get_features();
+
+  // TODO: Currently not all messages supports reencode like MOSDMap, so here
+  // only let fast dispatch support messages prepare message
+  bool can_fast_prepare = messenger->ms_can_fast_dispatch(m);
+  if (can_fast_prepare) {
+    prepare_send_message(f, m, bl);
+  }
+
+  std::lock_guard<std::mutex> l(connection->write_lock);
+  // "features" changes will change the payload encoding
+  if (can_fast_prepare &&
+      (can_write == WriteStatus::NOWRITE || connection->get_features() != f)) {
+    // ensure the correctness of message encoding
+    bl.clear();
+    m->get_payload().clear();
+    ldout(cct, 5) << __func__ << " clear encoded buffer previous " << f
+                  << " != " << connection->get_features() << dendl;
+  }
+  if (can_write == WriteStatus::CLOSED) {
+    ldout(cct, 10) << __func__ << " connection closed."
+                   << " Drop message " << m << dendl;
+    m->put();
+  } else {
+    m->trace.event("async enqueueing message");
+    out_q[m->get_priority()].emplace_back(std::move(bl), m);
+    ldout(cct, 15) << __func__ << " inline write is denied, reschedule m=" << m
+                   << dendl;
+    if (can_write != WriteStatus::REPLACING) {
+      connection->center->dispatch_event_external(connection->write_handler);
+    }
+  }
+}
+
+void ProtocolV1::prepare_send_message(uint64_t features, Message *m,
+                                      bufferlist &bl) {
+  ldout(cct, 20) << __func__ << " m " << *m << dendl;
+
+  // associate message with Connection (for benefit of encode_payload)
+  if (m->empty_payload()) {
+    ldout(cct, 20) << __func__ << " encoding features " << features << " " << m
+                   << " " << *m << dendl;
+  } else {
+    ldout(cct, 20) << __func__ << " half-reencoding features " << features
+                   << " " << m << " " << *m << dendl;
+  }
+
+  // encode and copy out of *m
+  m->encode(features, messenger->crcflags);
+
+  bl.append(m->get_payload());
+  bl.append(m->get_middle());
+  bl.append(m->get_data());
+}
+
+void ProtocolV1::send_keepalive() {
+  ldout(cct, 10) << __func__ << dendl;
+  std::lock_guard<std::mutex> l(connection->write_lock);
+  if (can_write != WriteStatus::CLOSED) {
+    keepalive = true;
+    connection->center->dispatch_event_external(connection->write_handler);
+  }
+}
+
+void ProtocolV1::read_event() {
+  ldout(cct, 20) << __func__ << dendl;
+  switch (state) {
+    case START_CONNECT:
+      send_client_banner();
+      break;
+    case START_ACCEPT:
+      send_server_banner();
+      break;
+    case OPENED:
+      wait_message();
+      break;
+    case THROTTLE_MESSAGE:
+      throttle_message();
+      break;
+    case THROTTLE_BYTES:
+      throttle_bytes();
+      break;
+    case THROTTLE_DISPATCH_QUEUE:
+      throttle_dispatch_queue();
+      break;
+    default:
+      break;
+  }
+}
+
+void ProtocolV1::write_event() {
+  ldout(cct, 10) << __func__ << dendl;
+  ssize_t r = 0;
+
+  connection->write_lock.lock();
+  if (can_write == WriteStatus::CANWRITE) {
+    if (keepalive) {
+      append_keepalive_or_ack();
+      keepalive = false;
+    }
+
+    auto start = ceph::mono_clock::now();
+    bool more;
+    do {
+      bufferlist data;
+      Message *m = _get_next_outgoing(&data);
+      if (!m) {
+        break;
+      }
+
+      if (!connection->policy.lossy) {
+        // put on sent list
+        sent.push_back(m);
+        m->get();
+      }
+      more = !out_q.empty();
+      connection->write_lock.unlock();
+
+      // send_message or requeue messages may not encode message
+      if (!data.length()) {
+        prepare_send_message(connection->get_features(), m, data);
+      }
+
+      r = write_message(m, data, more);
+
+      connection->write_lock.lock();
+      if (r == 0) {
+        ;
+      } else if (r < 0) {
+        ldout(cct, 1) << __func__ << " send msg failed" << dendl;
+        break;
+      } else if (r > 0)
+        break;
+    } while (can_write == WriteStatus::CANWRITE);
+    connection->write_lock.unlock();
+
+    // if r > 0 mean data still lefted, so no need _try_send.
+    if (r == 0) {
+      uint64_t left = ack_left;
+      if (left) {
+        ceph_le64 s;
+        s = in_seq;
+        connection->outcoming_bl.append(CEPH_MSGR_TAG_ACK);
+        connection->outcoming_bl.append((char *)&s, sizeof(s));
+        ldout(cct, 10) << __func__ << " try send msg ack, acked " << left
+                       << " messages" << dendl;
+        ack_left -= left;
+        left = ack_left;
+        r = connection->_try_send(left);
+      } else if (connection->is_queued()) {
+        r = connection->_try_send();
+      }
+    }
+
+    connection->logger->tinc(l_msgr_running_send_time,
+                             ceph::mono_clock::now() - start);
+    if (r < 0) {
+      ldout(cct, 1) << __func__ << " send msg failed" << dendl;
+      connection->lock.lock();
+      fault();
+      connection->lock.unlock();
+      return;
+    }
+  } else {
+    connection->write_lock.unlock();
+    connection->lock.lock();
+    connection->write_lock.lock();
+    if (state == STANDBY && !connection->policy.server &&
+        connection->is_queued()) {
+      ldout(cct, 10) << __func__ << " policy.server is false" << dendl;
+      connection->_connect();
+    } else if (connection->cs && state != NONE && state != CLOSED &&
+               state != START_CONNECT) {
+      r = connection->_try_send();
+      if (r < 0) {
+        ldout(cct, 1) << __func__ << " send outcoming bl failed" << dendl;
+        connection->write_lock.unlock();
+        fault();
+        connection->lock.unlock();
+        return;
+      }
+    }
+    connection->write_lock.unlock();
+    connection->lock.unlock();
+  }
+}
+
+bool ProtocolV1::is_queued() { return !out_q.empty(); }
+
+void ProtocolV1::ready() {
+  ldout(cct, 25) << __func__ << dendl;
+
+  // make sure no pending tick timer
+  if (connection->last_tick_id) {
+    connection->center->delete_time_event(connection->last_tick_id);
+  }
+  connection->last_tick_id = connection->center->create_time_event(
+      connection->inactive_timeout_us, connection->tick_handler);
+
+  connection->write_lock.lock();
+  can_write = WriteStatus::CANWRITE;
+  if (connection->is_queued()) {
+    connection->center->dispatch_event_external(connection->write_handler);
+  }
+  connection->write_lock.unlock();
+  connection->maybe_start_delay_thread();
+
+  state = OPENED;
+  wait_message();
+}
+
+void ProtocolV1::wait_message() {
+  if (state != OPENED) {  // must have changed due to a replace
+    return;
+  }
+
+  ldout(cct, 20) << __func__ << dendl;
+
+  READ(sizeof(char), &ProtocolV1::handle_message);
+}
+
+void ProtocolV1::handle_message(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read tag failed" << dendl;
+    fault();
+    return;
+  }
+
+  char tag = buffer[0];
+  ldout(cct, 20) << __func__ << " process tag " << (int)tag << dendl;
+
+  if (tag == CEPH_MSGR_TAG_KEEPALIVE) {
+    ldout(cct, 20) << __func__ << " got KEEPALIVE" << dendl;
+    connection->set_last_keepalive(ceph_clock_now());
+  } else if (tag == CEPH_MSGR_TAG_KEEPALIVE2) {
+    READ(sizeof(ceph_timespec), &ProtocolV1::handle_keepalive2);
+  } else if (tag == CEPH_MSGR_TAG_KEEPALIVE2_ACK) {
+    READ(sizeof(ceph_timespec), &ProtocolV1::handle_keepalive2_ack);
+  } else if (tag == CEPH_MSGR_TAG_ACK) {
+    READ(sizeof(ceph_le64), &ProtocolV1::handle_tag_ack);
+  } else if (tag == CEPH_MSGR_TAG_MSG) {
+#if defined(WITH_LTTNG) && defined(WITH_EVENTTRACE)
+    ltt_recv_stamp = ceph_clock_now();
+#endif
+    recv_stamp = ceph_clock_now();
+    ldout(cct, 20) << __func__ << " begin MSG" << dendl;
+    READ(sizeof(ceph_msg_header), &ProtocolV1::handle_message_header);
+  } else if (tag == CEPH_MSGR_TAG_CLOSE) {
+    ldout(cct, 20) << __func__ << " got CLOSE" << dendl;
+    stop();
+  } else {
+    ldout(cct, 0) << __func__ << " bad tag " << (int)tag << dendl;
+    fault();
+  }
+}
+
+void ProtocolV1::handle_keepalive2(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read keeplive timespec failed" << dendl;
+    fault();
+    return;
+  }
+
+  ldout(cct, 30) << __func__ << " got KEEPALIVE2 tag ..." << dendl;
+
+  ceph_timespec *t;
+  t = (ceph_timespec *)buffer;
+  utime_t kp_t = utime_t(*t);
+  connection->write_lock.lock();
+  append_keepalive_or_ack(true, &kp_t);
+  connection->write_lock.unlock();
+
+  ldout(cct, 20) << __func__ << " got KEEPALIVE2 " << kp_t << dendl;
+  connection->set_last_keepalive(ceph_clock_now());
+
+  if (is_connected()) {
+    connection->center->dispatch_event_external(connection->write_handler);
+  }
+
+  wait_message();
+}
+
+void ProtocolV1::append_keepalive_or_ack(bool ack, utime_t *tp) {
+  ldout(cct, 10) << __func__ << dendl;
+  if (ack) {
+    ceph_assert(tp);
+    struct ceph_timespec ts;
+    tp->encode_timeval(&ts);
+    connection->outcoming_bl.append(CEPH_MSGR_TAG_KEEPALIVE2_ACK);
+    connection->outcoming_bl.append((char *)&ts, sizeof(ts));
+  } else if (connection->has_feature(CEPH_FEATURE_MSGR_KEEPALIVE2)) {
+    struct ceph_timespec ts;
+    utime_t t = ceph_clock_now();
+    t.encode_timeval(&ts);
+    connection->outcoming_bl.append(CEPH_MSGR_TAG_KEEPALIVE2);
+    connection->outcoming_bl.append((char *)&ts, sizeof(ts));
+  } else {
+    connection->outcoming_bl.append(CEPH_MSGR_TAG_KEEPALIVE);
+  }
+}
+
+void ProtocolV1::handle_keepalive2_ack(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read keeplive timespec failed" << dendl;
+    fault();
+    return;
+  }
+
+  ceph_timespec *t;
+  t = (ceph_timespec *)buffer;
+  connection->set_last_keepalive_ack(utime_t(*t));
+  ldout(cct, 20) << __func__ << " got KEEPALIVE_ACK" << dendl;
+
+  wait_message();
+}
+
+void ProtocolV1::handle_tag_ack(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read ack seq failed" << dendl;
+    fault();
+    return;
+  }
+
+  ceph_le64 seq;
+  seq = *(ceph_le64 *)buffer;
+  ldout(cct, 20) << __func__ << " got ACK" << dendl;
+
+  ldout(cct, 15) << __func__ << " got ack seq " << seq << dendl;
+  // trim sent list
+  static const int max_pending = 128;
+  int i = 0;
+  Message *pending[max_pending];
+  connection->write_lock.lock();
+  while (!sent.empty() && sent.front()->get_seq() <= seq && i < max_pending) {
+    Message *m = sent.front();
+    sent.pop_front();
+    pending[i++] = m;
+    ldout(cct, 10) << __func__ << " got ack seq " << seq
+                   << " >= " << m->get_seq() << " on " << m << " " << *m
+                   << dendl;
+  }
+  connection->write_lock.unlock();
+  for (int k = 0; k < i; k++) {
+    pending[k]->put();
+  }
+
+  wait_message();
+}
+
+void ProtocolV1::handle_message_header(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read message header failed" << dendl;
+    fault();
+    return;
+  }
+
+  ldout(cct, 20) << __func__ << " got MSG header" << dendl;
+
+  ceph_msg_header header;
+  header = *((ceph_msg_header *)buffer);
+
+  ldout(cct, 20) << __func__ << " got envelope type=" << header.type << " src "
+                 << entity_name_t(header.src) << " front=" << header.front_len
+                 << " data=" << header.data_len << " off " << header.data_off
+                 << dendl;
+
+  if (messenger->crcflags & MSG_CRC_HEADER) {
+    __u32 header_crc = 0;
+    header_crc = ceph_crc32c(0, (unsigned char *)&header,
+                             sizeof(header) - sizeof(header.crc));
+    // verify header crc
+    if (header_crc != header.crc) {
+      ldout(cct, 0) << __func__ << " got bad header crc " << header_crc
+                    << " != " << header.crc << dendl;
+      fault();
+      return;
+    }
+  }
+
+  // Reset state
+  data_buf.clear();
+  front.clear();
+  middle.clear();
+  data.clear();
+  current_header = header;
+
+  state = THROTTLE_MESSAGE;
+  throttle_message();
+}
+
+void ProtocolV1::throttle_message() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  if (connection->policy.throttler_messages) {
+    ldout(cct, 10) << __func__ << " wants " << 1
+                   << " message from policy throttler "
+                   << connection->policy.throttler_messages->get_current()
+                   << "/" << connection->policy.throttler_messages->get_max()
+                   << dendl;
+    if (!connection->policy.throttler_messages->get_or_fail()) {
+      ldout(cct, 10) << __func__ << " wants 1 message from policy throttle "
+                     << connection->policy.throttler_messages->get_current()
+                     << "/" << connection->policy.throttler_messages->get_max()
+                     << " failed, just wait." << dendl;
+      // following thread pool deal with th full message queue isn't a
+      // short time, so we can wait a ms.
+      if (connection->register_time_events.empty()) {
+        connection->register_time_events.insert(
+            connection->center->create_time_event(1000,
+                                                  connection->wakeup_handler));
+      }
+      return;
+    }
+  }
+
+  state = THROTTLE_BYTES;
+  throttle_bytes();
+}
+
+void ProtocolV1::throttle_bytes() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  cur_msg_size = current_header.front_len + current_header.middle_len +
+                 current_header.data_len;
+  if (cur_msg_size) {
+    if (connection->policy.throttler_bytes) {
+      ldout(cct, 10) << __func__ << " wants " << cur_msg_size
+                     << " bytes from policy throttler "
+                     << connection->policy.throttler_bytes->get_current() << "/"
+                     << connection->policy.throttler_bytes->get_max() << dendl;
+      if (!connection->policy.throttler_bytes->get_or_fail(cur_msg_size)) {
+        ldout(cct, 10) << __func__ << " wants " << cur_msg_size
+                       << " bytes from policy throttler "
+                       << connection->policy.throttler_bytes->get_current()
+                       << "/" << connection->policy.throttler_bytes->get_max()
+                       << " failed, just wait." << dendl;
+        // following thread pool deal with th full message queue isn't a
+        // short time, so we can wait a ms.
+        if (connection->register_time_events.empty()) {
+          connection->register_time_events.insert(
+              connection->center->create_time_event(
+                  1000, connection->wakeup_handler));
+        }
+        return;
+      }
+    }
+  }
+
+  state = THROTTLE_DISPATCH_QUEUE;
+  throttle_dispatch_queue();
+}
+
+void ProtocolV1::throttle_dispatch_queue() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  if (cur_msg_size) {
+    if (!connection->dispatch_queue->dispatch_throttler.get_or_fail(
+            cur_msg_size)) {
+      ldout(cct, 10)
+          << __func__ << " wants " << cur_msg_size
+          << " bytes from dispatch throttle "
+          << connection->dispatch_queue->dispatch_throttler.get_current() << "/"
+          << connection->dispatch_queue->dispatch_throttler.get_max()
+          << " failed, just wait." << dendl;
+      // following thread pool deal with th full message queue isn't a
+      // short time, so we can wait a ms.
+      if (connection->register_time_events.empty()) {
+        connection->register_time_events.insert(
+            connection->center->create_time_event(1000,
+                                                  connection->wakeup_handler));
+      }
+      return;
+    }
+  }
+
+  throttle_stamp = ceph_clock_now();
+
+  state = READ_MESSAGE_FRONT;
+  read_message_front();
+}
+
+void ProtocolV1::read_message_front() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  unsigned front_len = current_header.front_len;
+  if (front_len) {
+    if (!front.length()) {
+      front.push_back(buffer::create(front_len));
+    }
+    READB(front_len, front.c_str(), &ProtocolV1::handle_message_front);
+  } else {
+    read_message_middle();
+  }
+}
+
+void ProtocolV1::handle_message_front(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read message front failed" << dendl;
+    fault();
+    return;
+  }
+
+  ldout(cct, 20) << __func__ << " got front " << front.length() << dendl;
+
+  read_message_middle();
+}
+
+void ProtocolV1::read_message_middle() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  if (current_header.middle_len) {
+    if (!middle.length()) {
+      middle.push_back(buffer::create(current_header.middle_len));
+    }
+    READB(current_header.middle_len, middle.c_str(),
+          &ProtocolV1::handle_message_middle);
+  } else {
+    read_message_data_prepare();
+  }
+}
+
+void ProtocolV1::handle_message_middle(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read message middle failed" << dendl;
+    fault();
+    return;
+  }
+
+  ldout(cct, 20) << __func__ << " got middle " << middle.length() << dendl;
+
+  read_message_data_prepare();
+}
+
+void ProtocolV1::read_message_data_prepare() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  unsigned data_len = le32_to_cpu(current_header.data_len);
+  unsigned data_off = le32_to_cpu(current_header.data_off);
+
+  if (data_len) {
+    // get a buffer
+    map<ceph_tid_t, pair<bufferlist, int> >::iterator p =
+        connection->rx_buffers.find(current_header.tid);
+    if (p != connection->rx_buffers.end()) {
+      ldout(cct, 10) << __func__ << " seleting rx buffer v " << p->second.second
+                     << " at offset " << data_off << " len "
+                     << p->second.first.length() << dendl;
+      data_buf = p->second.first;
+      // make sure it's big enough
+      if (data_buf.length() < data_len)
+        data_buf.push_back(buffer::create(data_len - data_buf.length()));
+      data_blp = data_buf.begin();
+    } else {
+      ldout(cct, 20) << __func__ << " allocating new rx buffer at offset "
+                     << data_off << dendl;
+      alloc_aligned_buffer(data_buf, data_len, data_off);
+      data_blp = data_buf.begin();
+    }
+  }
+
+  msg_left = data_len;
+
+  read_message_data();
+}
+
+void ProtocolV1::read_message_data() {
+  ldout(cct, 20) << __func__ << " msg_left=" << msg_left << dendl;
+
+  if (msg_left > 0) {
+    bufferptr bp = data_blp.get_current_ptr();
+    unsigned read_len = std::min(bp.length(), msg_left);
+
+    READB(read_len, bp.c_str(), &ProtocolV1::handle_message_data);
+  } else {
+    read_message_footer();
+  }
+}
+
+void ProtocolV1::handle_message_data(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read data error " << dendl;
+    fault();
+    return;
+  }
+
+  bufferptr bp = data_blp.get_current_ptr();
+  unsigned read_len = std::min(bp.length(), msg_left);
+  data_blp.advance(read_len);
+  data.append(bp, 0, read_len);
+  msg_left -= read_len;
+
+  read_message_data();
+}
+
+void ProtocolV1::read_message_footer() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  state = READ_FOOTER_AND_DISPATCH;
+
+  unsigned len;
+  if (connection->has_feature(CEPH_FEATURE_MSG_AUTH)) {
+    len = sizeof(ceph_msg_footer);
+  } else {
+    len = sizeof(ceph_msg_footer_old);
+  }
+
+  READ(len, &ProtocolV1::handle_message_footer);
+}
+
+void ProtocolV1::handle_message_footer(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read footer data error " << dendl;
+    fault();
+    return;
+  }
+
+  ceph_msg_footer footer;
+  ceph_msg_footer_old old_footer;
+
+  if (connection->has_feature(CEPH_FEATURE_MSG_AUTH)) {
+    footer = *((ceph_msg_footer *)buffer);
+  } else {
+    old_footer = *((ceph_msg_footer_old *)buffer);
+    footer.front_crc = old_footer.front_crc;
+    footer.middle_crc = old_footer.middle_crc;
+    footer.data_crc = old_footer.data_crc;
+    footer.sig = 0;
+    footer.flags = old_footer.flags;
+  }
+
+  int aborted = (footer.flags & CEPH_MSG_FOOTER_COMPLETE) == 0;
+  ldout(cct, 10) << __func__ << " aborted = " << aborted << dendl;
+  if (aborted) {
+    ldout(cct, 0) << __func__ << " got " << front.length() << " + "
+                  << middle.length() << " + " << data.length()
+                  << " byte message.. ABORTED" << dendl;
+    fault();
+    return;
+  }
+
+  ldout(cct, 20) << __func__ << " got " << front.length() << " + "
+                 << middle.length() << " + " << data.length() << " byte message"
+                 << dendl;
+  Message *message = decode_message(cct, messenger->crcflags, current_header,
+                                    footer, front, middle, data, connection);
+  if (!message) {
+    ldout(cct, 1) << __func__ << " decode message failed " << dendl;
+    fault();
+    return;
+  }
+
+  //
+  //  Check the signature if one should be present.  A zero return indicates
+  //  success. PLR
+  //
+
+  if (session_security.get() == NULL) {
+    ldout(cct, 10) << __func__ << " no session security set" << dendl;
+  } else {
+    if (session_security->check_message_signature(message)) {
+      ldout(cct, 0) << __func__ << " Signature check failed" << dendl;
+      message->put();
+      fault();
+      return;
+    }
+  }
+  message->set_byte_throttler(connection->policy.throttler_bytes);
+  message->set_message_throttler(connection->policy.throttler_messages);
+
+  // store reservation size in message, so we don't get confused
+  // by messages entering the dispatch queue through other paths.
+  message->set_dispatch_throttle_size(cur_msg_size);
+
+  message->set_recv_stamp(recv_stamp);
+  message->set_throttle_stamp(throttle_stamp);
+  message->set_recv_complete_stamp(ceph_clock_now());
+
+  // check received seq#.  if it is old, drop the message.
+  // note that incoming messages may skip ahead.  this is convenient for the
+  // client side queueing because messages can't be renumbered, but the (kernel)
+  // client will occasionally pull a message out of the sent queue to send
+  // elsewhere.  in that case it doesn't matter if we "got" it or not.
+  uint64_t cur_seq = in_seq;
+  if (message->get_seq() <= cur_seq) {
+    ldout(cct, 0) << __func__ << " got old message " << message->get_seq()
+                  << " <= " << cur_seq << " " << message << " " << *message
+                  << ", discarding" << dendl;
+    message->put();
+    if (connection->has_feature(CEPH_FEATURE_RECONNECT_SEQ) &&
+        cct->_conf->ms_die_on_old_message) {
+      ceph_assert(0 == "old msgs despite reconnect_seq feature");
+    }
+    return;
+  }
+  if (message->get_seq() > cur_seq + 1) {
+    ldout(cct, 0) << __func__ << " missed message?  skipped from seq "
+                  << cur_seq << " to " << message->get_seq() << dendl;
+    if (cct->_conf->ms_die_on_skipped_message) {
+      ceph_assert(0 == "skipped incoming seq");
+    }
+  }
+
+  message->set_connection(connection);
+
+#if defined(WITH_LTTNG) && defined(WITH_EVENTTRACE)
+  if (message->get_type() == CEPH_MSG_OSD_OP ||
+      message->get_type() == CEPH_MSG_OSD_OPREPLY) {
+    utime_t ltt_processed_stamp = ceph_clock_now();
+    double usecs_elapsed =
+        (ltt_processed_stamp.to_nsec() - ltt_recv_stamp.to_nsec()) / 1000;
+    ostringstream buf;
+    if (message->get_type() == CEPH_MSG_OSD_OP)
+      OID_ELAPSED_WITH_MSG(message, usecs_elapsed, "TIME_TO_DECODE_OSD_OP",
+                           false);
+    else
+      OID_ELAPSED_WITH_MSG(message, usecs_elapsed, "TIME_TO_DECODE_OSD_OPREPLY",
+                           false);
+  }
+#endif
+
+  // note last received message.
+  in_seq = message->get_seq();
+  ldout(cct, 5) << " rx " << message->get_source() << " seq "
+                << message->get_seq() << " " << message << " " << *message
+                << dendl;
+
+  bool need_dispatch_writer = true;
+  if (!connection->policy.lossy) {
+    ack_left++;
+    need_dispatch_writer = true;
+  }
+
+  state = OPENED;
+
+  connection->logger->inc(l_msgr_recv_messages);
+  connection->logger->inc(
+      l_msgr_recv_bytes,
+      cur_msg_size + sizeof(ceph_msg_header) + sizeof(ceph_msg_footer));
+
+  messenger->ms_fast_preprocess(message);
+  auto fast_dispatch_time = ceph::mono_clock::now();
+  connection->logger->tinc(l_msgr_running_recv_time,
+                           fast_dispatch_time - connection->recv_start_time);
+  if (connection->delay_state) {
+    double delay_period = 0;
+    if (rand() % 10000 < cct->_conf->ms_inject_delay_probability * 10000.0) {
+      delay_period =
+          cct->_conf->ms_inject_delay_max * (double)(rand() % 10000) / 10000.0;
+      ldout(cct, 1) << "queue_received will delay after "
+                    << (ceph_clock_now() + delay_period) << " on " << message
+                    << " " << *message << dendl;
+    }
+    connection->delay_state->queue(delay_period, message);
+  } else if (messenger->ms_can_fast_dispatch(message)) {
+    connection->lock.unlock();
+    connection->dispatch_queue->fast_dispatch(message);
+    connection->recv_start_time = ceph::mono_clock::now();
+    connection->logger->tinc(l_msgr_running_fast_dispatch_time,
+                             connection->recv_start_time - fast_dispatch_time);
+    connection->lock.lock();
+  } else {
+    connection->dispatch_queue->enqueue(message, message->get_priority(),
+                                        connection->conn_id);
+  }
+
+  // clean up local buffer references
+  data_buf.clear();
+  front.clear();
+  middle.clear();
+  data.clear();
+
+  if (need_dispatch_writer && connection->is_connected()) {
+    connection->center->dispatch_event_external(connection->write_handler);
+  }
+
+  wait_message();
+}
+
+void ProtocolV1::session_reset() {
+  ldout(cct, 10) << __func__ << " started" << dendl;
+
+  std::lock_guard<std::mutex> l(connection->write_lock);
+  if (connection->delay_state) {
+    connection->delay_state->discard();
+  }
+
+  connection->dispatch_queue->discard_queue(connection->conn_id);
+  discard_out_queue();
+  // note: we need to clear outcoming_bl here, but session_reset may be
+  // called by other thread, so let caller clear this itself!
+  // outcoming_bl.clear();
+
+  connection->dispatch_queue->queue_remote_reset(connection);
+
+  randomize_out_seq();
+
+  in_seq = 0;
+  connect_seq = 0;
+  // it's safe to directly set 0, double locked
+  ack_left = 0;
+  once_ready = false;
+  can_write = WriteStatus::NOWRITE;
+}
+
+void ProtocolV1::randomize_out_seq() {
+  if (connection->get_features() & CEPH_FEATURE_MSG_AUTH) {
+    // Set out_seq to a random value, so CRC won't be predictable.
+    auto rand_seq = ceph::util::generate_random_number<uint64_t>(0, SEQ_MASK);
+    ldout(cct, 10) << __func__ << " randomize_out_seq " << rand_seq << dendl;
+    out_seq = rand_seq;
+  } else {
+    // previously, seq #'s always started at 0.
+    out_seq = 0;
+  }
+}
+
+ssize_t ProtocolV1::write_message(Message *m, bufferlist &bl, bool more) {
+  FUNCTRACE(cct);
+  ceph_assert(connection->center->in_thread());
+  m->set_seq(++out_seq);
+
+  if (messenger->crcflags & MSG_CRC_HEADER) {
+    m->calc_header_crc();
+  }
+
+  ceph_msg_header &header = m->get_header();
+  ceph_msg_footer &footer = m->get_footer();
+
+  // TODO: let sign_message could be reentry?
+  // Now that we have all the crcs calculated, handle the
+  // digital signature for the message, if the AsyncConnection has session
+  // security set up.  Some session security options do not
+  // actually calculate and check the signature, but they should
+  // handle the calls to sign_message and check_signature.  PLR
+  if (session_security.get() == NULL) {
+    ldout(cct, 20) << __func__ << " no session security" << dendl;
+  } else {
+    if (session_security->sign_message(m)) {
+      ldout(cct, 20) << __func__ << " failed to sign m=" << m
+                     << "): sig = " << footer.sig << dendl;
+    } else {
+      ldout(cct, 20) << __func__ << " signed m=" << m
+                     << "): sig = " << footer.sig << dendl;
+    }
+  }
+
+  connection->outcoming_bl.append(CEPH_MSGR_TAG_MSG);
+  connection->outcoming_bl.append((char *)&header, sizeof(header));
+
+  ldout(cct, 20) << __func__ << " sending message type=" << header.type
+                 << " src " << entity_name_t(header.src)
+                 << " front=" << header.front_len << " data=" << header.data_len
+                 << " off " << header.data_off << dendl;
+
+  if ((bl.length() <= ASYNC_COALESCE_THRESHOLD) && (bl.buffers().size() > 1)) {
+    for (const auto &pb : bl.buffers()) {
+      connection->outcoming_bl.append((char *)pb.c_str(), pb.length());
+    }
+  } else {
+    connection->outcoming_bl.claim_append(bl);
+  }
+
+  // send footer; if receiver doesn't support signatures, use the old footer
+  // format
+  ceph_msg_footer_old old_footer;
+  if (connection->has_feature(CEPH_FEATURE_MSG_AUTH)) {
+    connection->outcoming_bl.append((char *)&footer, sizeof(footer));
+  } else {
+    if (messenger->crcflags & MSG_CRC_HEADER) {
+      old_footer.front_crc = footer.front_crc;
+      old_footer.middle_crc = footer.middle_crc;
+      old_footer.data_crc = footer.data_crc;
+    } else {
+      old_footer.front_crc = old_footer.middle_crc = 0;
+    }
+    old_footer.data_crc =
+        messenger->crcflags & MSG_CRC_DATA ? footer.data_crc : 0;
+    old_footer.flags = footer.flags;
+    connection->outcoming_bl.append((char *)&old_footer, sizeof(old_footer));
+  }
+
+  m->trace.event("async writing message");
+  ldout(cct, 20) << __func__ << " sending " << m->get_seq() << " " << m
+                 << dendl;
+  ssize_t total_send_size = connection->outcoming_bl.length();
+  ssize_t rc = connection->_try_send(more);
+  if (rc < 0) {
+    ldout(cct, 1) << __func__ << " error sending " << m << ", "
+                  << cpp_strerror(rc) << dendl;
+  } else {
+    connection->logger->inc(
+        l_msgr_send_bytes, total_send_size - connection->outcoming_bl.length());
+    ldout(cct, 10) << __func__ << " sending " << m
+                   << (rc ? " continuely." : " done.") << dendl;
+  }
+  if (m->get_type() == CEPH_MSG_OSD_OP)
+    OID_EVENT_TRACE_WITH_MSG(m, "SEND_MSG_OSD_OP_END", false);
+  else if (m->get_type() == CEPH_MSG_OSD_OPREPLY)
+    OID_EVENT_TRACE_WITH_MSG(m, "SEND_MSG_OSD_OPREPLY_END", false);
+  m->put();
+
+  return rc;
+}
+
+void ProtocolV1::requeue_sent() {
+  if (sent.empty()) {
+    return;
+  }
+
+  list<pair<bufferlist, Message *> > &rq = out_q[CEPH_MSG_PRIO_HIGHEST];
+  out_seq -= sent.size();
+  while (!sent.empty()) {
+    Message *m = sent.back();
+    sent.pop_back();
+    ldout(cct, 10) << __func__ << " " << *m << " for resend "
+                   << " (" << m->get_seq() << ")" << dendl;
+    rq.push_front(make_pair(bufferlist(), m));
+  }
+}
+
+uint64_t ProtocolV1::discard_requeued_up_to(uint64_t out_seq, uint64_t seq) {
+  ldout(cct, 10) << __func__ << " " << seq << dendl;
+  std::lock_guard<std::mutex> l(connection->write_lock);
+  if (out_q.count(CEPH_MSG_PRIO_HIGHEST) == 0) {
+    return seq;
+  }
+  list<pair<bufferlist, Message *> > &rq = out_q[CEPH_MSG_PRIO_HIGHEST];
+  uint64_t count = out_seq;
+  while (!rq.empty()) {
+    pair<bufferlist, Message *> p = rq.front();
+    if (p.second->get_seq() == 0 || p.second->get_seq() > seq) break;
+    ldout(cct, 10) << __func__ << " " << *(p.second) << " for resend seq "
+                   << p.second->get_seq() << " <= " << seq << ", discarding"
+                   << dendl;
+    p.second->put();
+    rq.pop_front();
+    count++;
+  }
+  if (rq.empty()) out_q.erase(CEPH_MSG_PRIO_HIGHEST);
+  return count;
+}
+
+/*
+ * Tears down the message queues, and removes them from the
+ * DispatchQueue Must hold write_lock prior to calling.
+ */
+void ProtocolV1::discard_out_queue() {
+  ldout(cct, 10) << __func__ << " started" << dendl;
+
+  for (list<Message *>::iterator p = sent.begin(); p != sent.end(); ++p) {
+    ldout(cct, 20) << __func__ << " discard " << *p << dendl;
+    (*p)->put();
+  }
+  sent.clear();
+  for (map<int, list<pair<bufferlist, Message *> > >::iterator p =
+           out_q.begin();
+       p != out_q.end(); ++p) {
+    for (list<pair<bufferlist, Message *> >::iterator r = p->second.begin();
+         r != p->second.end(); ++r) {
+      ldout(cct, 20) << __func__ << " discard " << r->second << dendl;
+      r->second->put();
+    }
+  }
+  out_q.clear();
+}
+
+void ProtocolV1::reset_recv_state() {
+  // clean up state internal variables and states
+  if (state == CONNECTING_SEND_CONNECT_MSG) {
+    if (authorizer) {
+      delete authorizer;
+    }
+    authorizer = nullptr;
+    got_bad_auth = false;
+  }
+
+  // clean read and write callbacks
+  connection->pendingReadLen.reset();
+  connection->writeCallback.reset();
+
+  if (state > THROTTLE_MESSAGE && state <= READ_FOOTER_AND_DISPATCH &&
+      connection->policy.throttler_messages) {
+    ldout(cct, 10) << __func__ << " releasing " << 1
+                   << " message to policy throttler "
+                   << connection->policy.throttler_messages->get_current()
+                   << "/" << connection->policy.throttler_messages->get_max()
+                   << dendl;
+    connection->policy.throttler_messages->put();
+  }
+  if (state > THROTTLE_BYTES && state <= READ_FOOTER_AND_DISPATCH) {
+    if (connection->policy.throttler_bytes) {
+      ldout(cct, 10) << __func__ << " releasing " << cur_msg_size
+                     << " bytes to policy throttler "
+                     << connection->policy.throttler_bytes->get_current() << "/"
+                     << connection->policy.throttler_bytes->get_max() << dendl;
+      connection->policy.throttler_bytes->put(cur_msg_size);
+    }
+  }
+  if (state > THROTTLE_DISPATCH_QUEUE && state <= READ_FOOTER_AND_DISPATCH) {
+    ldout(cct, 10)
+        << __func__ << " releasing " << cur_msg_size
+        << " bytes to dispatch_queue throttler "
+        << connection->dispatch_queue->dispatch_throttler.get_current() << "/"
+        << connection->dispatch_queue->dispatch_throttler.get_max() << dendl;
+    connection->dispatch_queue->dispatch_throttle_release(cur_msg_size);
+  }
+}
+
+Message *ProtocolV1::_get_next_outgoing(bufferlist *bl) {
+  Message *m = 0;
+  if (!out_q.empty()) {
+    map<int, list<pair<bufferlist, Message *> > >::reverse_iterator it =
+        out_q.rbegin();
+    ceph_assert(!it->second.empty());
+    list<pair<bufferlist, Message *> >::iterator p = it->second.begin();
+    m = p->second;
+    if (bl) bl->swap(p->first);
+    it->second.erase(p);
+    if (it->second.empty()) out_q.erase(it->first);
+  }
+  return m;
+}
+
+/**
+ * Client Protocol V1
+ **/
+
+void ProtocolV1::send_client_banner() {
+  ldout(cct, 20) << __func__ << dendl;
+  state = CONNECTING;
+
+  bufferlist bl;
+  bl.append(CEPH_BANNER, strlen(CEPH_BANNER));
+  WRITE(bl, &ProtocolV1::handle_client_banner_write);
+}
+
+void ProtocolV1::handle_client_banner_write(int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " write client banner failed" << dendl;
+    fault();
+    return;
+  }
+  ldout(cct, 10) << __func__ << " connect write banner done: "
+                 << connection->get_peer_addr() << dendl;
+
+  wait_server_banner();
+}
+
+void ProtocolV1::wait_server_banner() {
+  state = CONNECTING_WAIT_BANNER_AND_IDENTIFY;
+
+  ldout(cct, 20) << __func__ << dendl;
+
+  bufferlist myaddrbl;
+  unsigned banner_len = strlen(CEPH_BANNER);
+  unsigned need_len = banner_len + sizeof(ceph_entity_addr) * 2;
+  READ(need_len, &ProtocolV1::handle_server_banner_and_identify);
+}
+
+void ProtocolV1::handle_server_banner_and_identify(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read banner and identify addresses failed"
+                  << dendl;
+    fault();
+    return;
+  }
+
+  unsigned banner_len = strlen(CEPH_BANNER);
+  if (memcmp(buffer, CEPH_BANNER, banner_len)) {
+    ldout(cct, 0) << __func__ << " connect protocol error (bad banner) on peer "
+                  << connection->get_peer_addr() << dendl;
+    fault();
+    return;
+  }
+
+  bufferlist bl;
+  entity_addr_t paddr, peer_addr_for_me;
+
+  bl.append(buffer + banner_len, sizeof(ceph_entity_addr) * 2);
+  auto p = bl.cbegin();
+  try {
+    decode(paddr, p);
+    decode(peer_addr_for_me, p);
+  } catch (const buffer::error &e) {
+    lderr(cct) << __func__ << " decode peer addr failed " << dendl;
+    fault();
+    return;
+  }
+  ldout(cct, 20) << __func__ << " connect read peer addr " << paddr
+                 << " on socket " << connection->cs.fd() << dendl;
+
+  entity_addr_t peer_addr = connection->peer_addrs.legacy_addr();
+  if (peer_addr != paddr) {
+    if (paddr.is_blank_ip() && peer_addr.get_port() == paddr.get_port() &&
+        peer_addr.get_nonce() == paddr.get_nonce()) {
+      ldout(cct, 0) << __func__ << " connect claims to be " << paddr << " not "
+                    << peer_addr << " - presumably this is the same node!"
+                    << dendl;
+    } else {
+      ldout(cct, 10) << __func__ << " connect claims to be " << paddr << " not "
+                     << peer_addr << dendl;
+      fault();
+      return;
+    }
+  }
+
+  ldout(cct, 20) << __func__ << " connect peer addr for me is "
+                 << peer_addr_for_me << dendl;
+  connection->lock.unlock();
+  messenger->learned_addr(peer_addr_for_me);
+  if (cct->_conf->ms_inject_internal_delays &&
+      cct->_conf->ms_inject_socket_failures) {
+    if (rand() % cct->_conf->ms_inject_socket_failures == 0) {
+      ldout(cct, 10) << __func__ << " sleep for "
+                     << cct->_conf->ms_inject_internal_delays << dendl;
+      utime_t t;
+      t.set_from_double(cct->_conf->ms_inject_internal_delays);
+      t.sleep();
+    }
+  }
+
+  connection->lock.lock();
+  if (state != CONNECTING_WAIT_BANNER_AND_IDENTIFY) {
+    ldout(cct, 1) << __func__
+                  << " state changed while learned_addr, mark_down or "
+                  << " replacing must be happened just now" << dendl;
+    return;
+  }
+
+  bufferlist myaddrbl;
+  encode(messenger->get_myaddrs().legacy_addr(), myaddrbl, 0);  // legacy
+  WRITE(myaddrbl, &ProtocolV1::handle_my_addr_write);
+}
+
+void ProtocolV1::handle_my_addr_write(int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 2) << __func__ << " connect couldn't write my addr, "
+                  << cpp_strerror(r) << dendl;
+    fault();
+    return;
+  }
+  ldout(cct, 10) << __func__ << " connect sent my addr "
+                 << messenger->get_myaddrs().legacy_addr() << dendl;
+
+  send_connect_message();
+}
+
+void ProtocolV1::send_connect_message() {
+  state = CONNECTING_SEND_CONNECT_MSG;
+
+  ldout(cct, 20) << __func__ << dendl;
+
+  if (!authorizer) {
+    authorizer = messenger->get_authorizer(connection->peer_type, false);
+  }
+
+  ceph_msg_connect connect;
+  connect.features = connection->policy.features_supported;
+  connect.host_type = messenger->get_myname().type();
+  connect.global_seq = global_seq;
+  connect.connect_seq = connect_seq;
+  connect.protocol_version =
+      messenger->get_proto_version(connection->peer_type, true);
+  connect.authorizer_protocol = authorizer ? authorizer->protocol : 0;
+  connect.authorizer_len = authorizer ? authorizer->bl.length() : 0;
+
+  if (authorizer) {
+    ldout(cct, 10) << __func__
+                   << " connect_msg.authorizer_len=" << connect.authorizer_len
+                   << " protocol=" << connect.authorizer_protocol << dendl;
+  }
+
+  connect.flags = 0;
+  if (connection->policy.lossy) {
+    connect.flags |=
+        CEPH_MSG_CONNECT_LOSSY;  // this is fyi, actually, server decides!
+  }
+
+  bufferlist bl;
+  bl.append((char *)&connect, sizeof(connect));
+  if (authorizer) {
+    bl.append(authorizer->bl.c_str(), authorizer->bl.length());
+  }
+
+  ldout(cct, 10) << __func__ << " connect sending gseq=" << global_seq
+                 << " cseq=" << connect_seq
+                 << " proto=" << connect.protocol_version << dendl;
+
+  WRITE(bl, &ProtocolV1::handle_connect_message_write);
+}
+
+void ProtocolV1::handle_connect_message_write(int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 2) << __func__ << " connect couldn't send reply "
+                  << cpp_strerror(r) << dendl;
+    fault();
+    return;
+  }
+
+  ldout(cct, 20) << __func__
+                 << " connect wrote (self +) cseq, waiting for reply" << dendl;
+
+  wait_connect_reply();
+}
+
+void ProtocolV1::wait_connect_reply() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  memset(&connect_reply, 0, sizeof(connect_reply));
+  READ(sizeof(connect_reply), &ProtocolV1::handle_connect_reply_1);
+}
+
+void ProtocolV1::handle_connect_reply_1(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read connect reply failed" << dendl;
+    fault();
+    return;
+  }
+
+  connect_reply = *((ceph_msg_connect_reply *)buffer);
+
+  ldout(cct, 20) << __func__ << " connect got reply tag "
+                 << (int)connect_reply.tag << " connect_seq "
+                 << connect_reply.connect_seq << " global_seq "
+                 << connect_reply.global_seq << " proto "
+                 << connect_reply.protocol_version << " flags "
+                 << (int)connect_reply.flags << " features "
+                 << connect_reply.features << dendl;
+
+  if (connect_reply.authorizer_len) {
+    wait_connect_reply_auth();
+    return;
+  }
+
+  handle_connect_reply_2();
+}
+
+void ProtocolV1::wait_connect_reply_auth() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  ldout(cct, 10) << __func__
+                 << " reply.authorizer_len=" << connect_reply.authorizer_len
+                 << dendl;
+
+  ceph_assert(connect_reply.authorizer_len < 4096);
+
+  READ(connect_reply.authorizer_len, &ProtocolV1::handle_connect_reply_auth);
+}
+
+void ProtocolV1::handle_connect_reply_auth(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read connect reply authorizer failed"
+                  << dendl;
+    fault();
+    return;
+  }
+
+  bufferlist authorizer_reply;
+  authorizer_reply.append(buffer, connect_reply.authorizer_len);
+
+  if (connect_reply.tag == CEPH_MSGR_TAG_CHALLENGE_AUTHORIZER) {
+    ldout(cct, 10) << __func__ << " connect got auth challenge" << dendl;
+    authorizer->add_challenge(cct, authorizer_reply);
+    send_connect_message();
+    return;
+  }
+
+  auto iter = authorizer_reply.cbegin();
+  if (authorizer && !authorizer->verify_reply(iter)) {
+    ldout(cct, 0) << __func__ << " failed verifying authorize reply" << dendl;
+    fault();
+    return;
+  }
+
+  handle_connect_reply_2();
+}
+
+void ProtocolV1::handle_connect_reply_2() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  if (connect_reply.tag == CEPH_MSGR_TAG_FEATURES) {
+    ldout(cct, 0) << __func__ << " connect protocol feature mismatch, my "
+                  << std::hex << connection->policy.features_supported
+                  << " < peer " << connect_reply.features << " missing "
+                  << (connect_reply.features &
+                      ~connection->policy.features_supported)
+                  << std::dec << dendl;
+    fault();
+    return;
+  }
+
+  if (connect_reply.tag == CEPH_MSGR_TAG_BADPROTOVER) {
+    ldout(cct, 0) << __func__ << " connect protocol version mismatch, my "
+                  << messenger->get_proto_version(connection->peer_type, true)
+                  << " != " << connect_reply.protocol_version << dendl;
+    fault();
+    return;
+  }
+
+  if (connect_reply.tag == CEPH_MSGR_TAG_BADAUTHORIZER) {
+    ldout(cct, 0) << __func__ << " connect got BADAUTHORIZER" << dendl;
+    if (got_bad_auth) {
+      fault();
+      return;
+    }
+    got_bad_auth = true;
+    delete authorizer;
+    authorizer =
+        messenger->get_authorizer(connection->peer_type, true);  // try harder
+    send_connect_message();
+    return;
+  }
+
+  if (connect_reply.tag == CEPH_MSGR_TAG_RESETSESSION) {
+    ldout(cct, 0) << __func__ << " connect got RESETSESSION" << dendl;
+    session_reset();
+    connect_seq = 0;
+
+    // see session_reset
+    connection->outcoming_bl.clear();
+
+    send_connect_message();
+    return;
+  }
+
+  if (connect_reply.tag == CEPH_MSGR_TAG_RETRY_GLOBAL) {
+    global_seq = messenger->get_global_seq(connect_reply.global_seq);
+    ldout(cct, 5) << __func__ << " connect got RETRY_GLOBAL "
+                  << connect_reply.global_seq << " chose new " << global_seq
+                  << dendl;
+    send_connect_message();
+    return;
+  }
+
+  if (connect_reply.tag == CEPH_MSGR_TAG_RETRY_SESSION) {
+    ceph_assert(connect_reply.connect_seq > connect_seq);
+    ldout(cct, 5) << __func__ << " connect got RETRY_SESSION " << connect_seq
+                  << " -> " << connect_reply.connect_seq << dendl;
+    connect_seq = connect_reply.connect_seq;
+    send_connect_message();
+    return;
+  }
+
+  if (connect_reply.tag == CEPH_MSGR_TAG_WAIT) {
+    ldout(cct, 1) << __func__ << " connect got WAIT (connection race)" << dendl;
+    state = WAIT;
+    fault();
+    return;
+  }
+
+  uint64_t feat_missing;
+  feat_missing =
+      connection->policy.features_required & ~(uint64_t)connect_reply.features;
+  if (feat_missing) {
+    ldout(cct, 1) << __func__ << " missing required features " << std::hex
+                  << feat_missing << std::dec << dendl;
+    fault();
+    return;
+  }
+
+  if (connect_reply.tag == CEPH_MSGR_TAG_SEQ) {
+    ldout(cct, 10)
+        << __func__
+        << " got CEPH_MSGR_TAG_SEQ, reading acked_seq and writing in_seq"
+        << dendl;
+
+    wait_ack_seq();
+    return;
+  }
+
+  if (connect_reply.tag == CEPH_MSGR_TAG_READY) {
+    ldout(cct, 10) << __func__ << " got CEPH_MSGR_TAG_READY " << dendl;
+  }
+
+  client_ready();
+}
+
+void ProtocolV1::wait_ack_seq() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  READ(sizeof(uint64_t), &ProtocolV1::handle_ack_seq);
+}
+
+void ProtocolV1::handle_ack_seq(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read connect ack seq failed" << dendl;
+    fault();
+    return;
+  }
+
+  uint64_t newly_acked_seq = 0;
+
+  newly_acked_seq = *((uint64_t *)buffer);
+  ldout(cct, 2) << __func__ << " got newly_acked_seq " << newly_acked_seq
+                << " vs out_seq " << out_seq << dendl;
+  out_seq = discard_requeued_up_to(out_seq, newly_acked_seq);
+
+  bufferlist bl;
+  uint64_t s = in_seq;
+  bl.append((char *)&s, sizeof(s));
+
+  WRITE(bl, &ProtocolV1::handle_in_seq_write);
+}
+
+void ProtocolV1::handle_in_seq_write(int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 10) << __func__ << " failed to send in_seq " << dendl;
+    fault();
+    return;
+  }
+
+  ldout(cct, 10) << __func__ << " send in_seq done " << dendl;
+
+  client_ready();
+}
+
+void ProtocolV1::client_ready() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  // hooray!
+  peer_global_seq = connect_reply.global_seq;
+  connection->policy.lossy = connect_reply.flags & CEPH_MSG_CONNECT_LOSSY;
+
+  once_ready = true;
+  connect_seq += 1;
+  ceph_assert(connect_seq == connect_reply.connect_seq);
+  backoff = utime_t();
+  connection->set_features((uint64_t)connect_reply.features &
+                           (uint64_t)connection->policy.features_supported);
+  ldout(cct, 10) << __func__ << " connect success " << connect_seq
+                 << ", lossy = " << connection->policy.lossy << ", features "
+                 << connection->get_features() << dendl;
+
+  // If we have an authorizer, get a new AuthSessionHandler to deal with
+  // ongoing security of the connection.  PLR
+  if (authorizer != NULL) {
+    session_security.reset(get_auth_session_handler(
+        cct, authorizer->protocol, authorizer->session_key,
+        connection->get_features()));
+  } else {
+    // We have no authorizer, so we shouldn't be applying security to messages
+    // in this AsyncConnection.  PLR
+    session_security.reset();
+  }
+
+  if (connection->delay_state) {
+    ceph_assert(connection->delay_state->ready());
+  }
+  connection->dispatch_queue->queue_connect(connection);
+  messenger->ms_deliver_handle_fast_connect(connection);
+
+  ready();
+}
+
+/**
+ * Server Protocol V1
+ **/
+
+void ProtocolV1::send_server_banner() {
+  ldout(cct, 20) << __func__ << dendl;
+  state = ACCEPTING;
+
+  bufferlist bl;
+
+  bl.append(CEPH_BANNER, strlen(CEPH_BANNER));
+
+  auto legacy = messenger->get_myaddrs().legacy_addr();
+  encode(legacy, bl, 0);  // legacy
+  connection->port = legacy.get_port();
+  encode(connection->socket_addr, bl, 0);  // legacy
+
+  ldout(cct, 1) << __func__ << " sd=" << connection->cs.fd() << " "
+                << connection->socket_addr << dendl;
+
+  WRITE(bl, &ProtocolV1::handle_server_banner_write);
+}
+
+void ProtocolV1::handle_server_banner_write(int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << " write server banner failed" << dendl;
+    fault();
+    return;
+  }
+  ldout(cct, 10) << __func__ << " write banner and addr done: "
+                 << connection->get_peer_addr() << dendl;
+
+  wait_client_banner();
+}
+
+void ProtocolV1::wait_client_banner() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  READ(strlen(CEPH_BANNER) + sizeof(ceph_entity_addr),
+       &ProtocolV1::handle_client_banner);
+}
+
+void ProtocolV1::handle_client_banner(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read peer banner and addr failed" << dendl;
+    fault();
+    return;
+  }
+
+  if (memcmp(buffer, CEPH_BANNER, strlen(CEPH_BANNER))) {
+    ldout(cct, 1) << __func__ << " accept peer sent bad banner '" << buffer
+                  << "' (should be '" << CEPH_BANNER << "')" << dendl;
+    fault();
+    return;
+  }
+
+  bufferlist addr_bl;
+  entity_addr_t peer_addr;
+
+  addr_bl.append(buffer + strlen(CEPH_BANNER), sizeof(ceph_entity_addr));
+  try {
+    auto ti = addr_bl.cbegin();
+    decode(peer_addr, ti);
+  } catch (const buffer::error &e) {
+    lderr(cct) << __func__ << " decode peer_addr failed " << dendl;
+    fault();
+    return;
+  }
+
+  ldout(cct, 10) << __func__ << " accept peer addr is " << peer_addr << dendl;
+  if (peer_addr.is_blank_ip()) {
+    // peer apparently doesn't know what ip they have; figure it out for them.
+    int port = peer_addr.get_port();
+    peer_addr.u = connection->socket_addr.u;
+    peer_addr.set_port(port);
+
+    ldout(cct, 0) << __func__ << " accept peer addr is really " << peer_addr
+                  << " (socket is " << connection->socket_addr << ")" << dendl;
+  }
+  connection->set_peer_addr(peer_addr);  // so that connection_state gets set up
+  connection->target_addr = peer_addr;
+
+  wait_connect_message();
+}
+
+void ProtocolV1::wait_connect_message() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  memset(&connect_msg, 0, sizeof(connect_msg));
+  READ(sizeof(connect_msg), &ProtocolV1::handle_connect_message_1);
+}
+
+void ProtocolV1::handle_connect_message_1(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read connect msg failed" << dendl;
+    fault();
+    return;
+  }
+
+  connect_msg = *((ceph_msg_connect *)buffer);
+
+  state = ACCEPTING_WAIT_CONNECT_MSG_AUTH;
+
+  if (connect_msg.authorizer_len) {
+    wait_connect_message_auth();
+    return;
+  }
+
+  handle_connect_message_2();
+}
+
+void ProtocolV1::wait_connect_message_auth() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  if (!authorizer_buf.length()) {
+    authorizer_buf.push_back(buffer::create(connect_msg.authorizer_len));
+  }
+  READB(connect_msg.authorizer_len, authorizer_buf.c_str(),
+        &ProtocolV1::handle_connect_message_auth);
+}
+
+void ProtocolV1::handle_connect_message_auth(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read connect authorizer failed" << dendl;
+    fault();
+    return;
+  }
+
+  handle_connect_message_2();
+}
+
+void ProtocolV1::handle_connect_message_2() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  ldout(cct, 20) << __func__ << " accept got peer connect_seq "
+                 << connect_msg.connect_seq << " global_seq "
+                 << connect_msg.global_seq << dendl;
+
+  connection->set_peer_type(connect_msg.host_type);
+  connection->policy = messenger->get_policy(connect_msg.host_type);
+
+  ldout(cct, 10) << __func__ << " accept of host_type " << connect_msg.host_type
+                 << ", policy.lossy=" << connection->policy.lossy
+                 << " policy.server=" << connection->policy.server
+                 << " policy.standby=" << connection->policy.standby
+                 << " policy.resetcheck=" << connection->policy.resetcheck
+                 << dendl;
+
+  ceph_msg_connect_reply reply;
+  bufferlist authorizer_reply;
+
+  memset(&reply, 0, sizeof(reply));
+  reply.protocol_version =
+      messenger->get_proto_version(connection->peer_type, false);
+
+  // mismatch?
+  ldout(cct, 10) << __func__ << " accept my proto " << reply.protocol_version
+                 << ", their proto " << connect_msg.protocol_version << dendl;
+
+  if (connect_msg.protocol_version != reply.protocol_version) {
+    send_connect_message_reply(CEPH_MSGR_TAG_BADPROTOVER, reply,
+                               authorizer_reply);
+    return;
+  }
+
+  // require signatures for cephx?
+  if (connect_msg.authorizer_protocol == CEPH_AUTH_CEPHX) {
+    if (connection->peer_type == CEPH_ENTITY_TYPE_OSD ||
+        connection->peer_type == CEPH_ENTITY_TYPE_MDS) {
+      if (cct->_conf->cephx_require_signatures ||
+          cct->_conf->cephx_cluster_require_signatures) {
+        ldout(cct, 10)
+            << __func__
+            << " using cephx, requiring MSG_AUTH feature bit for cluster"
+            << dendl;
+        connection->policy.features_required |= CEPH_FEATURE_MSG_AUTH;
+      }
+    } else {
+      if (cct->_conf->cephx_require_signatures ||
+          cct->_conf->cephx_service_require_signatures) {
+        ldout(cct, 10)
+            << __func__
+            << " using cephx, requiring MSG_AUTH feature bit for service"
+            << dendl;
+        connection->policy.features_required |= CEPH_FEATURE_MSG_AUTH;
+      }
+    }
+  }
+
+  uint64_t feat_missing =
+      connection->policy.features_required & ~(uint64_t)connect_msg.features;
+  if (feat_missing) {
+    ldout(cct, 1) << __func__ << " peer missing required features " << std::hex
+                  << feat_missing << std::dec << dendl;
+    send_connect_message_reply(CEPH_MSGR_TAG_FEATURES, reply, authorizer_reply);
+    return;
+  }
+
+  connection->lock.unlock();
+
+  bool authorizer_valid;
+  bool need_challenge = HAVE_FEATURE(connect_msg.features, CEPHX_V2);
+  bool had_challenge = (bool)authorizer_challenge;
+  if (!messenger->verify_authorizer(
+          connection, connection->peer_type, connect_msg.authorizer_protocol,
+          authorizer_buf, authorizer_reply, authorizer_valid, session_key,
+          need_challenge ? &authorizer_challenge : nullptr) ||
+      !authorizer_valid) {
+    connection->lock.lock();
+
+    if (need_challenge && !had_challenge && authorizer_challenge) {
+      ldout(cct, 10) << __func__ << ": challenging authorizer" << dendl;
+      ceph_assert(authorizer_reply.length());
+      send_connect_message_reply(CEPH_MSGR_TAG_CHALLENGE_AUTHORIZER, reply,
+                                 authorizer_reply);
+      return;
+    } else {
+      ldout(cct, 0) << __func__ << ": got bad authorizer, auth_reply_len="
+                    << authorizer_reply.length() << dendl;
+      session_security.reset();
+      send_connect_message_reply(CEPH_MSGR_TAG_BADAUTHORIZER, reply,
+                                 authorizer_reply);
+      return;
+    }
+  }
+
+  // We've verified the authorizer for this AsyncConnection, so set up the
+  // session security structure.  PLR
+  ldout(cct, 10) << __func__ << " accept setting up session_security." << dendl;
+
+  // existing?
+  AsyncConnectionRef existing = messenger->lookup_conn(connection->peer_addrs);
+
+  connection->inject_delay();
+
+  connection->lock.lock();
+  if (state != ACCEPTING_WAIT_CONNECT_MSG_AUTH) {
+    ldout(cct, 1) << __func__
+                  << " state changed while accept, it must be mark_down"
+                  << dendl;
+    ceph_assert(state == CLOSED);
+    fault();
+    return;
+  }
+
+  if (existing == connection) {
+    existing = nullptr;
+  }
+
+  if (existing) {
+    // There is no possible that existing connection will acquire this
+    // connection's lock
+    existing->lock.lock();  // skip lockdep check (we are locking a second
+                            // AsyncConnection here)
+
+    ProtocolV1 *exproto = dynamic_cast<ProtocolV1 *>(existing->protocol.get());
+
+    if (!exproto) {
+      ldout(cct, 1) << __func__ << " existing=" << existing << dendl;
+      ceph_assert(false);
+    }
+
+    if (exproto->state == CLOSED) {
+      ldout(cct, 1) << __func__ << " existing already closed." << dendl;
+      existing->lock.unlock();
+      existing = nullptr;
+
+      open(reply, authorizer_reply);
+      return;
+    }
+
+    if (exproto->replacing) {
+      ldout(cct, 1) << __func__
+                    << " existing racing replace happened while replacing."
+                    << " existing_state="
+                    << connection->get_state_name(existing->state) << dendl;
+      reply.global_seq = exproto->peer_global_seq;
+      existing->lock.unlock();
+      send_connect_message_reply(CEPH_MSGR_TAG_RETRY_GLOBAL, reply,
+                                 authorizer_reply);
+      return;
+    }
+
+    if (connect_msg.global_seq < exproto->peer_global_seq) {
+      ldout(cct, 10) << __func__ << " accept existing " << existing << ".gseq "
+                     << exproto->peer_global_seq << " > "
+                     << connect_msg.global_seq << ", RETRY_GLOBAL" << dendl;
+      reply.global_seq = exproto->peer_global_seq;  // so we can send it below..
+      existing->lock.unlock();
+      send_connect_message_reply(CEPH_MSGR_TAG_RETRY_GLOBAL, reply,
+                                 authorizer_reply);
+      return;
+    } else {
+      ldout(cct, 10) << __func__ << " accept existing " << existing << ".gseq "
+                     << exproto->peer_global_seq
+                     << " <= " << connect_msg.global_seq << ", looks ok"
+                     << dendl;
+    }
+
+    if (existing->policy.lossy) {
+      ldout(cct, 0)
+          << __func__
+          << " accept replacing existing (lossy) channel (new one lossy="
+          << connection->policy.lossy << ")" << dendl;
+      exproto->session_reset();
+      replace(existing, reply, authorizer_reply);
+      return;
+    }
+
+    ldout(cct, 1) << __func__ << " accept connect_seq "
+                  << connect_msg.connect_seq
+                  << " vs existing csq=" << exproto->connect_seq
+                  << " existing_state="
+                  << connection->get_state_name(existing->state) << dendl;
+
+    if (connect_msg.connect_seq == 0 && exproto->connect_seq > 0) {
+      ldout(cct, 0)
+          << __func__
+          << " accept peer reset, then tried to connect to us, replacing"
+          << dendl;
+      // this is a hard reset from peer
+      is_reset_from_peer = true;
+      if (connection->policy.resetcheck) {
+        exproto->session_reset();  // this resets out_queue, msg_ and
+                                   // connect_seq #'s
+      }
+      replace(existing, reply, authorizer_reply);
+      return;
+    }
+
+    if (connect_msg.connect_seq < exproto->connect_seq) {
+      // old attempt, or we sent READY but they didn't get it.
+      ldout(cct, 10) << __func__ << " accept existing " << existing << ".cseq "
+                     << exproto->connect_seq << " > " << connect_msg.connect_seq
+                     << ", RETRY_SESSION" << dendl;
+      reply.connect_seq = exproto->connect_seq + 1;
+      existing->lock.unlock();
+      send_connect_message_reply(CEPH_MSGR_TAG_RETRY_SESSION, reply,
+                                 authorizer_reply);
+      return;
+    }
+
+    if (connect_msg.connect_seq == exproto->connect_seq) {
+      // if the existing connection successfully opened, and/or
+      // subsequently went to standby, then the peer should bump
+      // their connect_seq and retry: this is not a connection race
+      // we need to resolve here.
+      if (exproto->state == OPENED || exproto->state == STANDBY) {
+        ldout(cct, 10) << __func__ << " accept connection race, existing "
+                       << existing << ".cseq " << exproto->connect_seq
+                       << " == " << connect_msg.connect_seq
+                       << ", OPEN|STANDBY, RETRY_SESSION " << dendl;
+        // if connect_seq both zero, dont stuck into dead lock. it's ok to
+        // replace
+        if (connection->policy.resetcheck && exproto->connect_seq == 0) {
+          replace(existing, reply, authorizer_reply);
+          return;
+        }
+
+        reply.connect_seq = exproto->connect_seq + 1;
+        existing->lock.unlock();
+        send_connect_message_reply(CEPH_MSGR_TAG_RETRY_SESSION, reply,
+                                   authorizer_reply);
+        return;
+      }
+
+      // connection race?
+      if (connection->peer_addrs.legacy_addr() < messenger->get_myaddr() ||
+          existing->policy.server) {
+        // incoming wins
+        ldout(cct, 10) << __func__ << " accept connection race, existing "
+                       << existing << ".cseq " << exproto->connect_seq
+                       << " == " << connect_msg.connect_seq
+                       << ", or we are server, replacing my attempt" << dendl;
+        replace(existing, reply, authorizer_reply);
+        return;
+      } else {
+        // our existing outgoing wins
+        ldout(messenger->cct, 10)
+            << __func__ << " accept connection race, existing " << existing
+            << ".cseq " << exproto->connect_seq
+            << " == " << connect_msg.connect_seq << ", sending WAIT" << dendl;
+        ceph_assert(connection->peer_addrs.legacy_addr() >
+                    messenger->get_myaddr());
+        existing->lock.unlock();
+        send_connect_message_reply(CEPH_MSGR_TAG_WAIT, reply, authorizer_reply);
+        return;
+      }
+    }
+
+    ceph_assert(connect_msg.connect_seq > exproto->connect_seq);
+    ceph_assert(connect_msg.global_seq >= exproto->peer_global_seq);
+    if (connection->policy.resetcheck &&  // RESETSESSION only used by servers;
+                                          // peers do not reset each other
+        exproto->connect_seq == 0) {
+      ldout(cct, 0) << __func__ << " accept we reset (peer sent cseq "
+                    << connect_msg.connect_seq << ", " << existing
+                    << ".cseq = " << exproto->connect_seq
+                    << "), sending RESETSESSION " << dendl;
+      existing->lock.unlock();
+      send_connect_message_reply(CEPH_MSGR_TAG_RESETSESSION, reply,
+                                 authorizer_reply);
+      return;
+    }
+
+    // reconnect
+    ldout(cct, 10) << __func__ << " accept peer sent cseq "
+                   << connect_msg.connect_seq << " > " << exproto->connect_seq
+                   << dendl;
+    replace(existing, reply, authorizer_reply);
+    return;
+  }  // existing
+  else if (!replacing && connect_msg.connect_seq > 0) {
+    // we reset, and they are opening a new session
+    ldout(cct, 0) << __func__ << " accept we reset (peer sent cseq "
+                  << connect_msg.connect_seq << "), sending RESETSESSION"
+                  << dendl;
+    send_connect_message_reply(CEPH_MSGR_TAG_RESETSESSION, reply,
+                               authorizer_reply);
+    return;
+  } else {
+    // new session
+    ldout(cct, 10) << __func__ << " accept new session" << dendl;
+    existing = nullptr;
+    open(reply, authorizer_reply);
+    return;
+  }
+}
+
+void ProtocolV1::send_connect_message_reply(char tag,
+                                            ceph_msg_connect_reply &reply,
+                                            bufferlist &authorizer_reply) {
+  ldout(cct, 20) << __func__ << dendl;
+  bufferlist reply_bl;
+  reply.tag = tag;
+  reply.features =
+      ((uint64_t)connect_msg.features & connection->policy.features_supported) |
+      connection->policy.features_required;
+  reply.authorizer_len = authorizer_reply.length();
+  reply_bl.append((char *)&reply, sizeof(reply));
+
+  if (reply.authorizer_len) {
+    reply_bl.append(authorizer_reply.c_str(), authorizer_reply.length());
+    authorizer_reply.clear();
+  }
+
+  WRITE(reply_bl, &ProtocolV1::handle_connect_message_reply_write);
+}
+
+void ProtocolV1::handle_connect_message_reply_write(int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << " write connect message reply failed" << dendl;
+    connection->inject_delay();
+    fault();
+    return;
+  }
+
+  wait_connect_message();
+}
+
+void ProtocolV1::replace(AsyncConnectionRef existing,
+                         ceph_msg_connect_reply &reply,
+                         bufferlist &authorizer_reply) {
+  ldout(cct, 10) << __func__ << " accept replacing " << existing << dendl;
+
+  connection->inject_delay();
+  if (existing->policy.lossy) {
+    // disconnect from the Connection
+    ldout(cct, 1) << __func__ << " replacing on lossy channel, failing existing"
+                  << dendl;
+    existing->protocol->stop();
+    existing->dispatch_queue->queue_reset(existing.get());
+  } else {
+    ceph_assert(can_write == WriteStatus::NOWRITE);
+    existing->write_lock.lock();
+
+    ProtocolV1 *exproto = dynamic_cast<ProtocolV1 *>(existing->protocol.get());
+
+    // reset the in_seq if this is a hard reset from peer,
+    // otherwise we respect our original connection's value
+    if (is_reset_from_peer) {
+      exproto->is_reset_from_peer = true;
+    }
+
+    connection->center->delete_file_event(connection->cs.fd(),
+                                          EVENT_READABLE | EVENT_WRITABLE);
+
+    if (existing->delay_state) {
+      existing->delay_state->flush();
+      ceph_assert(!connection->delay_state);
+    }
+    exproto->reset_recv_state();
+
+    auto temp_cs = std::move(connection->cs);
+    EventCenter *new_center = connection->center;
+    Worker *new_worker = connection->worker;
+    // avoid _stop shutdown replacing socket
+    // queue a reset on the new connection, which we're dumping for the old
+    stop();
+
+    connection->dispatch_queue->queue_reset(connection);
+    ldout(messenger->cct, 1)
+        << __func__ << " stop myself to swap existing" << dendl;
+    exproto->can_write = WriteStatus::REPLACING;
+    exproto->replacing = true;
+    existing->state_offset = 0;
+    // avoid previous thread modify event
+    exproto->state = NONE;
+    existing->state = AsyncConnection::STATE_NONE;
+    // Discard existing prefetch buffer in `recv_buf`
+    existing->recv_start = existing->recv_end = 0;
+    // there shouldn't exist any buffer
+    ceph_assert(connection->recv_start == connection->recv_end);
+
+    exproto->authorizer_challenge.reset();
+
+    auto deactivate_existing = std::bind(
+        [existing, new_worker, new_center, exproto, reply,
+         authorizer_reply](ConnectedSocket &cs) mutable {
+          // we need to delete time event in original thread
+          {
+            std::lock_guard<std::mutex> l(existing->lock);
+            existing->write_lock.lock();
+            exproto->requeue_sent();
+            existing->outcoming_bl.clear();
+            existing->open_write = false;
+            existing->write_lock.unlock();
+            if (exproto->state == NONE) {
+              existing->shutdown_socket();
+              existing->cs = std::move(cs);
+              existing->worker->references--;
+              new_worker->references++;
+              existing->logger = new_worker->get_perf_counter();
+              existing->worker = new_worker;
+              existing->center = new_center;
+              if (existing->delay_state)
+                existing->delay_state->set_center(new_center);
+            } else if (exproto->state == CLOSED) {
+              auto back_to_close =
+                  std::bind([](ConnectedSocket &cs) mutable { cs.close(); },
+                            std::move(cs));
+              new_center->submit_to(new_center->get_id(),
+                                    std::move(back_to_close), true);
+              return;
+            } else {
+              ceph_abort();
+            }
+          }
+
+          // Before changing existing->center, it may already exists some
+          // events in existing->center's queue. Then if we mark down
+          // `existing`, it will execute in another thread and clean up
+          // connection. Previous event will result in segment fault
+          auto transfer_existing = [existing, exproto, reply,
+                                    authorizer_reply]() mutable {
+            std::lock_guard<std::mutex> l(existing->lock);
+            if (exproto->state == CLOSED) return;
+            ceph_assert(exproto->state == NONE);
+
+            existing->state = AsyncConnection::STATE_CONNECTION_ESTABLISHED;
+            exproto->state = ACCEPTING;
+
+            existing->center->create_file_event(
+                existing->cs.fd(), EVENT_READABLE, existing->read_handler);
+            reply.global_seq = exproto->peer_global_seq;
+            exproto->send_connect_message_reply(CEPH_MSGR_TAG_RETRY_GLOBAL,
+                                                reply, authorizer_reply);
+          };
+          if (existing->center->in_thread())
+            transfer_existing();
+          else
+            existing->center->submit_to(existing->center->get_id(),
+                                        std::move(transfer_existing), true);
+        },
+        std::move(temp_cs));
+
+    existing->center->submit_to(existing->center->get_id(),
+                                std::move(deactivate_existing), true);
+    existing->write_lock.unlock();
+    existing->lock.unlock();
+    return;
+  }
+  existing->lock.unlock();
+
+  open(reply, authorizer_reply);
+}
+
+void ProtocolV1::open(ceph_msg_connect_reply &reply,
+                      bufferlist &authorizer_reply) {
+  ldout(cct, 20) << __func__ << dendl;
+
+  connect_seq = connect_msg.connect_seq + 1;
+  peer_global_seq = connect_msg.global_seq;
+  ldout(cct, 10) << __func__ << " accept success, connect_seq = " << connect_seq
+                 << " in_seq=" << in_seq << ", sending READY" << dendl;
+
+  // if it is a hard reset from peer, we don't need a round-trip to negotiate
+  // in/out sequence
+  if ((connect_msg.features & CEPH_FEATURE_RECONNECT_SEQ) &&
+      !is_reset_from_peer) {
+    reply.tag = CEPH_MSGR_TAG_SEQ;
+    wait_for_seq = true;
+  } else {
+    reply.tag = CEPH_MSGR_TAG_READY;
+    wait_for_seq = false;
+    out_seq = discard_requeued_up_to(out_seq, 0);
+    is_reset_from_peer = false;
+    in_seq = 0;
+  }
+
+  // send READY reply
+  reply.features = connection->policy.features_supported;
+  reply.global_seq = messenger->get_global_seq();
+  reply.connect_seq = connect_seq;
+  reply.flags = 0;
+  reply.authorizer_len = authorizer_reply.length();
+  if (connection->policy.lossy) {
+    reply.flags = reply.flags | CEPH_MSG_CONNECT_LOSSY;
+  }
+
+  connection->set_features((uint64_t)reply.features &
+                           (uint64_t)connect_msg.features);
+  ldout(cct, 10) << __func__ << " accept features "
+                 << connection->get_features() << dendl;
+
+  session_security.reset(
+      get_auth_session_handler(cct, connect_msg.authorizer_protocol,
+                               session_key, connection->get_features()));
+
+  bufferlist reply_bl;
+  reply_bl.append((char *)&reply, sizeof(reply));
+
+  if (reply.authorizer_len) {
+    reply_bl.append(authorizer_reply.c_str(), authorizer_reply.length());
+  }
+
+  if (reply.tag == CEPH_MSGR_TAG_SEQ) {
+    uint64_t s = in_seq;
+    reply_bl.append((char *)&s, sizeof(s));
+  }
+
+  connection->lock.unlock();
+  // Because "replacing" will prevent other connections preempt this addr,
+  // it's safe that here we don't acquire Connection's lock
+  ssize_t r = messenger->accept_conn(connection);
+
+  connection->inject_delay();
+
+  connection->lock.lock();
+  replacing = false;
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " existing race replacing process for addr = "
+                  << connection->peer_addrs.legacy_addr()
+                  << " just fail later one(this)" << dendl;
+    ldout(cct, 10) << "accept fault after register" << dendl;
+    connection->inject_delay();
+    fault();
+    return;
+  }
+  if (state != ACCEPTING_WAIT_CONNECT_MSG_AUTH) {
+    ldout(cct, 1) << __func__
+                  << " state changed while accept_conn, it must be mark_down"
+                  << dendl;
+    ceph_assert(state == CLOSED || state == NONE);
+    ldout(cct, 10) << "accept fault after register" << dendl;
+    connection->inject_delay();
+    fault();
+    return;
+  }
+
+  WRITE(reply_bl, &ProtocolV1::handle_ready_connect_message_reply_write);
+}
+
+void ProtocolV1::handle_ready_connect_message_reply_write(int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " write ready connect message reply failed"
+                  << dendl;
+    fault();
+    return;
+  }
+
+  // notify
+  connection->dispatch_queue->queue_accept(connection);
+  messenger->ms_deliver_handle_fast_accept(connection);
+  once_ready = true;
+
+  state = ACCEPTING_HANDLED_CONNECT_MSG;
+
+  if (wait_for_seq) {
+    wait_seq();
+  } else {
+    server_ready();
+  }
+}
+
+void ProtocolV1::wait_seq() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  READ(sizeof(uint64_t), &ProtocolV1::handle_seq);
+}
+
+void ProtocolV1::handle_seq(char *buffer, int r) {
+  ldout(cct, 20) << __func__ << " r=" << r << dendl;
+
+  if (r < 0) {
+    ldout(cct, 1) << __func__ << " read ack seq failed" << dendl;
+    fault();
+    return;
+  }
+
+  uint64_t newly_acked_seq = *(uint64_t *)buffer;
+  ldout(cct, 2) << __func__ << " accept get newly_acked_seq " << newly_acked_seq
+                << dendl;
+  out_seq = discard_requeued_up_to(out_seq, newly_acked_seq);
+
+  server_ready();
+}
+
+void ProtocolV1::server_ready() {
+  ldout(cct, 20) << __func__ << dendl;
+
+  ldout(cct, 20) << __func__ << " accept done" << dendl;
+  memset(&connect_msg, 0, sizeof(connect_msg));
+
+  if (connection->delay_state) {
+    ceph_assert(connection->delay_state->ready());
+  }
+
+  ready();
+}
diff --git a/src/msg/async/Protocol.h b/src/msg/async/Protocol.h
new file mode 100644 (file)
index 0000000..825d3da
--- /dev/null
@@ -0,0 +1,242 @@
+#ifndef _MSG_ASYNC_PROTOCOL_
+#define _MSG_ASYNC_PROTOCOL_
+
+#include <list>
+#include <map>
+
+#include "AsyncConnection.h"
+#include "include/buffer.h"
+#include "include/msgr.h"
+
+class AsyncMessenger;
+
+class Protocol {
+protected:
+  AsyncConnection *connection;
+  AsyncMessenger *messenger;
+  CephContext *cct;
+
+public:
+  Protocol(AsyncConnection *connection);
+  virtual ~Protocol();
+
+  // prepare protocol for connecting to peer
+  virtual void connect() = 0;
+  // prepare protocol for accepting peer connections
+  virtual void accept() = 0;
+  // true -> protocol is ready for sending messages
+  virtual bool is_connected() = 0;
+  // stop connection
+  virtual void stop() = 0;
+  // signal and handle connection failure
+  virtual void fault() = 0;
+  // send message
+  virtual void send_message(Message *m) = 0;
+  // send keepalive
+  virtual void send_keepalive() = 0;
+
+  virtual void read_event() = 0;
+  virtual void write_event() = 0;
+  virtual bool is_queued() = 0;
+};
+
+class ProtocolV1 : public Protocol {
+protected:
+  enum State {
+    NONE = 0,
+    START_CONNECT,
+    CONNECTING,
+    CONNECTING_WAIT_BANNER_AND_IDENTIFY,
+    CONNECTING_SEND_CONNECT_MSG,
+    START_ACCEPT,
+    ACCEPTING,
+    ACCEPTING_WAIT_CONNECT_MSG_AUTH,
+    ACCEPTING_HANDLED_CONNECT_MSG,
+    OPENED,
+    THROTTLE_MESSAGE,
+    THROTTLE_BYTES,
+    THROTTLE_DISPATCH_QUEUE,
+    READ_MESSAGE_FRONT,
+    READ_FOOTER_AND_DISPATCH,
+    CLOSED,
+    WAIT,
+    STANDBY
+  };
+
+  static const char *get_state_name(int state) {
+    const char *const statenames[] = {"NONE",
+                                      "START_CONNECT",
+                                      "CONNECTING",
+                                      "CONNECTING_WAIT_BANNER_AND_IDENTIFY",
+                                      "CONNECTING_SEND_CONNECT_MSG",
+                                      "START_ACCEPT",
+                                      "ACCEPTING",
+                                      "ACCEPTING_WAIT_CONNECT_MSG_AUTH",
+                                      "ACCEPTING_HANDLED_CONNECT_MSG",
+                                      "OPENED",
+                                      "THROTTLE_MESSAGE",
+                                      "THROTTLE_BYTES",
+                                      "THROTTLE_DISPATCH_QUEUE",
+                                      "READ_MESSAGE_FRONT",
+                                      "READ_FOOTER_AND_DISPATCH",
+                                      "CLOSED",
+                                      "WAIT",
+                                      "STANDBY"};
+    return statenames[state];
+  }
+
+  char *temp_buffer;
+
+  enum class WriteStatus { NOWRITE, REPLACING, CANWRITE, CLOSED };
+  std::atomic<WriteStatus> can_write;
+  std::list<Message *> sent;  // the first bufferlist need to inject seq
+  // priority queue for outbound msgs
+  std::map<int, std::list<std::pair<bufferlist, Message *>>> out_q;
+  bool keepalive;
+
+  __u32 connect_seq, peer_global_seq;
+  std::atomic<uint64_t> in_seq{0};
+  std::atomic<uint64_t> out_seq{0};
+  std::atomic<uint64_t> ack_left{0};
+
+  CryptoKey session_key;
+  std::shared_ptr<AuthSessionHandler> session_security;
+  std::unique_ptr<AuthAuthorizerChallenge> authorizer_challenge;  // accept side
+
+  // Open state
+  ceph_msg_connect connect_msg;
+  ceph_msg_connect_reply connect_reply;
+  bufferlist authorizer_buf;
+
+  utime_t backoff;  // backoff time
+  utime_t recv_stamp;
+  utime_t throttle_stamp;
+  unsigned msg_left;
+  uint64_t cur_msg_size;
+  ceph_msg_header current_header;
+  bufferlist data_buf;
+  bufferlist::iterator data_blp;
+  bufferlist front, middle, data;
+
+  bool replacing;  // when replacing process happened, we will reply connect
+                   // side with RETRY tag and accept side will clear replaced
+                   // connection. So when connect side reissue connect_msg,
+                   // there won't exists conflicting connection so we use
+                   // "replacing" to skip RESETSESSION to avoid detect wrong
+                   // presentation
+  bool is_reset_from_peer;
+  bool once_ready;
+
+  State state;
+
+  void ready();
+  void wait_message();
+  void handle_message(char *buffer, int r);
+
+  void handle_keepalive2(char *buffer, int r);
+  void append_keepalive_or_ack(bool ack = false, utime_t *t = nullptr);
+  void handle_keepalive2_ack(char *buffer, int r);
+  void handle_tag_ack(char *buffer, int r);
+
+  void handle_message_header(char *buffer, int r);
+  void throttle_message();
+  void throttle_bytes();
+  void throttle_dispatch_queue();
+  void read_message_front();
+  void handle_message_front(char *buffer, int r);
+  void read_message_middle();
+  void handle_message_middle(char *buffer, int r);
+  void read_message_data_prepare();
+  void read_message_data();
+  void handle_message_data(char *buffer, int r);
+  void read_message_footer();
+  void handle_message_footer(char *buffer, int r);
+
+  void session_reset();
+  void randomize_out_seq();
+
+  Message *_get_next_outgoing(bufferlist *bl);
+
+  void prepare_send_message(uint64_t features, Message *m, bufferlist &bl);
+  ssize_t write_message(Message *m, bufferlist &bl, bool more);
+
+  void requeue_sent();
+  uint64_t discard_requeued_up_to(uint64_t out_seq, uint64_t seq);
+  void discard_out_queue();
+
+  void reset_recv_state();
+
+  ostream &_conn_prefix(std::ostream *_dout);
+
+public:
+  ProtocolV1(AsyncConnection *connection);
+  virtual ~ProtocolV1();
+
+  virtual void connect() override;
+  virtual void accept() override;
+  virtual bool is_connected() override;
+  virtual void stop() override;
+  virtual void fault() override;
+  virtual void send_message(Message *m) override;
+  virtual void send_keepalive() override;
+
+  virtual void read_event() override;
+  virtual void write_event() override;
+  virtual bool is_queued() override;
+
+  // Client Protocol
+private:
+  int global_seq;
+  bool got_bad_auth;
+  AuthAuthorizer *authorizer;
+
+  void send_client_banner();
+  void handle_client_banner_write(int r);
+  void wait_server_banner();
+  void handle_server_banner_and_identify(char *buffer, int r);
+  void handle_my_addr_write(int r);
+  void send_connect_message();
+  void handle_connect_message_write(int r);
+  void wait_connect_reply();
+  void handle_connect_reply_1(char *buffer, int r);
+  void wait_connect_reply_auth();
+  void handle_connect_reply_auth(char *buffer, int r);
+  void handle_connect_reply_2();
+  void wait_ack_seq();
+  void handle_ack_seq(char *buffer, int r);
+  void handle_in_seq_write(int r);
+  void client_ready();
+
+  // Server Protocol
+private:
+  bool wait_for_seq;
+
+  void send_server_banner();
+  void handle_server_banner_write(int r);
+  void wait_client_banner();
+  void handle_client_banner(char *buffer, int r);
+  void wait_connect_message();
+  void handle_connect_message_1(char *buffer, int r);
+  void wait_connect_message_auth();
+  void handle_connect_message_auth(char *buffer, int r);
+  void handle_connect_message_2();
+  void send_connect_message_reply(char tag, ceph_msg_connect_reply &reply,
+                                  bufferlist &authorizer_reply);
+  void handle_connect_message_reply_write(int r);
+  void replace(AsyncConnectionRef existing, ceph_msg_connect_reply &reply,
+               bufferlist &authorizer_reply);
+  void open(ceph_msg_connect_reply &reply, bufferlist &authorizer_reply);
+  void handle_ready_connect_message_reply_write(int r);
+  void wait_seq();
+  void handle_seq(char *buffer, int r);
+  void server_ready();
+};
+
+class LoopbackProtocolV1 : public ProtocolV1 {
+public:
+  LoopbackProtocolV1(AsyncConnection *connection) : ProtocolV1(connection) {
+    this->can_write = WriteStatus::CANWRITE;
+  }
+};
+
+#endif /* _MSG_ASYNC_PROTOCOL_V1_ */
\ No newline at end of file