]> git-server-git.apps.pok.os.sepia.ceph.com Git - ceph.git/commitdiff
msg: move SimpleMessenger to msg/simple/ 2652/head
authorSage Weil <sage@redhat.com>
Mon, 6 Oct 2014 15:40:16 +0000 (08:40 -0700)
committerSage Weil <sage@redhat.com>
Tue, 7 Oct 2014 04:23:07 +0000 (21:23 -0700)
Signed-off-by: Sage Weil <sage@redhat.com>
22 files changed:
src/msg/Accepter.cc [deleted file]
src/msg/Accepter.h [deleted file]
src/msg/DispatchQueue.cc [deleted file]
src/msg/DispatchQueue.h [deleted file]
src/msg/Makefile.am
src/msg/Messenger.cc
src/msg/Pipe.cc [deleted file]
src/msg/Pipe.h [deleted file]
src/msg/PipeConnection.cc [deleted file]
src/msg/PipeConnection.h [deleted file]
src/msg/SimpleMessenger.cc [deleted file]
src/msg/SimpleMessenger.h [deleted file]
src/msg/simple/Accepter.cc [new file with mode: 0644]
src/msg/simple/Accepter.h [new file with mode: 0644]
src/msg/simple/DispatchQueue.cc [new file with mode: 0644]
src/msg/simple/DispatchQueue.h [new file with mode: 0644]
src/msg/simple/Pipe.cc [new file with mode: 0644]
src/msg/simple/Pipe.h [new file with mode: 0644]
src/msg/simple/PipeConnection.cc [new file with mode: 0644]
src/msg/simple/PipeConnection.h [new file with mode: 0644]
src/msg/simple/SimpleMessenger.cc [new file with mode: 0644]
src/msg/simple/SimpleMessenger.h [new file with mode: 0644]

diff --git a/src/msg/Accepter.cc b/src/msg/Accepter.cc
deleted file mode 100644 (file)
index 029a037..0000000
+++ /dev/null
@@ -1,255 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software 
- * Foundation.  See file COPYING.
- * 
- */
-
-#include <sys/socket.h>
-#include <netinet/tcp.h>
-#include <sys/uio.h>
-#include <limits.h>
-#include <poll.h>
-
-#include "Accepter.h"
-#include "SimpleMessenger.h"
-
-#include "Message.h"
-#include "Pipe.h"
-
-#include "common/debug.h"
-#include "common/errno.h"
-
-#define dout_subsys ceph_subsys_ms
-
-#undef dout_prefix
-#define dout_prefix *_dout << "accepter."
-
-
-/********************************************
- * Accepter
- */
-
-int Accepter::bind(const entity_addr_t &bind_addr, const set<int>& avoid_ports)
-{
-  const md_config_t *conf = msgr->cct->_conf;
-  // bind to a socket
-  ldout(msgr->cct,10) << "accepter.bind" << dendl;
-  
-  int family;
-  switch (bind_addr.get_family()) {
-  case AF_INET:
-  case AF_INET6:
-    family = bind_addr.get_family();
-    break;
-
-  default:
-    // bind_addr is empty
-    family = conf->ms_bind_ipv6 ? AF_INET6 : AF_INET;
-  }
-
-  /* socket creation */
-  listen_sd = ::socket(family, SOCK_STREAM, 0);
-  if (listen_sd < 0) {
-    lderr(msgr->cct) << "accepter.bind unable to create socket: "
-                    << cpp_strerror(errno) << dendl;
-    return -errno;
-  }
-
-  // use whatever user specified (if anything)
-  entity_addr_t listen_addr = bind_addr;
-  listen_addr.set_family(family);
-
-  /* bind to port */
-  int rc = -1;
-  if (listen_addr.get_port()) {
-    // specific port
-
-    // reuse addr+port when possible
-    int on = 1;
-    rc = ::setsockopt(listen_sd, SOL_SOCKET, SO_REUSEADDR, &on, sizeof(on));
-    if (rc < 0) {
-      lderr(msgr->cct) << "accepter.bind unable to setsockopt: "
-                        << cpp_strerror(errno) << dendl;
-      return -errno;
-    }
-
-    rc = ::bind(listen_sd, (struct sockaddr *) &listen_addr.ss_addr(), listen_addr.addr_size());
-    if (rc < 0) {
-      lderr(msgr->cct) << "accepter.bind unable to bind to " << listen_addr.ss_addr()
-                      << ": " << cpp_strerror(errno) << dendl;
-      return -errno;
-    }
-  } else {
-    // try a range of ports
-    for (int port = msgr->cct->_conf->ms_bind_port_min; port <= msgr->cct->_conf->ms_bind_port_max; port++) {
-      if (avoid_ports.count(port))
-       continue;
-      listen_addr.set_port(port);
-      rc = ::bind(listen_sd, (struct sockaddr *) &listen_addr.ss_addr(), listen_addr.addr_size());
-      if (rc == 0)
-       break;
-    }
-    if (rc < 0) {
-      lderr(msgr->cct) << "accepter.bind unable to bind to " << listen_addr.ss_addr()
-                      << " on any port in range " << msgr->cct->_conf->ms_bind_port_min
-                      << "-" << msgr->cct->_conf->ms_bind_port_max
-                      << ": " << cpp_strerror(errno)
-                      << dendl;
-      return -errno;
-    }
-    ldout(msgr->cct,10) << "accepter.bind bound on random port " << listen_addr << dendl;
-  }
-
-  // what port did we get?
-  socklen_t llen = sizeof(listen_addr.ss_addr());
-  rc = getsockname(listen_sd, (sockaddr*)&listen_addr.ss_addr(), &llen);
-  if (rc < 0) {
-    rc = -errno;
-    lderr(msgr->cct) << "accepter.bind failed getsockname: " << cpp_strerror(rc) << dendl;
-    return rc;
-  }
-  
-  ldout(msgr->cct,10) << "accepter.bind bound to " << listen_addr << dendl;
-
-  // listen!
-  rc = ::listen(listen_sd, 128);
-  if (rc < 0) {
-    rc = -errno;
-    lderr(msgr->cct) << "accepter.bind unable to listen on " << listen_addr
-                    << ": " << cpp_strerror(rc) << dendl;
-    return rc;
-  }
-  
-  msgr->set_myaddr(bind_addr);
-  if (bind_addr != entity_addr_t())
-    msgr->learned_addr(bind_addr);
-  else
-    assert(msgr->get_need_addr());  // should still be true.
-
-  if (msgr->get_myaddr().get_port() == 0) {
-    msgr->set_myaddr(listen_addr);
-  }
-  entity_addr_t addr = msgr->get_myaddr();
-  addr.nonce = nonce;
-  msgr->set_myaddr(addr);
-
-  msgr->init_local_connection();
-
-  ldout(msgr->cct,1) << "accepter.bind my_inst.addr is " << msgr->get_myaddr()
-                    << " need_addr=" << msgr->get_need_addr() << dendl;
-  return 0;
-}
-
-int Accepter::rebind(const set<int>& avoid_ports)
-{
-  ldout(msgr->cct,1) << "accepter.rebind avoid " << avoid_ports << dendl;
-  
-  entity_addr_t addr = msgr->get_myaddr();
-  set<int> new_avoid = avoid_ports;
-  new_avoid.insert(addr.get_port());
-  addr.set_port(0);
-
-  // adjust the nonce; we want our entity_addr_t to be truly unique.
-  nonce += 1000000;
-  msgr->my_inst.addr.nonce = nonce;
-  ldout(msgr->cct,10) << " new nonce " << nonce << " and inst " << msgr->my_inst << dendl;
-
-  ldout(msgr->cct,10) << " will try " << addr << " and avoid ports " << new_avoid << dendl;
-  int r = bind(addr, new_avoid);
-  if (r == 0)
-    start();
-  return r;
-}
-
-int Accepter::start()
-{
-  ldout(msgr->cct,1) << "accepter.start" << dendl;
-
-  // start thread
-  create();
-
-  return 0;
-}
-
-void *Accepter::entry()
-{
-  ldout(msgr->cct,10) << "accepter starting" << dendl;
-  
-  int errors = 0;
-
-  struct pollfd pfd;
-  pfd.fd = listen_sd;
-  pfd.events = POLLIN | POLLERR | POLLNVAL | POLLHUP;
-  while (!done) {
-    ldout(msgr->cct,20) << "accepter calling poll" << dendl;
-    int r = poll(&pfd, 1, -1);
-    if (r < 0)
-      break;
-    ldout(msgr->cct,20) << "accepter poll got " << r << dendl;
-
-    if (pfd.revents & (POLLERR | POLLNVAL | POLLHUP))
-      break;
-
-    ldout(msgr->cct,10) << "pfd.revents=" << pfd.revents << dendl;
-    if (done) break;
-
-    // accept
-    entity_addr_t addr;
-    socklen_t slen = sizeof(addr.ss_addr());
-    int sd = ::accept(listen_sd, (sockaddr*)&addr.ss_addr(), &slen);
-    if (sd >= 0) {
-      errors = 0;
-      ldout(msgr->cct,10) << "accepted incoming on sd " << sd << dendl;
-      
-      msgr->add_accept_pipe(sd);
-    } else {
-      ldout(msgr->cct,0) << "accepter no incoming connection?  sd = " << sd
-             << " errno " << errno << " " << cpp_strerror(errno) << dendl;
-      if (++errors > 4)
-       break;
-    }
-  }
-
-  ldout(msgr->cct,20) << "accepter closing" << dendl;
-  // don't close socket, in case we start up again?  blech.
-  if (listen_sd >= 0) {
-    ::close(listen_sd);
-    listen_sd = -1;
-  }
-  ldout(msgr->cct,10) << "accepter stopping" << dendl;
-  return 0;
-}
-
-void Accepter::stop()
-{
-  done = true;
-  ldout(msgr->cct,10) << "stop accepter" << dendl;
-
-  if (listen_sd >= 0) {
-    ::shutdown(listen_sd, SHUT_RDWR);
-  }
-
-  // wait for thread to stop before closing the socket, to avoid
-  // racing against fd re-use.
-  if (is_started()) {
-    join();
-  }
-
-  if (listen_sd >= 0) {
-    ::close(listen_sd);
-    listen_sd = -1;
-  }
-  done = false;
-}
-
-
-
-
diff --git a/src/msg/Accepter.h b/src/msg/Accepter.h
deleted file mode 100644 (file)
index 4b1421f..0000000
+++ /dev/null
@@ -1,44 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software 
- * Foundation.  See file COPYING.
- * 
- */
-
-#ifndef CEPH_MSG_ACCEPTER_H
-#define CEPH_MSG_ACCEPTER_H
-
-#include "msg/msg_types.h"
-#include "common/Thread.h"
-
-class SimpleMessenger;
-
-/**
- * If the SimpleMessenger binds to a specific address, the Accepter runs
- * and listens for incoming connections.
- */
-class Accepter : public Thread {
-  SimpleMessenger *msgr;
-  bool done;
-  int listen_sd;
-  uint64_t nonce;
-
-public:
-  Accepter(SimpleMessenger *r, uint64_t n) : msgr(r), done(false), listen_sd(-1), nonce(n) {}
-    
-  void *entry();
-  void stop();
-  int bind(const entity_addr_t &bind_addr, const set<int>& avoid_ports);
-  int rebind(const set<int>& avoid_port);
-  int start();
-};
-
-
-#endif
diff --git a/src/msg/DispatchQueue.cc b/src/msg/DispatchQueue.cc
deleted file mode 100644 (file)
index 12c4846..0000000
+++ /dev/null
@@ -1,243 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software 
- * Foundation.  See file COPYING.
- * 
- */
-
-#include "msg/Message.h"
-#include "DispatchQueue.h"
-#include "SimpleMessenger.h"
-#include "common/ceph_context.h"
-
-#define dout_subsys ceph_subsys_ms
-#include "common/debug.h"
-
-
-/*******************
- * DispatchQueue
- */
-
-#undef dout_prefix
-#define dout_prefix *_dout << "-- " << msgr->get_myaddr() << " "
-
-double DispatchQueue::get_max_age(utime_t now) {
-  Mutex::Locker l(lock);
-  if (marrival.empty())
-    return 0;
-  else
-    return (now - marrival.begin()->first);
-}
-
-uint64_t DispatchQueue::pre_dispatch(Message *m)
-{
-  ldout(cct,1) << "<== " << m->get_source_inst()
-              << " " << m->get_seq()
-              << " ==== " << *m
-              << " ==== " << m->get_payload().length()
-              << "+" << m->get_middle().length()
-              << "+" << m->get_data().length()
-              << " (" << m->get_footer().front_crc << " "
-              << m->get_footer().middle_crc
-              << " " << m->get_footer().data_crc << ")"
-              << " " << m << " con " << m->get_connection()
-              << dendl;
-  uint64_t msize = m->get_dispatch_throttle_size();
-  m->set_dispatch_throttle_size(0); // clear it out, in case we requeue this message.
-  return msize;
-}
-
-void DispatchQueue::post_dispatch(Message *m, uint64_t msize)
-{
-  msgr->dispatch_throttle_release(msize);
-  ldout(cct,20) << "done calling dispatch on " << m << dendl;
-}
-
-bool DispatchQueue::can_fast_dispatch(Message *m)
-{
-  return msgr->ms_can_fast_dispatch(m);
-}
-
-void DispatchQueue::fast_dispatch(Message *m)
-{
-  uint64_t msize = pre_dispatch(m);
-  msgr->ms_fast_dispatch(m);
-  post_dispatch(m, msize);
-}
-
-void DispatchQueue::fast_preprocess(Message *m)
-{
-  msgr->ms_fast_preprocess(m);
-}
-
-void DispatchQueue::enqueue(Message *m, int priority, uint64_t id)
-{
-
-  Mutex::Locker l(lock);
-  ldout(cct,20) << "queue " << m << " prio " << priority << dendl;
-  add_arrival(m);
-  if (priority >= CEPH_MSG_PRIO_LOW) {
-    mqueue.enqueue_strict(
-        id, priority, QueueItem(m));
-  } else {
-    mqueue.enqueue(
-        id, priority, m->get_cost(), QueueItem(m));
-  }
-  cond.Signal();
-}
-
-void DispatchQueue::local_delivery(Message *m, int priority)
-{
-  m->set_connection(msgr->local_connection.get());
-  m->set_recv_stamp(ceph_clock_now(msgr->cct));
-  Mutex::Locker l(local_delivery_lock);
-  if (local_messages.empty())
-    local_delivery_cond.Signal();
-  local_messages.push_back(make_pair(m, priority));
-  return;
-}
-
-void DispatchQueue::run_local_delivery()
-{
-  local_delivery_lock.Lock();
-  while (true) {
-    if (stop_local_delivery)
-      break;
-    if (local_messages.empty()) {
-      local_delivery_cond.Wait(local_delivery_lock);
-      continue;
-    }
-    pair<Message *, int> mp = local_messages.front();
-    local_messages.pop_front();
-    local_delivery_lock.Unlock();
-    Message *m = mp.first;
-    int priority = mp.second;
-    fast_preprocess(m);
-    if (can_fast_dispatch(m)) {
-      fast_dispatch(m);
-    } else {
-      Mutex::Locker l(lock);
-      add_arrival(m);
-      if (priority >= CEPH_MSG_PRIO_LOW) {
-        mqueue.enqueue_strict(
-            0, priority, QueueItem(m));
-      } else {
-        mqueue.enqueue(
-            0, priority, m->get_cost(), QueueItem(m));
-      }
-      cond.Signal();
-    }
-    local_delivery_lock.Lock();
-  }
-  local_delivery_lock.Unlock();
-}
-
-/*
- * This function delivers incoming messages to the Messenger.
- * Pipes with messages are kept in queues; when beginning a message
- * delivery the highest-priority queue is selected, the pipe from the
- * front of the queue is removed, and its message read. If the pipe
- * has remaining messages at that priority level, it is re-placed on to the
- * end of the queue. If the queue is empty; it's removed.
- * The message is then delivered and the process starts again.
- */
-void DispatchQueue::entry()
-{
-  lock.Lock();
-  while (true) {
-    while (!mqueue.empty()) {
-      QueueItem qitem = mqueue.dequeue();
-      if (!qitem.is_code())
-       remove_arrival(qitem.get_message());
-      lock.Unlock();
-
-      if (qitem.is_code()) {
-       switch (qitem.get_code()) {
-       case D_BAD_REMOTE_RESET:
-         msgr->ms_deliver_handle_remote_reset(qitem.get_connection());
-         break;
-       case D_CONNECT:
-         msgr->ms_deliver_handle_connect(qitem.get_connection());
-         break;
-       case D_ACCEPT:
-         msgr->ms_deliver_handle_accept(qitem.get_connection());
-         break;
-       case D_BAD_RESET:
-         msgr->ms_deliver_handle_reset(qitem.get_connection());
-         break;
-       default:
-         assert(0);
-       }
-      } else {
-       Message *m = qitem.get_message();
-       if (stop) {
-         ldout(cct,10) << " stop flag set, discarding " << m << " " << *m << dendl;
-         m->put();
-       } else {
-         uint64_t msize = pre_dispatch(m);
-         msgr->ms_deliver_dispatch(m);
-         post_dispatch(m, msize);
-       }
-      }
-
-      lock.Lock();
-    }
-    if (stop)
-      break;
-
-    // wait for something to be put on queue
-    cond.Wait(lock);
-  }
-  lock.Unlock();
-}
-
-void DispatchQueue::discard_queue(uint64_t id) {
-  Mutex::Locker l(lock);
-  list<QueueItem> removed;
-  mqueue.remove_by_class(id, &removed);
-  for (list<QueueItem>::iterator i = removed.begin();
-       i != removed.end();
-       ++i) {
-    assert(!(i->is_code())); // We don't discard id 0, ever!
-    Message *m = i->get_message();
-    remove_arrival(m);
-    msgr->dispatch_throttle_release(m->get_dispatch_throttle_size());
-    m->put();
-  }
-}
-
-void DispatchQueue::start()
-{
-  assert(!stop);
-  assert(!dispatch_thread.is_started());
-  dispatch_thread.create();
-  local_delivery_thread.create();
-}
-
-void DispatchQueue::wait()
-{
-  local_delivery_thread.join();
-  dispatch_thread.join();
-}
-
-void DispatchQueue::shutdown()
-{
-  // stop my local delivery thread
-  local_delivery_lock.Lock();
-  stop_local_delivery = true;
-  local_delivery_cond.Signal();
-  local_delivery_lock.Unlock();
-
-  // stop my dispatch thread
-  lock.Lock();
-  stop = true;
-  cond.Signal();
-  lock.Unlock();
-}
diff --git a/src/msg/DispatchQueue.h b/src/msg/DispatchQueue.h
deleted file mode 100644 (file)
index 5fe17dc..0000000
+++ /dev/null
@@ -1,208 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software 
- * Foundation.  See file COPYING.
- * 
- */
-
-#ifndef CEPH_DISPATCHQUEUE_H
-#define CEPH_DISPATCHQUEUE_H
-
-#include <map>
-#include <boost/intrusive_ptr.hpp>
-#include "include/assert.h"
-#include "include/xlist.h"
-#include "include/atomic.h"
-#include "common/Mutex.h"
-#include "common/Cond.h"
-#include "common/Thread.h"
-#include "common/RefCountedObj.h"
-#include "common/PrioritizedQueue.h"
-
-class CephContext;
-class DispatchQueue;
-class Pipe;
-class SimpleMessenger;
-class Message;
-struct Connection;
-
-/**
- * The DispatchQueue contains all the Pipes which have Messages
- * they want to be dispatched, carefully organized by Message priority
- * and permitted to deliver in a round-robin fashion.
- * See SimpleMessenger::dispatch_entry for details.
- */
-class DispatchQueue {
-  class QueueItem {
-    int type;
-    ConnectionRef con;
-    MessageRef m;
-  public:
-    QueueItem(Message *m) : type(-1), con(0), m(m) {}
-    QueueItem(int type, Connection *con) : type(type), con(con), m(0) {}
-    bool is_code() const {
-      return type != -1;
-    }
-    int get_code () {
-      assert(is_code());
-      return type;
-    }
-    Message *get_message() {
-      assert(!is_code());
-      return m.get();
-    }
-    Connection *get_connection() {
-      assert(is_code());
-      return con.get();
-    }
-  };
-    
-  CephContext *cct;
-  SimpleMessenger *msgr;
-  Mutex lock;
-  Cond cond;
-
-  PrioritizedQueue<QueueItem, uint64_t> mqueue;
-
-  set<pair<double, Message*> > marrival;
-  map<Message *, set<pair<double, Message*> >::iterator> marrival_map;
-  void add_arrival(Message *m) {
-    marrival_map.insert(
-      make_pair(
-       m,
-       marrival.insert(make_pair(m->get_recv_stamp(), m)).first
-       )
-      );
-  }
-  void remove_arrival(Message *m) {
-    map<Message *, set<pair<double, Message*> >::iterator>::iterator i =
-      marrival_map.find(m);
-    assert(i != marrival_map.end());
-    marrival.erase(i->second);
-    marrival_map.erase(i);
-  }
-
-  uint64_t next_pipe_id;
-    
-  enum { D_CONNECT = 1, D_ACCEPT, D_BAD_REMOTE_RESET, D_BAD_RESET, D_NUM_CODES };
-
-  /**
-   * The DispatchThread runs dispatch_entry to empty out the dispatch_queue.
-   */
-  class DispatchThread : public Thread {
-    DispatchQueue *dq;
-  public:
-    DispatchThread(DispatchQueue *dq) : dq(dq) {}
-    void *entry() {
-      dq->entry();
-      return 0;
-    }
-  } dispatch_thread;
-
-  Mutex local_delivery_lock;
-  Cond local_delivery_cond;
-  bool stop_local_delivery;
-  list<pair<Message *, int> > local_messages;
-  class LocalDeliveryThread : public Thread {
-    DispatchQueue *dq;
-  public:
-    LocalDeliveryThread(DispatchQueue *dq) : dq(dq) {}
-    void *entry() {
-      dq->run_local_delivery();
-      return 0;
-    }
-  } local_delivery_thread;
-
-  uint64_t pre_dispatch(Message *m);
-  void post_dispatch(Message *m, uint64_t msize);
-
-  public:
-  bool stop;
-  void local_delivery(Message *m, int priority);
-  void run_local_delivery();
-
-  double get_max_age(utime_t now);
-
-  int get_queue_len() {
-    Mutex::Locker l(lock);
-    return mqueue.length();
-  }
-    
-  void queue_connect(Connection *con) {
-    Mutex::Locker l(lock);
-    if (stop)
-      return;
-    mqueue.enqueue_strict(
-      0,
-      CEPH_MSG_PRIO_HIGHEST,
-      QueueItem(D_CONNECT, con));
-    cond.Signal();
-  }
-  void queue_accept(Connection *con) {
-    Mutex::Locker l(lock);
-    if (stop)
-      return;
-    mqueue.enqueue_strict(
-      0,
-      CEPH_MSG_PRIO_HIGHEST,
-      QueueItem(D_ACCEPT, con));
-    cond.Signal();
-  }
-  void queue_remote_reset(Connection *con) {
-    Mutex::Locker l(lock);
-    if (stop)
-      return;
-    mqueue.enqueue_strict(
-      0,
-      CEPH_MSG_PRIO_HIGHEST,
-      QueueItem(D_BAD_REMOTE_RESET, con));
-    cond.Signal();
-  }
-  void queue_reset(Connection *con) {
-    Mutex::Locker l(lock);
-    if (stop)
-      return;
-    mqueue.enqueue_strict(
-      0,
-      CEPH_MSG_PRIO_HIGHEST,
-      QueueItem(D_BAD_RESET, con));
-    cond.Signal();
-  }
-
-  bool can_fast_dispatch(Message *m);
-  void fast_dispatch(Message *m);
-  void fast_preprocess(Message *m);
-  void enqueue(Message *m, int priority, uint64_t id);
-  void discard_queue(uint64_t id);
-  uint64_t get_id() {
-    Mutex::Locker l(lock);
-    return next_pipe_id++;
-  }
-  void start();
-  void entry();
-  void wait();
-  void shutdown();
-  bool is_started() {return dispatch_thread.is_started();}
-
-  DispatchQueue(CephContext *cct, SimpleMessenger *msgr)
-    : cct(cct), msgr(msgr),
-      lock("SimpleMessenger::DispatchQeueu::lock"), 
-      mqueue(cct->_conf->ms_pq_max_tokens_per_priority,
-            cct->_conf->ms_pq_min_cost),
-      next_pipe_id(1),
-      dispatch_thread(this),
-      local_delivery_lock("SimpleMessenger::DispatchQueue::local_delivery_lock"),
-      stop_local_delivery(false),
-      local_delivery_thread(this),
-      stop(false)
-    {}
-};
-
-#endif
index e4a94106127d89aa7ebf1b9597f4896edddd12d5..189ecfa9661198cd236502aee3b91e8875569136 100644 (file)
@@ -1,24 +1,32 @@
 libmsg_la_SOURCES = \
-       msg/Accepter.cc \
-       msg/DispatchQueue.cc \
        msg/Message.cc \
        msg/Messenger.cc \
-       msg/Pipe.cc \
-       msg/PipeConnection.cc \
-       msg/SimpleMessenger.cc \
        msg/msg_types.cc
 
 noinst_HEADERS += \
-       msg/Accepter.h \
        msg/Connection.h \
-       msg/DispatchQueue.h \
        msg/Dispatcher.h \
        msg/Message.h \
        msg/Messenger.h \
-       msg/Pipe.h \
-       msg/PipeConnection.h \
-       msg/SimpleMessenger.h \
        msg/SimplePolicyMessenger.h \
        msg/msg_types.h
 
+# simple
+libmsg_la_SOURCES += \
+       msg/simple/Accepter.cc \
+       msg/simple/DispatchQueue.cc \
+       msg/simple/Pipe.cc \
+       msg/simple/PipeConnection.cc \
+       msg/simple/SimpleMessenger.cc
+
+noinst_HEADERS += \
+       msg/simple/Accepter.h \
+       msg/simple/DispatchQueue.h \
+       msg/simple/Pipe.h \
+       msg/simple/PipeConnection.h \
+       msg/simple/SimpleMessenger.h
+
+
 noinst_LTLIBRARIES += libmsg.la
+
+
index c02929c63c4d8e531ebc949fa3cf694d939c0873..8be11d44ac72327bcc4df8bb2f64f1f25989f0d1 100644 (file)
@@ -2,7 +2,7 @@
 #include "include/types.h"
 #include "Messenger.h"
 
-#include "SimpleMessenger.h"
+#include "msg/simple/SimpleMessenger.h"
 
 Messenger *Messenger::create(CephContext *cct,
                             entity_name_t name,
diff --git a/src/msg/Pipe.cc b/src/msg/Pipe.cc
deleted file mode 100644 (file)
index aaa04cb..0000000
+++ /dev/null
@@ -1,2488 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software 
- * Foundation.  See file COPYING.
- * 
- */
-
-#include <sys/socket.h>
-#include <netinet/tcp.h>
-#include <sys/uio.h>
-#include <limits.h>
-#include <poll.h>
-
-#include "Message.h"
-#include "Pipe.h"
-#include "SimpleMessenger.h"
-
-#include "common/debug.h"
-#include "common/errno.h"
-
-// Below included to get encode_encrypt(); That probably should be in Crypto.h, instead
-
-#include "auth/Crypto.h"
-#include "auth/cephx/CephxProtocol.h"
-#include "auth/AuthSessionHandler.h"
-
-// Constant to limit starting sequence number to 2^31.  Nothing special about it, just a big number.  PLR
-#define SEQ_MASK  0x7fffffff 
-#define dout_subsys ceph_subsys_ms
-
-#undef dout_prefix
-#define dout_prefix _pipe_prefix(_dout)
-ostream& Pipe::_pipe_prefix(std::ostream *_dout) {
-  return *_dout << "-- " << msgr->get_myinst().addr << " >> " << peer_addr << " pipe(" << this
-               << " sd=" << sd << " :" << port
-               << " s=" << state
-               << " pgs=" << peer_global_seq
-               << " cs=" << connect_seq
-               << " l=" << policy.lossy
-               << " c=" << connection_state
-               << ").";
-}
-
-/*
- * This optimization may not be available on all platforms (e.g. OSX).
- * Apparently a similar approach based on TCP_CORK can be used.
- */
-#ifndef MSG_MORE
-# define MSG_MORE 0
-#endif
-
-/*
- * On BSD SO_NOSIGPIPE can be set via setsockopt to block SIGPIPE.
- */
-#ifndef MSG_NOSIGNAL
-# define MSG_NOSIGNAL 0
-# ifdef SO_NOSIGPIPE
-#  define CEPH_USE_SO_NOSIGPIPE
-# else
-#  error "Cannot block SIGPIPE!"
-# endif
-#endif
-
-/**************************************
- * Pipe
- */
-
-Pipe::Pipe(SimpleMessenger *r, int st, PipeConnection *con)
-  : RefCountedObject(r->cct),
-    reader_thread(this),
-    writer_thread(this),
-    delay_thread(NULL),
-    msgr(r),
-    conn_id(r->dispatch_queue.get_id()),
-    recv_ofs(0),
-    recv_len(0),
-    sd(-1), port(0),
-    peer_type(-1),
-    pipe_lock("SimpleMessenger::Pipe::pipe_lock"),
-    state(st),
-    connection_state(NULL),
-    reader_running(false), reader_needs_join(false),
-    reader_dispatching(false),
-    writer_running(false),
-    in_q(&(r->dispatch_queue)),
-    send_keepalive(false),
-    send_keepalive_ack(false),
-    connect_seq(0), peer_global_seq(0),
-    out_seq(0), in_seq(0), in_seq_acked(0) {
-  if (con) {
-    connection_state = con;
-    connection_state->reset_pipe(this);
-  } else {
-    connection_state = new PipeConnection(msgr->cct, msgr);
-    connection_state->pipe = get();
-  }
-
-  if (randomize_out_seq()) {
-    lsubdout(msgr->cct,ms,15) << "Pipe(): Could not get random bytes to set seq number for session reset; set seq number to " << out_seq << dendl;
-  }
-    
-
-  msgr->timeout = msgr->cct->_conf->ms_tcp_read_timeout * 1000; //convert to ms
-  if (msgr->timeout == 0)
-    msgr->timeout = -1;
-
-  recv_max_prefetch = msgr->cct->_conf->ms_tcp_prefetch_max_size;
-  recv_buf = new char[recv_max_prefetch];
-}
-
-Pipe::~Pipe()
-{
-  assert(out_q.empty());
-  assert(sent.empty());
-  delete delay_thread;
-  delete[] recv_buf;
-}
-
-void Pipe::handle_ack(uint64_t seq)
-{
-  lsubdout(msgr->cct, ms, 15) << "reader got ack seq " << seq << dendl;
-  // trim sent list
-  while (!sent.empty() &&
-        sent.front()->get_seq() <= seq) {
-    Message *m = sent.front();
-    sent.pop_front();
-    lsubdout(msgr->cct, ms, 10) << "reader got ack seq "
-                               << seq << " >= " << m->get_seq() << " on " << m << " " << *m << dendl;
-    m->put();
-  }
-}
-
-void Pipe::start_reader()
-{
-  assert(pipe_lock.is_locked());
-  assert(!reader_running);
-  if (reader_needs_join) {
-    reader_thread.join();
-    reader_needs_join = false;
-  }
-  reader_running = true;
-  reader_thread.create(msgr->cct->_conf->ms_rwthread_stack_bytes);
-}
-
-void Pipe::maybe_start_delay_thread()
-{
-  if (!delay_thread &&
-      msgr->cct->_conf->ms_inject_delay_type.find(ceph_entity_type_name(connection_state->peer_type)) != string::npos) {
-    lsubdout(msgr->cct, ms, 1) << "setting up a delay queue on Pipe " << this << dendl;
-    delay_thread = new DelayedDelivery(this);
-    delay_thread->create();
-  }
-}
-
-void Pipe::start_writer()
-{
-  assert(pipe_lock.is_locked());
-  assert(!writer_running);
-  writer_running = true;
-  writer_thread.create(msgr->cct->_conf->ms_rwthread_stack_bytes);
-}
-
-void Pipe::join_reader()
-{
-  if (!reader_running)
-    return;
-  cond.Signal();
-  pipe_lock.Unlock();
-  reader_thread.join();
-  pipe_lock.Lock();
-  reader_needs_join = false;
-}
-
-void Pipe::DelayedDelivery::discard()
-{
-  lgeneric_subdout(pipe->msgr->cct, ms, 20) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::discard" << dendl;
-  Mutex::Locker l(delay_lock);
-  while (!delay_queue.empty()) {
-    Message *m = delay_queue.front().second;
-    pipe->msgr->dispatch_throttle_release(m->get_dispatch_throttle_size());
-    m->put();
-    delay_queue.pop_front();
-  }
-}
-
-void Pipe::DelayedDelivery::flush()
-{
-  lgeneric_subdout(pipe->msgr->cct, ms, 20) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::flush" << dendl;
-  Mutex::Locker l(delay_lock);
-  flush_count = delay_queue.size();
-  delay_cond.Signal();
-}
-
-void *Pipe::DelayedDelivery::entry()
-{
-  Mutex::Locker locker(delay_lock);
-  lgeneric_subdout(pipe->msgr->cct, ms, 20) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::entry start" << dendl;
-
-  while (!stop_delayed_delivery) {
-    if (delay_queue.empty()) {
-      lgeneric_subdout(pipe->msgr->cct, ms, 30) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::entry sleeping on delay_cond because delay queue is empty" << dendl;
-      delay_cond.Wait(delay_lock);
-      continue;
-    }
-    utime_t release = delay_queue.front().first;
-    Message *m = delay_queue.front().second;
-    string delay_msg_type = pipe->msgr->cct->_conf->ms_inject_delay_msg_type;
-    if (!flush_count &&
-        (release > ceph_clock_now(pipe->msgr->cct) &&
-         (delay_msg_type.empty() || m->get_type_name() == delay_msg_type))) {
-      lgeneric_subdout(pipe->msgr->cct, ms, 10) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::entry sleeping on delay_cond until " << release << dendl;
-      delay_cond.WaitUntil(delay_lock, release);
-      continue;
-    }
-    lgeneric_subdout(pipe->msgr->cct, ms, 10) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::entry dequeuing message " << m << " for delivery, past " << release << dendl;
-    delay_queue.pop_front();
-    if (flush_count > 0) {
-      --flush_count;
-      active_flush = true;
-    }
-    if (pipe->in_q->can_fast_dispatch(m)) {
-      if (!stop_fast_dispatching_flag) {
-        delay_dispatching = true;
-        delay_lock.Unlock();
-        pipe->in_q->fast_dispatch(m);
-        delay_lock.Lock();
-        delay_dispatching = false;
-        if (stop_fast_dispatching_flag) {
-          // we need to let the stopping thread proceed
-          delay_cond.Signal();
-          delay_lock.Unlock();
-          delay_lock.Lock();
-        }
-      }
-    } else {
-      pipe->in_q->enqueue(m, m->get_priority(), pipe->conn_id);
-    }
-    active_flush = false;
-  }
-  lgeneric_subdout(pipe->msgr->cct, ms, 20) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::entry stop" << dendl;
-  return NULL;
-}
-
-void Pipe::DelayedDelivery::stop_fast_dispatching() {
-  Mutex::Locker l(delay_lock);
-  stop_fast_dispatching_flag = true;
-  // we can't block if we're the delay thread; see Pipe::stop_and_wait()
-  while (delay_dispatching && !am_self())
-    delay_cond.Wait(delay_lock);
-}
-
-
-int Pipe::accept()
-{
-  ldout(msgr->cct,10) << "accept" << dendl;
-  assert(pipe_lock.is_locked());
-  assert(state == STATE_ACCEPTING);
-
-  pipe_lock.Unlock();
-
-  // vars
-  bufferlist addrs;
-  entity_addr_t socket_addr;
-  socklen_t len;
-  int r;
-  char banner[strlen(CEPH_BANNER)+1];
-  bufferlist addrbl;
-  ceph_msg_connect connect;
-  ceph_msg_connect_reply reply;
-  Pipe *existing = 0;
-  bufferptr bp;
-  bufferlist authorizer, authorizer_reply;
-  bool authorizer_valid;
-  uint64_t feat_missing;
-  bool replaced = false;
-  // this variable denotes if the connection attempt from peer is a hard 
-  // reset or not, it is true if there is an existing connection and the
-  // connection sequence from peer is equal to zero
-  bool is_reset_from_peer = false;
-  CryptoKey session_key;
-  int removed; // single-use down below
-
-  // this should roughly mirror pseudocode at
-  //  http://ceph.newdream.net/wiki/Messaging_protocol
-  int reply_tag = 0;
-  uint64_t existing_seq = -1;
-
-  // used for reading in the remote acked seq on connect
-  uint64_t newly_acked_seq = 0;
-
-  recv_reset();
-
-  set_socket_options();
-
-  // announce myself.
-  r = tcp_write(CEPH_BANNER, strlen(CEPH_BANNER));
-  if (r < 0) {
-    ldout(msgr->cct,10) << "accept couldn't write banner" << dendl;
-    goto fail_unlocked;
-  }
-
-  // and my addr
-  ::encode(msgr->my_inst.addr, addrs);
-
-  port = msgr->my_inst.addr.get_port();
-
-  // and peer's socket addr (they might not know their ip)
-  len = sizeof(socket_addr.ss_addr());
-  r = ::getpeername(sd, (sockaddr*)&socket_addr.ss_addr(), &len);
-  if (r < 0) {
-    ldout(msgr->cct,0) << "accept failed to getpeername " << cpp_strerror(errno) << dendl;
-    goto fail_unlocked;
-  }
-  ::encode(socket_addr, addrs);
-
-  r = tcp_write(addrs.c_str(), addrs.length());
-  if (r < 0) {
-    ldout(msgr->cct,10) << "accept couldn't write my+peer addr" << dendl;
-    goto fail_unlocked;
-  }
-
-  ldout(msgr->cct,1) << "accept sd=" << sd << " " << socket_addr << dendl;
-  
-  // identify peer
-  if (tcp_read(banner, strlen(CEPH_BANNER)) < 0) {
-    ldout(msgr->cct,10) << "accept couldn't read banner" << dendl;
-    goto fail_unlocked;
-  }
-  if (memcmp(banner, CEPH_BANNER, strlen(CEPH_BANNER))) {
-    banner[strlen(CEPH_BANNER)] = 0;
-    ldout(msgr->cct,1) << "accept peer sent bad banner '" << banner << "' (should be '" << CEPH_BANNER << "')" << dendl;
-    goto fail_unlocked;
-  }
-  {
-    bufferptr tp(sizeof(peer_addr));
-    addrbl.push_back(tp);
-  }
-  if (tcp_read(addrbl.c_str(), addrbl.length()) < 0) {
-    ldout(msgr->cct,10) << "accept couldn't read peer_addr" << dendl;
-    goto fail_unlocked;
-  }
-  {
-    bufferlist::iterator ti = addrbl.begin();
-    ::decode(peer_addr, ti);
-  }
-
-  ldout(msgr->cct,10) << "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.addr = socket_addr.addr;
-    peer_addr.set_port(port);
-    ldout(msgr->cct,0) << "accept peer addr is really " << peer_addr
-           << " (socket is " << socket_addr << ")" << dendl;
-  }
-  set_peer_addr(peer_addr);  // so that connection_state gets set up
-  
-  while (1) {
-    if (tcp_read((char*)&connect, sizeof(connect)) < 0) {
-      ldout(msgr->cct,10) << "accept couldn't read connect" << dendl;
-      goto fail_unlocked;
-    }
-
-    // sanitize features
-    connect.features = ceph_sanitize_features(connect.features);
-
-    authorizer.clear();
-    if (connect.authorizer_len) {
-      bp = buffer::create(connect.authorizer_len);
-      if (tcp_read(bp.c_str(), connect.authorizer_len) < 0) {
-        ldout(msgr->cct,10) << "accept couldn't read connect authorizer" << dendl;
-        goto fail_unlocked;
-      }
-      authorizer.push_back(bp);
-      authorizer_reply.clear();
-    }
-
-    ldout(msgr->cct,20) << "accept got peer connect_seq " << connect.connect_seq
-            << " global_seq " << connect.global_seq
-            << dendl;
-    
-    msgr->lock.Lock();   // FIXME
-    pipe_lock.Lock();
-    if (msgr->dispatch_queue.stop)
-      goto shutting_down;
-    if (state != STATE_ACCEPTING) {
-      goto shutting_down;
-    }
-
-    // note peer's type, flags
-    set_peer_type(connect.host_type);
-    policy = msgr->get_policy(connect.host_type);
-    ldout(msgr->cct,10) << "accept of host_type " << connect.host_type
-                       << ", policy.lossy=" << policy.lossy
-                       << " policy.server=" << policy.server
-                       << " policy.standby=" << policy.standby
-                       << " policy.resetcheck=" << policy.resetcheck
-                       << dendl;
-
-    memset(&reply, 0, sizeof(reply));
-    reply.protocol_version = msgr->get_proto_version(peer_type, false);
-    msgr->lock.Unlock();
-
-    // mismatch?
-    ldout(msgr->cct,10) << "accept my proto " << reply.protocol_version
-            << ", their proto " << connect.protocol_version << dendl;
-    if (connect.protocol_version != reply.protocol_version) {
-      reply.tag = CEPH_MSGR_TAG_BADPROTOVER;
-      goto 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) {
-       if (msgr->cct->_conf->cephx_require_signatures ||
-           msgr->cct->_conf->cephx_cluster_require_signatures) {
-         ldout(msgr->cct,10) << "using cephx, requiring MSG_AUTH feature bit for cluster" << dendl;
-         policy.features_required |= CEPH_FEATURE_MSG_AUTH;
-       }
-      } else {
-       if (msgr->cct->_conf->cephx_require_signatures ||
-           msgr->cct->_conf->cephx_service_require_signatures) {
-         ldout(msgr->cct,10) << "using cephx, requiring MSG_AUTH feature bit for service" << dendl;
-         policy.features_required |= CEPH_FEATURE_MSG_AUTH;
-       }
-      }
-    }
-
-    feat_missing = policy.features_required & ~(uint64_t)connect.features;
-    if (feat_missing) {
-      ldout(msgr->cct,1) << "peer missing required features " << std::hex << feat_missing << std::dec << dendl;
-      reply.tag = CEPH_MSGR_TAG_FEATURES;
-      goto reply;
-    }
-    
-    // Check the authorizer.  If not good, bail out.
-
-    pipe_lock.Unlock();
-
-    if (!msgr->verify_authorizer(connection_state.get(), peer_type, connect.authorizer_protocol, authorizer,
-                                authorizer_reply, authorizer_valid, session_key) ||
-       !authorizer_valid) {
-      ldout(msgr->cct,0) << "accept: got bad authorizer" << dendl;
-      pipe_lock.Lock();
-      if (state != STATE_ACCEPTING)
-       goto shutting_down_msgr_unlocked;
-      reply.tag = CEPH_MSGR_TAG_BADAUTHORIZER;
-      session_security.reset();
-      goto reply;
-    } 
-
-    // We've verified the authorizer for this pipe, so set up the session security structure.  PLR
-
-    ldout(msgr->cct,10) << "accept:  setting up session_security." << dendl;
-
-    msgr->lock.Lock();
-    pipe_lock.Lock();
-    if (msgr->dispatch_queue.stop)
-      goto shutting_down;
-    if (state != STATE_ACCEPTING)
-      goto shutting_down;
-    
-    // existing?
-    existing = msgr->_lookup_pipe(peer_addr);
-    if (existing) {
-      existing->pipe_lock.Lock(true);  // skip lockdep check (we are locking a second Pipe here)
-
-      if (connect.global_seq < existing->peer_global_seq) {
-       ldout(msgr->cct,10) << "accept existing " << existing << ".gseq " << existing->peer_global_seq
-                << " > " << connect.global_seq << ", RETRY_GLOBAL" << dendl;
-       reply.tag = CEPH_MSGR_TAG_RETRY_GLOBAL;
-       reply.global_seq = existing->peer_global_seq;  // so we can send it below..
-       existing->pipe_lock.Unlock();
-       msgr->lock.Unlock();
-       goto reply;
-      } else {
-       ldout(msgr->cct,10) << "accept existing " << existing << ".gseq " << existing->peer_global_seq
-                << " <= " << connect.global_seq << ", looks ok" << dendl;
-      }
-      
-      if (existing->policy.lossy) {
-       ldout(msgr->cct,0) << "accept replacing existing (lossy) channel (new one lossy="
-               << policy.lossy << ")" << dendl;
-       existing->was_session_reset();
-       goto replace;
-      }
-
-      ldout(msgr->cct,0) << "accept connect_seq " << connect.connect_seq
-                        << " vs existing " << existing->connect_seq
-                        << " state " << existing->get_state_name() << dendl;
-
-      if (connect.connect_seq == 0 && existing->connect_seq > 0) {
-       ldout(msgr->cct,0) << "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;
-      }
-
-      if (connect.connect_seq < existing->connect_seq) {
-       // old attempt, or we sent READY but they didn't get it.
-       ldout(msgr->cct,10) << "accept existing " << existing << ".cseq " << existing->connect_seq
-                           << " > " << connect.connect_seq << ", RETRY_SESSION" << dendl;
-       goto retry_session;
-      }
-
-      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(msgr->cct,10) << "accept connection race, existing " << existing
-                             << ".cseq " << existing->connect_seq
-                             << " == " << connect.connect_seq
-                             << ", OPEN|STANDBY, RETRY_SESSION" << dendl;
-         goto retry_session;
-       }
-
-       // connection race?
-       if (peer_addr < msgr->my_inst.addr ||
-           existing->policy.server) {
-         // incoming wins
-         ldout(msgr->cct,10) << "accept connection race, existing " << existing << ".cseq " << existing->connect_seq
-                  << " == " << connect.connect_seq << ", or we are server, replacing my attempt" << dendl;
-         if (!(existing->state == STATE_CONNECTING ||
-               existing->state == STATE_WAIT))
-           lderr(msgr->cct) << "accept race bad state, would replace, existing="
-                            << existing->get_state_name()
-                            << " " << existing << ".cseq=" << existing->connect_seq
-                            << " == " << connect.connect_seq
-                            << dendl;
-         assert(existing->state == STATE_CONNECTING ||
-                existing->state == STATE_WAIT);
-         goto replace;
-       } else {
-         // our existing outgoing wins
-         ldout(msgr->cct,10) << "accept connection race, existing " << existing << ".cseq " << existing->connect_seq
-                  << " == " << connect.connect_seq << ", sending WAIT" << dendl;
-         assert(peer_addr > msgr->my_inst.addr);
-         if (!(existing->state == STATE_CONNECTING))
-           lderr(msgr->cct) << "accept race bad state, would send wait, existing="
-                            << existing->get_state_name()
-                            << " " << existing << ".cseq=" << existing->connect_seq
-                            << " == " << connect.connect_seq
-                            << dendl;
-         assert(existing->state == STATE_CONNECTING);
-         // make sure our outgoing connection will follow through
-         existing->_send_keepalive();
-         reply.tag = CEPH_MSGR_TAG_WAIT;
-         existing->pipe_lock.Unlock();
-         msgr->lock.Unlock();
-         goto reply;
-       }
-      }
-
-      assert(connect.connect_seq > existing->connect_seq);
-      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(msgr->cct,0) << "accept we reset (peer sent cseq " << connect.connect_seq 
-                << ", " << existing << ".cseq = " << existing->connect_seq
-                << "), sending RESETSESSION" << dendl;
-       reply.tag = CEPH_MSGR_TAG_RESETSESSION;
-       msgr->lock.Unlock();
-       existing->pipe_lock.Unlock();
-       goto reply;
-      }
-
-      // reconnect
-      ldout(msgr->cct,10) << "accept peer sent cseq " << connect.connect_seq
-              << " > " << existing->connect_seq << dendl;
-      goto replace;
-    } // existing
-    else if (policy.resetcheck && connect.connect_seq > 0) {
-      // we reset, and they are opening a new session
-      ldout(msgr->cct,0) << "accept we reset (peer sent cseq " << connect.connect_seq << "), sending RESETSESSION" << dendl;
-      msgr->lock.Unlock();
-      reply.tag = CEPH_MSGR_TAG_RESETSESSION;
-      goto reply;
-    } else {
-      // new session
-      ldout(msgr->cct,10) << "accept new session" << dendl;
-      existing = NULL;
-      goto open;
-    }
-    assert(0);
-
-  retry_session:
-    assert(existing->pipe_lock.is_locked());
-    assert(pipe_lock.is_locked());
-    reply.tag = CEPH_MSGR_TAG_RETRY_SESSION;
-    reply.connect_seq = existing->connect_seq + 1;
-    existing->pipe_lock.Unlock();
-    msgr->lock.Unlock();
-    goto reply;    
-
-  reply:
-    assert(pipe_lock.is_locked());
-    reply.features = ((uint64_t)connect.features & policy.features_supported) | policy.features_required;
-    reply.authorizer_len = authorizer_reply.length();
-    pipe_lock.Unlock();
-    r = tcp_write((char*)&reply, sizeof(reply));
-    if (r < 0)
-      goto fail_unlocked;
-    if (reply.authorizer_len) {
-      r = tcp_write(authorizer_reply.c_str(), authorizer_reply.length());
-      if (r < 0)
-       goto fail_unlocked;
-    }
-  }
-  
- replace:
-  assert(existing->pipe_lock.is_locked());
-  assert(pipe_lock.is_locked());
-  // 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;
-    existing_seq = existing->in_seq;
-  }
-  ldout(msgr->cct,10) << "accept replacing " << existing << dendl;
-  existing->stop();
-  existing->unregister_pipe();
-  replaced = true;
-
-  if (existing->policy.lossy) {
-    // disconnect from the Connection
-    assert(existing->connection_state);
-    if (existing->connection_state->clear_pipe(existing))
-      msgr->dispatch_queue.queue_reset(existing->connection_state.get());
-  } else {
-    // queue a reset on the new connection, which we're dumping for the old
-    msgr->dispatch_queue.queue_reset(connection_state.get());
-
-    // drop my Connection, and take a ref to the existing one. do not
-    // clear existing->connection_state, since read_message and
-    // write_message both dereference it without pipe_lock.
-    connection_state = existing->connection_state;
-
-    // make existing Connection reference us
-    connection_state->reset_pipe(this);
-
-    if (existing->delay_thread) {
-      existing->delay_thread->steal_for_pipe(this);
-      delay_thread = existing->delay_thread;
-      existing->delay_thread = NULL;
-      delay_thread->flush();
-    }
-
-    // steal incoming queue
-    uint64_t replaced_conn_id = conn_id;
-    conn_id = existing->conn_id;
-    existing->conn_id = replaced_conn_id;
-
-    // reset the in_seq if this is a hard reset from peer,
-    // otherwise we respect our original connection's value
-    in_seq = is_reset_from_peer ? 0 : existing->in_seq;
-    in_seq_acked = in_seq;
-
-    // steal outgoing queue and out_seq
-    existing->requeue_sent();
-    out_seq = existing->out_seq;
-    ldout(msgr->cct,10) << "accept re-queuing on out_seq " << out_seq << " in_seq " << in_seq << dendl;
-    for (map<int, list<Message*> >::iterator p = existing->out_q.begin();
-         p != existing->out_q.end();
-         ++p)
-      out_q[p->first].splice(out_q[p->first].begin(), p->second);
-  }
-  existing->stop_and_wait();
-  existing->pipe_lock.Unlock();
-
- open:
-  // open
-  assert(pipe_lock.is_locked());
-  connect_seq = connect.connect_seq + 1;
-  peer_global_seq = connect.global_seq;
-  assert(state == STATE_ACCEPTING);
-  state = STATE_OPEN;
-  ldout(msgr->cct,10) << "accept success, connect_seq = " << connect_seq << ", sending READY" << dendl;
-
-  // send READY reply
-  reply.tag = (reply_tag ? reply_tag : CEPH_MSGR_TAG_READY);
-  reply.features = policy.features_supported;
-  reply.global_seq = 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;
-
-  connection_state->set_features((uint64_t)reply.features & (uint64_t)connect.features);
-  ldout(msgr->cct,10) << "accept features " << connection_state->get_features() << dendl;
-
-  session_security.reset(
-      get_auth_session_handler(msgr->cct,
-                              connect.authorizer_protocol,
-                              session_key,
-                              connection_state->get_features()));
-
-  // notify
-  msgr->dispatch_queue.queue_accept(connection_state.get());
-  msgr->ms_deliver_handle_fast_accept(connection_state.get());
-
-  // ok!
-  if (msgr->dispatch_queue.stop)
-    goto shutting_down;
-  removed = msgr->accepting_pipes.erase(this);
-  assert(removed == 1);
-  register_pipe();
-  msgr->lock.Unlock();
-  pipe_lock.Unlock();
-
-  r = tcp_write((char*)&reply, sizeof(reply));
-  if (r < 0) {
-    goto fail_registered;
-  }
-
-  if (reply.authorizer_len) {
-    r = tcp_write(authorizer_reply.c_str(), authorizer_reply.length());
-    if (r < 0) {
-      goto fail_registered;
-    }
-  }
-
-  if (reply_tag == CEPH_MSGR_TAG_SEQ) {
-    if (tcp_write((char*)&existing_seq, sizeof(existing_seq)) < 0) {
-      ldout(msgr->cct,2) << "accept write error on in_seq" << dendl;
-      goto fail_registered;
-    }
-    if (tcp_read((char*)&newly_acked_seq, sizeof(newly_acked_seq)) < 0) {
-      ldout(msgr->cct,2) << "accept read error on newly_acked_seq" << dendl;
-      goto fail_registered;
-    }
-  }
-
-  pipe_lock.Lock();
-  discard_requeued_up_to(newly_acked_seq);
-  if (state != STATE_CLOSED) {
-    ldout(msgr->cct,10) << "accept starting writer, state " << get_state_name() << dendl;
-    start_writer();
-  }
-  ldout(msgr->cct,20) << "accept done" << dendl;
-
-  maybe_start_delay_thread();
-
-  return 0;   // success.
-
- fail_registered:
-  ldout(msgr->cct, 10) << "accept fault after register" << dendl;
-
-  if (msgr->cct->_conf->ms_inject_internal_delays) {
-    ldout(msgr->cct, 10) << " sleep for " << msgr->cct->_conf->ms_inject_internal_delays << dendl;
-    utime_t t;
-    t.set_from_double(msgr->cct->_conf->ms_inject_internal_delays);
-    t.sleep();
-  }
-
- fail_unlocked:
-  pipe_lock.Lock();
-  if (state != STATE_CLOSED) {
-    bool queued = is_queued();
-    ldout(msgr->cct, 10) << "  queued = " << (int)queued << dendl;
-    if (queued) {
-      state = policy.server ? STATE_STANDBY : STATE_CONNECTING;
-    } else if (replaced) {
-      state = STATE_STANDBY;
-    } else {
-      state = STATE_CLOSED;
-      state_closed.set(1);
-    }
-    fault();
-    if (queued || replaced)
-      start_writer();
-  }
-  return -1;
-
- shutting_down:
-  msgr->lock.Unlock();
- shutting_down_msgr_unlocked:
-  assert(pipe_lock.is_locked());
-
-  if (msgr->cct->_conf->ms_inject_internal_delays) {
-    ldout(msgr->cct, 10) << " sleep for " << msgr->cct->_conf->ms_inject_internal_delays << dendl;
-    utime_t t;
-    t.set_from_double(msgr->cct->_conf->ms_inject_internal_delays);
-    t.sleep();
-  }
-
-  state = STATE_CLOSED;
-  state_closed.set(1);
-  fault();
-  return -1;
-}
-
-void Pipe::set_socket_options()
-{
-  // disable Nagle algorithm?
-  if (msgr->cct->_conf->ms_tcp_nodelay) {
-    int flag = 1;
-    int r = ::setsockopt(sd, IPPROTO_TCP, TCP_NODELAY, (char*)&flag, sizeof(flag));
-    if (r < 0) {
-      r = -errno;
-      ldout(msgr->cct,0) << "couldn't set TCP_NODELAY: " << cpp_strerror(r) << dendl;
-    }
-  }
-  if (msgr->cct->_conf->ms_tcp_rcvbuf) {
-    int size = msgr->cct->_conf->ms_tcp_rcvbuf;
-    int r = ::setsockopt(sd, SOL_SOCKET, SO_RCVBUF, (void*)&size, sizeof(size));
-    if (r < 0)  {
-      r = -errno;
-      ldout(msgr->cct,0) << "couldn't set SO_RCVBUF to " << size << ": " << cpp_strerror(r) << dendl;
-    }
-  }
-
-  // block ESIGPIPE
-#ifdef CEPH_USE_SO_NOSIGPIPE
-  int val = 1;
-  int r = ::setsockopt(sd, SOL_SOCKET, SO_NOSIGPIPE, (void*)&val, sizeof(val));
-  if (r) {
-    r = -errno;
-    ldout(msgr->cct,0) << "couldn't set SO_NOSIGPIPE: " << cpp_strerror(r) << dendl;
-  }
-#endif
-}
-
-int Pipe::connect()
-{
-  bool got_bad_auth = false;
-
-  ldout(msgr->cct,10) << "connect " << connect_seq << dendl;
-  assert(pipe_lock.is_locked());
-
-  __u32 cseq = connect_seq;
-  __u32 gseq = msgr->get_global_seq();
-
-  // stop reader thrad
-  join_reader();
-
-  pipe_lock.Unlock();
-  
-  char tag = -1;
-  int rc;
-  struct msghdr msg;
-  struct iovec msgvec[2];
-  int msglen;
-  char banner[strlen(CEPH_BANNER) + 1];  // extra byte makes coverity happy
-  entity_addr_t paddr;
-  entity_addr_t peer_addr_for_me, socket_addr;
-  AuthAuthorizer *authorizer = NULL;
-  bufferlist addrbl, myaddrbl;
-  const md_config_t *conf = msgr->cct->_conf;
-
-  // close old socket.  this is safe because we stopped the reader thread above.
-  if (sd >= 0)
-    ::close(sd);
-
-  // create socket?
-  sd = ::socket(peer_addr.get_family(), SOCK_STREAM, 0);
-  if (sd < 0) {
-    lderr(msgr->cct) << "connect couldn't created socket " << cpp_strerror(errno) << dendl;
-    goto fail;
-  }
-
-  recv_reset();
-  // connect!
-  ldout(msgr->cct,10) << "connecting to " << peer_addr << dendl;
-  rc = ::connect(sd, (sockaddr*)&peer_addr.addr, peer_addr.addr_size());
-  if (rc < 0) {
-    ldout(msgr->cct,2) << "connect error " << peer_addr
-            << ", " << cpp_strerror(errno) << dendl;
-    goto fail;
-  }
-
-  set_socket_options();
-
-  // verify banner
-  // FIXME: this should be non-blocking, or in some other way verify the banner as we get it.
-  if (tcp_read((char*)&banner, strlen(CEPH_BANNER)) < 0) {
-    ldout(msgr->cct,2) << "connect couldn't read banner, " << cpp_strerror(errno) << dendl;
-    goto fail;
-  }
-  if (memcmp(banner, CEPH_BANNER, strlen(CEPH_BANNER))) {
-    ldout(msgr->cct,0) << "connect protocol error (bad banner) on peer " << peer_addr << dendl;
-    goto fail;
-  }
-
-  memset(&msg, 0, sizeof(msg));
-  msgvec[0].iov_base = banner;
-  msgvec[0].iov_len = strlen(CEPH_BANNER);
-  msg.msg_iov = msgvec;
-  msg.msg_iovlen = 1;
-  msglen = msgvec[0].iov_len;
-  if (do_sendmsg(&msg, msglen)) {
-    ldout(msgr->cct,2) << "connect couldn't write my banner, " << cpp_strerror(errno) << dendl;
-    goto fail;
-  }
-
-  // identify peer
-  {
-    bufferptr p(sizeof(paddr) * 2);
-    addrbl.push_back(p);
-  }
-  if (tcp_read(addrbl.c_str(), addrbl.length()) < 0) {
-    ldout(msgr->cct,2) << "connect couldn't read peer addrs, " << cpp_strerror(errno) << dendl;
-    goto fail;
-  }
-  {
-    bufferlist::iterator p = addrbl.begin();
-    ::decode(paddr, p);
-    ::decode(peer_addr_for_me, p);
-    port = peer_addr_for_me.get_port();
-  }
-
-  ldout(msgr->cct,20) << "connect read peer addr " << paddr << " on socket " << sd << dendl;
-  if (peer_addr != paddr) {
-    if (paddr.is_blank_ip() &&
-       peer_addr.get_port() == paddr.get_port() &&
-       peer_addr.get_nonce() == paddr.get_nonce()) {
-      ldout(msgr->cct,0) << "connect claims to be " 
-             << paddr << " not " << peer_addr << " - presumably this is the same node!" << dendl;
-    } else {
-      ldout(msgr->cct,0) << "connect claims to be " 
-             << paddr << " not " << peer_addr << " - wrong node!" << dendl;
-      goto fail;
-    }
-  }
-
-  ldout(msgr->cct,20) << "connect peer addr for me is " << peer_addr_for_me << dendl;
-
-  msgr->learned_addr(peer_addr_for_me);
-
-  ::encode(msgr->my_inst.addr, myaddrbl);
-
-  memset(&msg, 0, sizeof(msg));
-  msgvec[0].iov_base = myaddrbl.c_str();
-  msgvec[0].iov_len = myaddrbl.length();
-  msg.msg_iov = msgvec;
-  msg.msg_iovlen = 1;
-  msglen = msgvec[0].iov_len;
-  if (do_sendmsg(&msg, msglen)) {
-    ldout(msgr->cct,2) << "connect couldn't write my addr, " << cpp_strerror(errno) << dendl;
-    goto fail;
-  }
-  ldout(msgr->cct,10) << "connect sent my addr " << msgr->my_inst.addr << dendl;
-
-
-  while (1) {
-    delete authorizer;
-    authorizer = msgr->get_authorizer(peer_type, false);
-    bufferlist authorizer_reply;
-
-    ceph_msg_connect connect;
-    connect.features = policy.features_supported;
-    connect.host_type = msgr->get_myinst().name.type();
-    connect.global_seq = gseq;
-    connect.connect_seq = cseq;
-    connect.protocol_version = msgr->get_proto_version(peer_type, true);
-    connect.authorizer_protocol = authorizer ? authorizer->protocol : 0;
-    connect.authorizer_len = authorizer ? authorizer->bl.length() : 0;
-    if (authorizer) 
-      ldout(msgr->cct,10) << "connect.authorizer_len=" << connect.authorizer_len
-              << " protocol=" << connect.authorizer_protocol << dendl;
-    connect.flags = 0;
-    if (policy.lossy)
-      connect.flags |= CEPH_MSG_CONNECT_LOSSY;  // this is fyi, actually, server decides!
-    memset(&msg, 0, sizeof(msg));
-    msgvec[0].iov_base = (char*)&connect;
-    msgvec[0].iov_len = sizeof(connect);
-    msg.msg_iov = msgvec;
-    msg.msg_iovlen = 1;
-    msglen = msgvec[0].iov_len;
-    if (authorizer) {
-      msgvec[1].iov_base = authorizer->bl.c_str();
-      msgvec[1].iov_len = authorizer->bl.length();
-      msg.msg_iovlen++;
-      msglen += msgvec[1].iov_len;
-    }
-
-    ldout(msgr->cct,10) << "connect sending gseq=" << gseq << " cseq=" << cseq
-            << " proto=" << connect.protocol_version << dendl;
-    if (do_sendmsg(&msg, msglen)) {
-      ldout(msgr->cct,2) << "connect couldn't write gseq, cseq, " << cpp_strerror(errno) << dendl;
-      goto fail;
-    }
-
-    ldout(msgr->cct,20) << "connect wrote (self +) cseq, waiting for reply" << dendl;
-    ceph_msg_connect_reply reply;
-    if (tcp_read((char*)&reply, sizeof(reply)) < 0) {
-      ldout(msgr->cct,2) << "connect read reply " << cpp_strerror(errno) << dendl;
-      goto fail;
-    }
-
-    // sanitize features
-    reply.features = ceph_sanitize_features(reply.features);
-
-    ldout(msgr->cct,20) << "connect got reply tag " << (int)reply.tag
-                       << " connect_seq " << reply.connect_seq
-                       << " global_seq " << reply.global_seq
-                       << " proto " << reply.protocol_version
-                       << " flags " << (int)reply.flags
-                       << " features " << reply.features
-                       << dendl;
-
-    authorizer_reply.clear();
-
-    if (reply.authorizer_len) {
-      ldout(msgr->cct,10) << "reply.authorizer_len=" << reply.authorizer_len << dendl;
-      bufferptr bp = buffer::create(reply.authorizer_len);
-      if (tcp_read(bp.c_str(), reply.authorizer_len) < 0) {
-        ldout(msgr->cct,10) << "connect couldn't read connect authorizer_reply" << dendl;
-       goto fail;
-      }
-      authorizer_reply.push_back(bp);
-    }
-
-    if (authorizer) {
-      bufferlist::iterator iter = authorizer_reply.begin();
-      if (!authorizer->verify_reply(iter)) {
-        ldout(msgr->cct,0) << "failed verifying authorize reply" << dendl;
-       goto fail;
-      }
-    }
-
-    if (conf->ms_inject_internal_delays) {
-      ldout(msgr->cct, 10) << " sleep for " << msgr->cct->_conf->ms_inject_internal_delays << dendl;
-      utime_t t;
-      t.set_from_double(msgr->cct->_conf->ms_inject_internal_delays);
-      t.sleep();
-    }
-
-    pipe_lock.Lock();
-    if (state != STATE_CONNECTING) {
-      ldout(msgr->cct,0) << "connect got RESETSESSION but no longer connecting" << dendl;
-      goto stop_locked;
-    }
-
-    if (reply.tag == CEPH_MSGR_TAG_FEATURES) {
-      ldout(msgr->cct,0) << "connect protocol feature mismatch, my " << std::hex
-             << connect.features << " < peer " << reply.features
-             << " missing " << (reply.features & ~policy.features_supported)
-             << std::dec << dendl;
-      goto fail_locked;
-    }
-
-    if (reply.tag == CEPH_MSGR_TAG_BADPROTOVER) {
-      ldout(msgr->cct,0) << "connect protocol version mismatch, my " << connect.protocol_version
-             << " != " << reply.protocol_version << dendl;
-      goto fail_locked;
-    }
-
-    if (reply.tag == CEPH_MSGR_TAG_BADAUTHORIZER) {
-      ldout(msgr->cct,0) << "connect got BADAUTHORIZER" << dendl;
-      if (got_bad_auth)
-        goto stop_locked;
-      got_bad_auth = true;
-      pipe_lock.Unlock();
-      delete authorizer;
-      authorizer = msgr->get_authorizer(peer_type, true);  // try harder
-      continue;
-    }
-    if (reply.tag == CEPH_MSGR_TAG_RESETSESSION) {
-      ldout(msgr->cct,0) << "connect got RESETSESSION" << dendl;
-      was_session_reset();
-      cseq = 0;
-      pipe_lock.Unlock();
-      continue;
-    }
-    if (reply.tag == CEPH_MSGR_TAG_RETRY_GLOBAL) {
-      gseq = msgr->get_global_seq(reply.global_seq);
-      ldout(msgr->cct,10) << "connect got RETRY_GLOBAL " << reply.global_seq
-              << " chose new " << gseq << dendl;
-      pipe_lock.Unlock();
-      continue;
-    }
-    if (reply.tag == CEPH_MSGR_TAG_RETRY_SESSION) {
-      assert(reply.connect_seq > connect_seq);
-      ldout(msgr->cct,10) << "connect got RETRY_SESSION " << connect_seq
-              << " -> " << reply.connect_seq << dendl;
-      cseq = connect_seq = reply.connect_seq;
-      pipe_lock.Unlock();
-      continue;
-    }
-
-    if (reply.tag == CEPH_MSGR_TAG_WAIT) {
-      ldout(msgr->cct,3) << "connect got WAIT (connection race)" << dendl;
-      state = STATE_WAIT;
-      goto stop_locked;
-    }
-
-    if (reply.tag == CEPH_MSGR_TAG_READY ||
-        reply.tag == CEPH_MSGR_TAG_SEQ) {
-      uint64_t feat_missing = policy.features_required & ~(uint64_t)reply.features;
-      if (feat_missing) {
-       ldout(msgr->cct,1) << "missing required features " << std::hex << feat_missing << std::dec << dendl;
-       goto fail_locked;
-      }
-
-      if (reply.tag == CEPH_MSGR_TAG_SEQ) {
-        ldout(msgr->cct,10) << "got CEPH_MSGR_TAG_SEQ, reading acked_seq and writing in_seq" << dendl;
-        uint64_t newly_acked_seq = 0;
-        if (tcp_read((char*)&newly_acked_seq, sizeof(newly_acked_seq)) < 0) {
-          ldout(msgr->cct,2) << "connect read error on newly_acked_seq" << dendl;
-          goto fail_locked;
-        }
-       ldout(msgr->cct,2) << " got newly_acked_seq " << newly_acked_seq
-                          << " vs out_seq " << out_seq << dendl;
-       while (newly_acked_seq > out_seq) {
-         Message *m = _get_next_outgoing();
-         assert(m);
-         ldout(msgr->cct,2) << " discarding previously sent " << m->get_seq()
-                            << " " << *m << dendl;
-         assert(m->get_seq() <= newly_acked_seq);
-         m->put();
-         ++out_seq;
-       }
-        if (tcp_write((char*)&in_seq, sizeof(in_seq)) < 0) {
-          ldout(msgr->cct,2) << "connect write error on in_seq" << dendl;
-          goto fail_locked;
-        }
-      }
-
-      // hooray!
-      peer_global_seq = reply.global_seq;
-      policy.lossy = reply.flags & CEPH_MSG_CONNECT_LOSSY;
-      state = STATE_OPEN;
-      connect_seq = cseq + 1;
-      assert(connect_seq == reply.connect_seq);
-      backoff = utime_t();
-      connection_state->set_features((uint64_t)reply.features & (uint64_t)connect.features);
-      ldout(msgr->cct,10) << "connect success " << connect_seq << ", lossy = " << policy.lossy
-              << ", features " << connection_state->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(msgr->cct,
-                                    authorizer->protocol,
-                                    authorizer->session_key,
-                                    connection_state->get_features()));
-      }  else {
-        // We have no authorizer, so we shouldn't be applying security to messages in this pipe.  PLR
-       session_security.reset();
-      }
-
-      msgr->dispatch_queue.queue_connect(connection_state.get());
-      msgr->ms_deliver_handle_fast_connect(connection_state.get());
-      
-      if (!reader_running) {
-       ldout(msgr->cct,20) << "connect starting reader" << dendl;
-       start_reader();
-      }
-      maybe_start_delay_thread();
-      delete authorizer;
-      return 0;
-    }
-    
-    // protocol error
-    ldout(msgr->cct,0) << "connect got bad tag " << (int)tag << dendl;
-    goto fail_locked;
-  }
-
- fail:
-  if (conf->ms_inject_internal_delays) {
-    ldout(msgr->cct, 10) << " sleep for " << msgr->cct->_conf->ms_inject_internal_delays << dendl;
-    utime_t t;
-    t.set_from_double(msgr->cct->_conf->ms_inject_internal_delays);
-    t.sleep();
-  }
-
-  pipe_lock.Lock();
- fail_locked:
-  if (state == STATE_CONNECTING)
-    fault();
-  else
-    ldout(msgr->cct,3) << "connect fault, but state = " << get_state_name()
-                      << " != connecting, stopping" << dendl;
-
- stop_locked:
-  delete authorizer;
-  return -1;
-}
-
-void Pipe::register_pipe()
-{
-  ldout(msgr->cct,10) << "register_pipe" << dendl;
-  assert(msgr->lock.is_locked());
-  Pipe *existing = msgr->_lookup_pipe(peer_addr);
-  assert(existing == NULL);
-  msgr->rank_pipe[peer_addr] = this;
-}
-
-void Pipe::unregister_pipe()
-{
-  assert(msgr->lock.is_locked());
-  ceph::unordered_map<entity_addr_t,Pipe*>::iterator p = msgr->rank_pipe.find(peer_addr);
-  if (p != msgr->rank_pipe.end() && p->second == this) {
-    ldout(msgr->cct,10) << "unregister_pipe" << dendl;
-    msgr->rank_pipe.erase(p);
-  } else {
-    ldout(msgr->cct,10) << "unregister_pipe - not registered" << dendl;
-    msgr->accepting_pipes.erase(this);  // somewhat overkill, but safe.
-  }
-}
-
-void Pipe::join()
-{
-  ldout(msgr->cct, 20) << "join" << dendl;
-  if (writer_thread.is_started())
-    writer_thread.join();
-  if (reader_thread.is_started())
-    reader_thread.join();
-  if (delay_thread) {
-    ldout(msgr->cct, 20) << "joining delay_thread" << dendl;
-    delay_thread->stop();
-    delay_thread->join();
-  }
-}
-
-void Pipe::requeue_sent()
-{
-  if (sent.empty())
-    return;
-
-  list<Message*>& rq = out_q[CEPH_MSG_PRIO_HIGHEST];
-  while (!sent.empty()) {
-    Message *m = sent.back();
-    sent.pop_back();
-    ldout(msgr->cct,10) << "requeue_sent " << *m << " for resend seq " << out_seq
-                       << " (" << m->get_seq() << ")" << dendl;
-    rq.push_front(m);
-    out_seq--;
-  }
-}
-
-void Pipe::discard_requeued_up_to(uint64_t seq)
-{
-  ldout(msgr->cct, 10) << "discard_requeued_up_to " << seq << dendl;
-  if (out_q.count(CEPH_MSG_PRIO_HIGHEST) == 0)
-    return;
-  list<Message*>& rq = out_q[CEPH_MSG_PRIO_HIGHEST];
-  while (!rq.empty()) {
-    Message *m = rq.front();
-    if (m->get_seq() == 0 || m->get_seq() > seq)
-      break;
-    ldout(msgr->cct,10) << "discard_requeued_up_to " << *m << " for resend seq " << out_seq
-                       << " <= " << seq << ", discarding" << dendl;
-    m->put();
-    rq.pop_front();
-    out_seq++;
-  }
-  if (rq.empty())
-    out_q.erase(CEPH_MSG_PRIO_HIGHEST);
-}
-
-/*
- * Tears down the Pipe's message queues, and removes them from the DispatchQueue
- * Must hold pipe_lock prior to calling.
- */
-void Pipe::discard_out_queue()
-{
-  ldout(msgr->cct,10) << "discard_queue" << dendl;
-
-  for (list<Message*>::iterator p = sent.begin(); p != sent.end(); ++p) {
-    ldout(msgr->cct,20) << "  discard " << *p << dendl;
-    (*p)->put();
-  }
-  sent.clear();
-  for (map<int,list<Message*> >::iterator p = out_q.begin(); p != out_q.end(); ++p)
-    for (list<Message*>::iterator r = p->second.begin(); r != p->second.end(); ++r) {
-      ldout(msgr->cct,20) << "  discard " << *r << dendl;
-      (*r)->put();
-    }
-  out_q.clear();
-}
-
-void Pipe::fault(bool onread)
-{
-  const md_config_t *conf = msgr->cct->_conf;
-  assert(pipe_lock.is_locked());
-  cond.Signal();
-
-  if (onread && state == STATE_CONNECTING) {
-    ldout(msgr->cct,10) << "fault already connecting, reader shutting down" << dendl;
-    return;
-  }
-  
-  ldout(msgr->cct,2) << "fault " << cpp_strerror(errno) << dendl;
-
-  if (state == STATE_CLOSED ||
-      state == STATE_CLOSING) {
-    ldout(msgr->cct,10) << "fault already closed|closing" << dendl;
-    if (connection_state->clear_pipe(this))
-      msgr->dispatch_queue.queue_reset(connection_state.get());
-    return;
-  }
-
-  shutdown_socket();
-
-  // lossy channel?
-  if (policy.lossy && state != STATE_CONNECTING) {
-    ldout(msgr->cct,10) << "fault on lossy channel, failing" << dendl;
-
-    stop();
-
-    // crib locks, blech.  note that Pipe is now STATE_CLOSED and the
-    // rank_pipe entry is ignored by others.
-    pipe_lock.Unlock();
-
-    if (conf->ms_inject_internal_delays) {
-      ldout(msgr->cct, 10) << " sleep for " << msgr->cct->_conf->ms_inject_internal_delays << dendl;
-      utime_t t;
-      t.set_from_double(msgr->cct->_conf->ms_inject_internal_delays);
-      t.sleep();
-    }
-
-    msgr->lock.Lock();
-    pipe_lock.Lock();
-    unregister_pipe();
-    msgr->lock.Unlock();
-
-    in_q->discard_queue(conn_id);
-    if (delay_thread)
-      delay_thread->discard();
-    discard_out_queue();
-
-    // disconnect from Connection, and mark it failed.  future messages
-    // will be dropped.
-    assert(connection_state);
-    if (connection_state->clear_pipe(this))
-      msgr->dispatch_queue.queue_reset(connection_state.get());
-    return;
-  }
-
-  // queue delayed items immediately
-  if (delay_thread)
-    delay_thread->flush();
-
-  // requeue sent items
-  requeue_sent();
-
-  if (policy.standby && !is_queued()) {
-    ldout(msgr->cct,0) << "fault with nothing to send, going to standby" << dendl;
-    state = STATE_STANDBY;
-    return;
-  }
-
-  if (state != STATE_CONNECTING) {
-    if (policy.server) {
-      ldout(msgr->cct,0) << "fault, server, going to standby" << dendl;
-      state = STATE_STANDBY;
-    } else {
-      ldout(msgr->cct,0) << "fault, initiating reconnect" << dendl;
-      connect_seq++;
-      state = STATE_CONNECTING;
-    }
-    backoff = utime_t();
-  } else if (backoff == utime_t()) {
-    ldout(msgr->cct,0) << "fault" << dendl;
-    backoff.set_from_double(conf->ms_initial_backoff);
-  } else {
-    ldout(msgr->cct,10) << "fault waiting " << backoff << dendl;
-    cond.WaitInterval(msgr->cct, pipe_lock, backoff);
-    backoff += backoff;
-    if (backoff > conf->ms_max_backoff)
-      backoff.set_from_double(conf->ms_max_backoff);
-    ldout(msgr->cct,10) << "fault done waiting or woke up" << dendl;
-  }
-}
-
-int Pipe::randomize_out_seq()
-{
-  if (connection_state->get_features() & CEPH_FEATURE_MSG_AUTH) {
-    // Set out_seq to a random value, so CRC won't be predictable.   Don't bother checking seq_error
-    // here.  We'll check it on the call.  PLR
-    int seq_error = get_random_bytes((char *)&out_seq, sizeof(out_seq));
-    out_seq &= SEQ_MASK;
-    lsubdout(msgr->cct, ms, 10) << "randomize_out_seq " << out_seq << dendl;
-    return seq_error;
-  } else {
-    // previously, seq #'s always started at 0.
-    out_seq = 0;
-    return 0;
-  }
-}
-
-void Pipe::was_session_reset()
-{
-  assert(pipe_lock.is_locked());
-
-  ldout(msgr->cct,10) << "was_session_reset" << dendl;
-  in_q->discard_queue(conn_id);
-  if (delay_thread)
-    delay_thread->discard();
-  discard_out_queue();
-
-  msgr->dispatch_queue.queue_remote_reset(connection_state.get());
-
-  if (randomize_out_seq()) {
-    lsubdout(msgr->cct,ms,15) << "was_session_reset(): Could not get random bytes to set seq number for session reset; set seq number to " << out_seq << dendl;
-  }
-
-  in_seq = 0;
-  connect_seq = 0;
-}
-
-void Pipe::stop()
-{
-  ldout(msgr->cct,10) << "stop" << dendl;
-  assert(pipe_lock.is_locked());
-  state = STATE_CLOSED;
-  state_closed.set(1);
-  cond.Signal();
-  shutdown_socket();
-}
-
-void Pipe::stop_and_wait()
-{
-  if (state != STATE_CLOSED)
-    stop();
-  
-  // HACK: we work around an annoying deadlock here.  If the fast
-  // dispatch method calls mark_down() on itself, it can block here
-  // waiting for the reader_dispatching flag to clear... which will
-  // clearly never happen.  Avoid the situation by skipping the wait
-  // if we are marking our *own* connect down. Do the same for the
-  // delayed dispatch thread.
-  if (delay_thread) {
-    delay_thread->stop_fast_dispatching();
-  }
-  while (reader_running &&
-        reader_dispatching &&
-        !reader_thread.am_self())
-    cond.Wait(pipe_lock);
-}
-
-/* read msgs from socket.
- * also, server.
- */
-void Pipe::reader()
-{
-  pipe_lock.Lock();
-
-  if (state == STATE_ACCEPTING) {
-    accept();
-    assert(pipe_lock.is_locked());
-  }
-
-  // loop.
-  while (state != STATE_CLOSED &&
-        state != STATE_CONNECTING) {
-    assert(pipe_lock.is_locked());
-
-    // sleep if (re)connecting
-    if (state == STATE_STANDBY) {
-      ldout(msgr->cct,20) << "reader sleeping during reconnect|standby" << dendl;
-      cond.Wait(pipe_lock);
-      continue;
-    }
-
-    // get a reference to the AuthSessionHandler while we have the pipe_lock
-    ceph::shared_ptr<AuthSessionHandler> auth_handler = session_security;
-
-    pipe_lock.Unlock();
-
-    char tag = -1;
-    ldout(msgr->cct,20) << "reader reading tag..." << dendl;
-    if (tcp_read((char*)&tag, 1) < 0) {
-      pipe_lock.Lock();
-      ldout(msgr->cct,2) << "reader couldn't read tag, " << cpp_strerror(errno) << dendl;
-      fault(true);
-      continue;
-    }
-
-    if (tag == CEPH_MSGR_TAG_KEEPALIVE) {
-      ldout(msgr->cct,20) << "reader got KEEPALIVE" << dendl;
-      pipe_lock.Lock();
-      continue;
-    }
-    if (tag == CEPH_MSGR_TAG_KEEPALIVE2) {
-      ldout(msgr->cct,30) << "reader got KEEPALIVE2 tag ..." << dendl;
-      ceph_timespec t;
-      int rc = tcp_read((char*)&t, sizeof(t));
-      pipe_lock.Lock();
-      if (rc < 0) {
-       ldout(msgr->cct,2) << "reader couldn't read KEEPALIVE2 stamp "
-                          << cpp_strerror(errno) << dendl;
-       fault(true);
-      } else {
-       send_keepalive_ack = true;
-       keepalive_ack_stamp = utime_t(t);
-       ldout(msgr->cct,20) << "reader got KEEPALIVE2 " << keepalive_ack_stamp
-                           << dendl;
-       cond.Signal();
-      }
-      continue;
-    }
-    if (tag == CEPH_MSGR_TAG_KEEPALIVE2_ACK) {
-      ldout(msgr->cct,20) << "reader got KEEPALIVE_ACK" << dendl;
-      struct ceph_timespec t;
-      int rc = tcp_read((char*)&t, sizeof(t));
-      pipe_lock.Lock();
-      if (rc < 0) {
-       ldout(msgr->cct,2) << "reader couldn't read KEEPALIVE2 stamp " << cpp_strerror(errno) << dendl;
-       fault(true);
-      } else {
-       connection_state->last_keepalive_ack = utime_t(t);
-      }
-      continue;
-    }
-
-    // open ...
-    if (tag == CEPH_MSGR_TAG_ACK) {
-      ldout(msgr->cct,20) << "reader got ACK" << dendl;
-      ceph_le64 seq;
-      int rc = tcp_read((char*)&seq, sizeof(seq));
-      pipe_lock.Lock();
-      if (rc < 0) {
-       ldout(msgr->cct,2) << "reader couldn't read ack seq, " << cpp_strerror(errno) << dendl;
-       fault(true);
-      } else if (state != STATE_CLOSED) {
-        handle_ack(seq);
-      }
-      continue;
-    }
-
-    else if (tag == CEPH_MSGR_TAG_MSG) {
-      ldout(msgr->cct,20) << "reader got MSG" << dendl;
-      Message *m = 0;
-      int r = read_message(&m, auth_handler.get());
-
-      pipe_lock.Lock();
-      
-      if (!m) {
-       if (r < 0)
-         fault(true);
-       continue;
-      }
-
-      if (state == STATE_CLOSED ||
-         state == STATE_CONNECTING) {
-       msgr->dispatch_throttle_release(m->get_dispatch_throttle_size());
-       m->put();
-       continue;
-      }
-
-      // 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.
-      if (m->get_seq() <= in_seq) {
-       ldout(msgr->cct,0) << "reader got old message "
-               << m->get_seq() << " <= " << in_seq << " " << m << " " << *m
-               << ", discarding" << dendl;
-       msgr->dispatch_throttle_release(m->get_dispatch_throttle_size());
-       m->put();
-       if (connection_state->has_feature(CEPH_FEATURE_RECONNECT_SEQ) &&
-           msgr->cct->_conf->ms_die_on_old_message)
-         assert(0 == "old msgs despite reconnect_seq feature");
-       continue;
-      }
-
-      m->set_connection(connection_state.get());
-
-      // note last received message.
-      in_seq = m->get_seq();
-
-      cond.Signal();  // wake up writer, to ack this
-      
-      ldout(msgr->cct,10) << "reader got message "
-              << m->get_seq() << " " << m << " " << *m
-              << dendl;
-      in_q->fast_preprocess(m);
-
-      if (delay_thread) {
-        utime_t release;
-        if (rand() % 10000 < msgr->cct->_conf->ms_inject_delay_probability * 10000.0) {
-          release = m->get_recv_stamp();
-          release += msgr->cct->_conf->ms_inject_delay_max * (double)(rand() % 10000) / 10000.0;
-          lsubdout(msgr->cct, ms, 1) << "queue_received will delay until " << release << " on " << m << " " << *m << dendl;
-        }
-        delay_thread->queue(release, m);
-      } else {
-        if (in_q->can_fast_dispatch(m)) {
-         reader_dispatching = true;
-          pipe_lock.Unlock();
-          in_q->fast_dispatch(m);
-          pipe_lock.Lock();
-         reader_dispatching = false;
-         if (state == STATE_CLOSED) // there might be somebody waiting
-           cond.Signal();
-        } else {
-          in_q->enqueue(m, m->get_priority(), conn_id);
-        }
-      }
-    }
-    
-    else if (tag == CEPH_MSGR_TAG_CLOSE) {
-      ldout(msgr->cct,20) << "reader got CLOSE" << dendl;
-      pipe_lock.Lock();
-      if (state == STATE_CLOSING) {
-       state = STATE_CLOSED;
-       state_closed.set(1);
-      } else {
-       state = STATE_CLOSING;
-      }
-      cond.Signal();
-      break;
-    }
-    else {
-      ldout(msgr->cct,0) << "reader bad tag " << (int)tag << dendl;
-      pipe_lock.Lock();
-      fault(true);
-    }
-  }
-
-  // reap?
-  reader_running = false;
-  reader_needs_join = true;
-  unlock_maybe_reap();
-  ldout(msgr->cct,10) << "reader done" << dendl;
-}
-
-/* write msgs to socket.
- * also, client.
- */
-void Pipe::writer()
-{
-  pipe_lock.Lock();
-  while (state != STATE_CLOSED) {// && state != STATE_WAIT) {
-    ldout(msgr->cct,10) << "writer: state = " << get_state_name()
-                       << " policy.server=" << policy.server << dendl;
-
-    // standby?
-    if (is_queued() && state == STATE_STANDBY && !policy.server) {
-      connect_seq++;
-      state = STATE_CONNECTING;
-    }
-
-    // connect?
-    if (state == STATE_CONNECTING) {
-      assert(!policy.server);
-      connect();
-      continue;
-    }
-    
-    if (state == STATE_CLOSING) {
-      // write close tag
-      ldout(msgr->cct,20) << "writer writing CLOSE tag" << dendl;
-      char tag = CEPH_MSGR_TAG_CLOSE;
-      state = STATE_CLOSED;
-      state_closed.set(1);
-      pipe_lock.Unlock();
-      if (sd) {
-       int r = ::write(sd, &tag, 1);
-       // we can ignore r, actually; we don't care if this succeeds.
-       r++; r = 0; // placate gcc
-      }
-      pipe_lock.Lock();
-      continue;
-    }
-
-    if (state != STATE_CONNECTING && state != STATE_WAIT && state != STATE_STANDBY &&
-       (is_queued() || in_seq > in_seq_acked)) {
-
-      // keepalive?
-      if (send_keepalive) {
-       int rc;
-       if (connection_state->has_feature(CEPH_FEATURE_MSGR_KEEPALIVE2)) {
-         pipe_lock.Unlock();
-         rc = write_keepalive2(CEPH_MSGR_TAG_KEEPALIVE2,
-                               ceph_clock_now(msgr->cct));
-       } else {
-         pipe_lock.Unlock();
-         rc = write_keepalive();
-       }
-       pipe_lock.Lock();
-       if (rc < 0) {
-         ldout(msgr->cct,2) << "writer couldn't write keepalive[2], "
-                            << cpp_strerror(errno) << dendl;
-         fault();
-         continue;
-       }
-       send_keepalive = false;
-      }
-      if (send_keepalive_ack) {
-       utime_t t = keepalive_ack_stamp;
-       pipe_lock.Unlock();
-       int rc = write_keepalive2(CEPH_MSGR_TAG_KEEPALIVE2_ACK, t);
-       pipe_lock.Lock();
-       if (rc < 0) {
-         ldout(msgr->cct,2) << "writer couldn't write keepalive_ack, " << cpp_strerror(errno) << dendl;
-         fault();
-         continue;
-       }
-       send_keepalive_ack = false;
-      }
-
-      // send ack?
-      if (in_seq > in_seq_acked) {
-       uint64_t send_seq = in_seq;
-       pipe_lock.Unlock();
-       int rc = write_ack(send_seq);
-       pipe_lock.Lock();
-       if (rc < 0) {
-         ldout(msgr->cct,2) << "writer couldn't write ack, " << cpp_strerror(errno) << dendl;
-         fault();
-         continue;
-       }
-       in_seq_acked = send_seq;
-      }
-
-      // grab outgoing message
-      Message *m = _get_next_outgoing();
-      if (m) {
-       m->set_seq(++out_seq);
-       if (!policy.lossy) {
-         // put on sent list
-         sent.push_back(m); 
-         m->get();
-       }
-
-       // associate message with Connection (for benefit of encode_payload)
-       m->set_connection(connection_state.get());
-
-       uint64_t features = connection_state->get_features();
-
-       if (m->empty_payload())
-         ldout(msgr->cct,20) << "writer encoding " << m->get_seq() << " features " << features
-                             << " " << m << " " << *m << dendl;
-       else
-         ldout(msgr->cct,20) << "writer half-reencoding " << m->get_seq() << " features " << features
-                             << " " << m << " " << *m << dendl;
-
-       // encode and copy out of *m
-       m->encode(features, !msgr->cct->_conf->ms_nocrc);
-
-       // prepare everything
-       ceph_msg_header& header = m->get_header();
-       ceph_msg_footer& footer = m->get_footer();
-
-       // Now that we have all the crcs calculated, handle the
-       // digital signature for the message, if the pipe 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(msgr->cct, 20) << "writer no session security" << dendl;
-       } else {
-         if (session_security->sign_message(m)) {
-           ldout(msgr->cct, 20) << "writer failed to sign seq # " << header.seq
-                                << "): sig = " << footer.sig << dendl;
-         } else {
-           ldout(msgr->cct, 20) << "writer signed seq # " << header.seq
-                                << "): sig = " << footer.sig << dendl;
-         }
-       }
-
-       bufferlist blist = m->get_payload();
-       blist.append(m->get_middle());
-       blist.append(m->get_data());
-
-        pipe_lock.Unlock();
-
-        ldout(msgr->cct,20) << "writer sending " << m->get_seq() << " " << m << dendl;
-       int rc = write_message(header, footer, blist);
-
-       pipe_lock.Lock();
-       if (rc < 0) {
-          ldout(msgr->cct,1) << "writer error sending " << m << ", "
-                 << cpp_strerror(errno) << dendl;
-         fault();
-        }
-       m->put();
-      }
-      continue;
-    }
-    
-    // wait
-    ldout(msgr->cct,20) << "writer sleeping" << dendl;
-    cond.Wait(pipe_lock);
-  }
-  
-  ldout(msgr->cct,20) << "writer finishing" << dendl;
-
-  // reap?
-  writer_running = false;
-  unlock_maybe_reap();
-  ldout(msgr->cct,10) << "writer done" << dendl;
-}
-
-void Pipe::unlock_maybe_reap()
-{
-  if (!reader_running && !writer_running) {
-    shutdown_socket();
-    pipe_lock.Unlock();
-    if (delay_thread && delay_thread->is_flushing()) {
-      delay_thread->wait_for_flush();
-    }
-    msgr->queue_reap(this);
-  } else {
-    pipe_lock.Unlock();
-  }
-}
-
-static void alloc_aligned_buffer(bufferlist& data, unsigned len, unsigned off)
-{
-  // create a buffer to read into that matches the data alignment
-  unsigned left = len;
-  if (off & ~CEPH_PAGE_MASK) {
-    // head
-    unsigned head = 0;
-    head = MIN(CEPH_PAGE_SIZE - (off & ~CEPH_PAGE_MASK), left);
-    bufferptr bp = buffer::create(head);
-    data.push_back(bp);
-    left -= head;
-  }
-  unsigned middle = left & CEPH_PAGE_MASK;
-  if (middle > 0) {
-    bufferptr bp = buffer::create_page_aligned(middle);
-    data.push_back(bp);
-    left -= middle;
-  }
-  if (left) {
-    bufferptr bp = buffer::create(left);
-    data.push_back(bp);
-  }
-}
-
-int Pipe::read_message(Message **pm, AuthSessionHandler* auth_handler)
-{
-  int ret = -1;
-  // envelope
-  //ldout(msgr->cct,10) << "receiver.read_message from sd " << sd  << dendl;
-  
-  ceph_msg_header header; 
-  ceph_msg_footer footer;
-  __u32 header_crc;
-  
-  if (connection_state->has_feature(CEPH_FEATURE_NOSRCADDR)) {
-    if (tcp_read((char*)&header, sizeof(header)) < 0)
-      return -1;
-    header_crc = ceph_crc32c(0, (unsigned char *)&header, sizeof(header) - sizeof(header.crc));
-  } else {
-    ceph_msg_header_old oldheader;
-    if (tcp_read((char*)&oldheader, sizeof(oldheader)) < 0)
-      return -1;
-    // this is fugly
-    memcpy(&header, &oldheader, sizeof(header));
-    header.src = oldheader.src.name;
-    header.reserved = oldheader.reserved;
-    header.crc = oldheader.crc;
-    header_crc = ceph_crc32c(0, (unsigned char *)&oldheader, sizeof(oldheader) - sizeof(oldheader.crc));
-  }
-
-  ldout(msgr->cct,20) << "reader got envelope type=" << header.type
-           << " src " << entity_name_t(header.src)
-           << " front=" << header.front_len
-          << " data=" << header.data_len
-          << " off " << header.data_off
-           << dendl;
-
-  // verify header crc
-  if (header_crc != header.crc) {
-    ldout(msgr->cct,0) << "reader got bad header crc " << header_crc << " != " << header.crc << dendl;
-    return -1;
-  }
-
-  bufferlist front, middle, data;
-  int front_len, middle_len;
-  unsigned data_len, data_off;
-  int aborted;
-  Message *message;
-  utime_t recv_stamp = ceph_clock_now(msgr->cct);
-
-  if (policy.throttler_messages) {
-    ldout(msgr->cct,10) << "reader wants " << 1 << " message from policy throttler "
-                       << policy.throttler_messages->get_current() << "/"
-                       << policy.throttler_messages->get_max() << dendl;
-    policy.throttler_messages->get();
-  }
-
-  uint64_t message_size = header.front_len + header.middle_len + header.data_len;
-  if (message_size) {
-    if (policy.throttler_bytes) {
-      ldout(msgr->cct,10) << "reader wants " << message_size << " bytes from policy throttler "
-              << policy.throttler_bytes->get_current() << "/"
-              << policy.throttler_bytes->get_max() << dendl;
-      policy.throttler_bytes->get(message_size);
-    }
-
-    // throttle total bytes waiting for dispatch.  do this _after_ the
-    // policy throttle, as this one does not deadlock (unless dispatch
-    // blocks indefinitely, which it shouldn't).  in contrast, the
-    // policy throttle carries for the lifetime of the message.
-    ldout(msgr->cct,10) << "reader wants " << message_size << " from dispatch throttler "
-            << msgr->dispatch_throttler.get_current() << "/"
-            << msgr->dispatch_throttler.get_max() << dendl;
-    msgr->dispatch_throttler.get(message_size);
-  }
-
-  utime_t throttle_stamp = ceph_clock_now(msgr->cct);
-
-  // read front
-  front_len = header.front_len;
-  if (front_len) {
-    bufferptr bp = buffer::create(front_len);
-    if (tcp_read(bp.c_str(), front_len) < 0)
-      goto out_dethrottle;
-    front.push_back(bp);
-    ldout(msgr->cct,20) << "reader got front " << front.length() << dendl;
-  }
-
-  // read middle
-  middle_len = header.middle_len;
-  if (middle_len) {
-    bufferptr bp = buffer::create(middle_len);
-    if (tcp_read(bp.c_str(), middle_len) < 0)
-      goto out_dethrottle;
-    middle.push_back(bp);
-    ldout(msgr->cct,20) << "reader got middle " << middle.length() << dendl;
-  }
-
-
-  // read data
-  data_len = le32_to_cpu(header.data_len);
-  data_off = le32_to_cpu(header.data_off);
-  if (data_len) {
-    unsigned offset = 0;
-    unsigned left = data_len;
-
-    bufferlist newbuf, rxbuf;
-    bufferlist::iterator blp;
-    int rxbuf_version = 0;
-       
-    while (left > 0) {
-      // wait for data
-      if (tcp_read_wait() < 0)
-       goto out_dethrottle;
-
-      // get a buffer
-      connection_state->lock.Lock();
-      map<ceph_tid_t,pair<bufferlist,int> >::iterator p = connection_state->rx_buffers.find(header.tid);
-      if (p != connection_state->rx_buffers.end()) {
-       if (rxbuf.length() == 0 || p->second.second != rxbuf_version) {
-         ldout(msgr->cct,10) << "reader seleting rx buffer v " << p->second.second
-                  << " at offset " << offset
-                  << " len " << p->second.first.length() << dendl;
-         rxbuf = p->second.first;
-         rxbuf_version = p->second.second;
-         // make sure it's big enough
-         if (rxbuf.length() < data_len)
-           rxbuf.push_back(buffer::create(data_len - rxbuf.length()));
-         blp = p->second.first.begin();
-         blp.advance(offset);
-       }
-      } else {
-       if (!newbuf.length()) {
-         ldout(msgr->cct,20) << "reader allocating new rx buffer at offset " << offset << dendl;
-         alloc_aligned_buffer(newbuf, data_len, data_off);
-         blp = newbuf.begin();
-         blp.advance(offset);
-       }
-      }
-      bufferptr bp = blp.get_current_ptr();
-      int read = MIN(bp.length(), left);
-      ldout(msgr->cct,20) << "reader reading nonblocking into " << (void*)bp.c_str() << " len " << bp.length() << dendl;
-      int got = tcp_read_nonblocking(bp.c_str(), read);
-      ldout(msgr->cct,30) << "reader read " << got << " of " << read << dendl;
-      connection_state->lock.Unlock();
-      if (got < 0)
-       goto out_dethrottle;
-      if (got > 0) {
-       blp.advance(got);
-       data.append(bp, 0, got);
-       offset += got;
-       left -= got;
-      } // else we got a signal or something; just loop.
-    }
-  }
-
-  // footer
-  if (connection_state->has_feature(CEPH_FEATURE_MSG_AUTH)) {
-    if (tcp_read((char*)&footer, sizeof(footer)) < 0)
-      goto out_dethrottle;
-  } else {
-    ceph_msg_footer_old old_footer;
-    if (tcp_read((char*)&old_footer, sizeof(old_footer)) < 0)
-      goto out_dethrottle;
-    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;
-  }
-  
-  aborted = (footer.flags & CEPH_MSG_FOOTER_COMPLETE) == 0;
-  ldout(msgr->cct,10) << "aborted = " << aborted << dendl;
-  if (aborted) {
-    ldout(msgr->cct,0) << "reader got " << front.length() << " + " << middle.length() << " + " << data.length()
-           << " byte message.. ABORTED" << dendl;
-    ret = 0;
-    goto out_dethrottle;
-  }
-
-  ldout(msgr->cct,20) << "reader got " << front.length() << " + " << middle.length() << " + " << data.length()
-          << " byte message" << dendl;
-  message = decode_message(msgr->cct, header, footer, front, middle, data);
-  if (!message) {
-    ret = -EINVAL;
-    goto out_dethrottle;
-  }
-
-  //
-  //  Check the signature if one should be present.  A zero return indicates success. PLR
-  //
-
-  if (auth_handler == NULL) {
-    ldout(msgr->cct, 10) << "No session security set" << dendl;
-  } else {
-    if (auth_handler->check_message_signature(message)) {
-      ldout(msgr->cct, 0) << "Signature check failed" << dendl;
-      ret = -EINVAL;
-      goto out_dethrottle;
-    } 
-  }
-
-  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(message_size);
-
-  message->set_recv_stamp(recv_stamp);
-  message->set_throttle_stamp(throttle_stamp);
-  message->set_recv_complete_stamp(ceph_clock_now(msgr->cct));
-
-  *pm = message;
-  return 0;
-
- out_dethrottle:
-  // release bytes reserved from the throttlers on failure
-  if (policy.throttler_messages) {
-    ldout(msgr->cct,10) << "reader releasing " << 1 << " message to policy throttler "
-                       << policy.throttler_messages->get_current() << "/"
-                       << policy.throttler_messages->get_max() << dendl;
-    policy.throttler_messages->put();
-  }
-  if (message_size) {
-    if (policy.throttler_bytes) {
-      ldout(msgr->cct,10) << "reader releasing " << message_size << " bytes to policy throttler "
-                         << policy.throttler_bytes->get_current() << "/"
-                         << policy.throttler_bytes->get_max() << dendl;
-      policy.throttler_bytes->put(message_size);
-    }
-
-    msgr->dispatch_throttle_release(message_size);
-  }
-  return ret;
-}
-
-int Pipe::do_sendmsg(struct msghdr *msg, int len, bool more)
-{
-  while (len > 0) {
-    if (0) { // sanity
-      int l = 0;
-      for (unsigned i=0; i<msg->msg_iovlen; i++)
-       l += msg->msg_iov[i].iov_len;
-      assert(l == len);
-    }
-
-    int r = ::sendmsg(sd, msg, MSG_NOSIGNAL | (more ? MSG_MORE : 0));
-    if (r == 0) 
-      ldout(msgr->cct,10) << "do_sendmsg hmm do_sendmsg got r==0!" << dendl;
-    if (r < 0) { 
-      ldout(msgr->cct,1) << "do_sendmsg error " << cpp_strerror(errno) << dendl;
-      return -1;
-    }
-    if (state == STATE_CLOSED) {
-      ldout(msgr->cct,10) << "do_sendmsg oh look, state == CLOSED, giving up" << dendl;
-      errno = EINTR;
-      return -1; // close enough
-    }
-
-    len -= r;
-    if (len == 0) break;
-    
-    // hrmph.  trim r bytes off the front of our message.
-    ldout(msgr->cct,20) << "do_sendmsg short write did " << r << ", still have " << len << dendl;
-    while (r > 0) {
-      if (msg->msg_iov[0].iov_len <= (size_t)r) {
-       // lose this whole item
-       //ldout(msgr->cct,30) << "skipping " << msg->msg_iov[0].iov_len << ", " << (msg->msg_iovlen-1) << " v, " << r << " left" << dendl;
-       r -= msg->msg_iov[0].iov_len;
-       msg->msg_iov++;
-       msg->msg_iovlen--;
-      } else {
-       // partial!
-       //ldout(msgr->cct,30) << "adjusting " << msg->msg_iov[0].iov_len << ", " << msg->msg_iovlen << " v, " << r << " left" << dendl;
-       msg->msg_iov[0].iov_base = (char *)msg->msg_iov[0].iov_base + r;
-       msg->msg_iov[0].iov_len -= r;
-       break;
-      }
-    }
-  }
-  return 0;
-}
-
-
-int Pipe::write_ack(uint64_t seq)
-{
-  ldout(msgr->cct,10) << "write_ack " << seq << dendl;
-
-  char c = CEPH_MSGR_TAG_ACK;
-  ceph_le64 s;
-  s = seq;
-
-  struct msghdr msg;
-  memset(&msg, 0, sizeof(msg));
-  struct iovec msgvec[2];
-  msgvec[0].iov_base = &c;
-  msgvec[0].iov_len = 1;
-  msgvec[1].iov_base = &s;
-  msgvec[1].iov_len = sizeof(s);
-  msg.msg_iov = msgvec;
-  msg.msg_iovlen = 2;
-  
-  if (do_sendmsg(&msg, 1 + sizeof(s), true) < 0)
-    return -1; 
-  return 0;
-}
-
-int Pipe::write_keepalive()
-{
-  ldout(msgr->cct,10) << "write_keepalive" << dendl;
-
-  char c = CEPH_MSGR_TAG_KEEPALIVE;
-
-  struct msghdr msg;
-  memset(&msg, 0, sizeof(msg));
-  struct iovec msgvec[2];
-  msgvec[0].iov_base = &c;
-  msgvec[0].iov_len = 1;
-  msg.msg_iov = msgvec;
-  msg.msg_iovlen = 1;
-  
-  if (do_sendmsg(&msg, 1) < 0)
-    return -1; 
-  return 0;
-}
-
-int Pipe::write_keepalive2(char tag, const utime_t& t)
-{
-  ldout(msgr->cct,10) << "write_keepalive2 " << (int)tag << " " << t << dendl;
-  struct ceph_timespec ts;
-  t.encode_timeval(&ts);
-  struct msghdr msg;
-  memset(&msg, 0, sizeof(msg));
-  struct iovec msgvec[2];
-  msgvec[0].iov_base = &tag;
-  msgvec[0].iov_len = 1;
-  msgvec[1].iov_base = &ts;
-  msgvec[1].iov_len = sizeof(ts);
-  msg.msg_iov = msgvec;
-  msg.msg_iovlen = 2;
-
-  if (do_sendmsg(&msg, 1 + sizeof(ts)) < 0)
-    return -1;
-  return 0;
-}
-
-
-int Pipe::write_message(ceph_msg_header& header, ceph_msg_footer& footer, bufferlist& blist)
-{
-  int ret;
-
-  // set up msghdr and iovecs
-  struct msghdr msg;
-  memset(&msg, 0, sizeof(msg));
-  msg.msg_iov = msgvec;
-  int msglen = 0;
-  
-  // send tag
-  char tag = CEPH_MSGR_TAG_MSG;
-  msgvec[msg.msg_iovlen].iov_base = &tag;
-  msgvec[msg.msg_iovlen].iov_len = 1;
-  msglen++;
-  msg.msg_iovlen++;
-
-  // send envelope
-  ceph_msg_header_old oldheader;
-  if (connection_state->has_feature(CEPH_FEATURE_NOSRCADDR)) {
-    msgvec[msg.msg_iovlen].iov_base = (char*)&header;
-    msgvec[msg.msg_iovlen].iov_len = sizeof(header);
-    msglen += sizeof(header);
-    msg.msg_iovlen++;
-  } else {
-    memcpy(&oldheader, &header, sizeof(header));
-    oldheader.src.name = header.src;
-    oldheader.src.addr = connection_state->get_peer_addr();
-    oldheader.orig_src = oldheader.src;
-    oldheader.reserved = header.reserved;
-    oldheader.crc = ceph_crc32c(0, (unsigned char*)&oldheader,
-                               sizeof(oldheader) - sizeof(oldheader.crc));
-    msgvec[msg.msg_iovlen].iov_base = (char*)&oldheader;
-    msgvec[msg.msg_iovlen].iov_len = sizeof(oldheader);
-    msglen += sizeof(oldheader);
-    msg.msg_iovlen++;
-  }
-
-  // payload (front+data)
-  list<bufferptr>::const_iterator pb = blist.buffers().begin();
-  int b_off = 0;  // carry-over buffer offset, if any
-  int bl_pos = 0; // blist pos
-  int left = blist.length();
-
-  while (left > 0) {
-    int donow = MIN(left, (int)pb->length()-b_off);
-    if (donow == 0) {
-      ldout(msgr->cct,0) << "donow = " << donow << " left " << left << " pb->length " << pb->length()
-             << " b_off " << b_off << dendl;
-    }
-    assert(donow > 0);
-    ldout(msgr->cct,30) << " bl_pos " << bl_pos << " b_off " << b_off
-            << " leftinchunk " << left
-            << " buffer len " << pb->length()
-            << " writing " << donow 
-            << dendl;
-    
-    if (msg.msg_iovlen >= IOV_MAX-2) {
-      if (do_sendmsg(&msg, msglen, true))
-       goto fail;
-      
-      // and restart the iov
-      msg.msg_iov = msgvec;
-      msg.msg_iovlen = 0;
-      msglen = 0;
-    }
-    
-    msgvec[msg.msg_iovlen].iov_base = (void*)(pb->c_str()+b_off);
-    msgvec[msg.msg_iovlen].iov_len = donow;
-    msglen += donow;
-    msg.msg_iovlen++;
-    
-    left -= donow;
-    assert(left >= 0);
-    b_off += donow;
-    bl_pos += donow;
-    if (left == 0)
-      break;
-    while (b_off == (int)pb->length()) {
-      ++pb;
-      b_off = 0;
-    }
-  }
-  assert(left == 0);
-
-  // send footer; if receiver doesn't support signatures, use the old footer format
-
-  ceph_msg_footer_old old_footer;
-  if (connection_state->has_feature(CEPH_FEATURE_MSG_AUTH)) {
-    msgvec[msg.msg_iovlen].iov_base = (void*)&footer;
-    msgvec[msg.msg_iovlen].iov_len = sizeof(footer);
-    msglen += sizeof(footer);
-    msg.msg_iovlen++;
-  } else {
-    old_footer.front_crc = footer.front_crc;   
-    old_footer.middle_crc = footer.middle_crc;   
-    old_footer.data_crc = footer.data_crc;   
-    old_footer.flags = footer.flags;   
-    msgvec[msg.msg_iovlen].iov_base = (char*)&old_footer;
-    msgvec[msg.msg_iovlen].iov_len = sizeof(old_footer);
-    msglen += sizeof(old_footer);
-    msg.msg_iovlen++;
-  }
-
-  // send
-  if (do_sendmsg(&msg, msglen))
-    goto fail;
-
-  ret = 0;
-
- out:
-  return ret;
-
- fail:
-  ret = -1;
-  goto out;
-}
-
-
-int Pipe::tcp_read(char *buf, int len)
-{
-  if (sd < 0)
-    return -1;
-
-  while (len > 0) {
-
-    if (msgr->cct->_conf->ms_inject_socket_failures && sd >= 0) {
-      if (rand() % msgr->cct->_conf->ms_inject_socket_failures == 0) {
-       ldout(msgr->cct, 0) << "injecting socket failure" << dendl;
-       ::shutdown(sd, SHUT_RDWR);
-      }
-    }
-
-    if (tcp_read_wait() < 0)
-      return -1;
-
-    int got = tcp_read_nonblocking(buf, len);
-
-    if (got < 0)
-      return -1;
-
-    len -= got;
-    buf += got;
-    //lgeneric_dout(cct, DBL) << "tcp_read got " << got << ", " << len << " left" << dendl;
-  }
-  return len;
-}
-
-int Pipe::tcp_read_wait()
-{
-  if (sd < 0)
-    return -1;
-  struct pollfd pfd;
-  short evmask;
-  pfd.fd = sd;
-  pfd.events = POLLIN;
-#if defined(__linux__)
-  pfd.events |= POLLRDHUP;
-#endif
-
-  if (has_pending_data())
-    return 0;
-
-  if (poll(&pfd, 1, msgr->timeout) <= 0)
-    return -1;
-
-  evmask = POLLERR | POLLHUP | POLLNVAL;
-#if defined(__linux__)
-  evmask |= POLLRDHUP;
-#endif
-  if (pfd.revents & evmask)
-    return -1;
-
-  if (!(pfd.revents & POLLIN))
-    return -1;
-
-  return 0;
-}
-
-int Pipe::do_recv(char *buf, size_t len, int flags)
-{
-again:
-  int got = ::recv( sd, buf, len, flags );
-  if (got < 0) {
-    if (errno == EAGAIN || errno == EINTR) {
-      goto again;
-    }
-    ldout(msgr->cct, 10) << __func__ << " socket " << sd << " returned "
-                    << got << " " << cpp_strerror(errno) << dendl;
-    return -1;
-  }
-  if (got == 0) {
-    return -1;
-  }
-  return got;
-}
-
-int Pipe::buffered_recv(char *buf, size_t len, int flags)
-{
-  int left = len;
-  int total_recv = 0;
-  if (recv_len > recv_ofs) {
-    int to_read = MIN(recv_len - recv_ofs, left);
-    memcpy(buf, &recv_buf[recv_ofs], to_read);
-    recv_ofs += to_read;
-    left -= to_read;
-    if (left == 0) {
-      return to_read;
-    }
-    buf += to_read;
-    total_recv += to_read;
-  }
-
-  /* nothing left in the prefetch buffer */
-
-  if (len > (size_t)recv_max_prefetch) {
-    /* this was a large read, we don't prefetch for these */
-    int ret = do_recv(buf, left, flags );
-    if (ret < 0) {
-      if (total_recv > 0)
-        return total_recv;
-      return ret;
-    }
-    total_recv += ret;
-    return total_recv;
-  }
-
-
-  int got = do_recv(recv_buf, recv_max_prefetch, flags);
-  if (got <= 0) {
-    if (total_recv > 0)
-      return total_recv;
-
-    return got;
-  }
-
-  recv_len = got;
-  got = MIN(left, got);
-  memcpy(buf, recv_buf, got);
-  recv_ofs = got;
-  total_recv += got;
-  return total_recv;
-}
-
-int Pipe::tcp_read_nonblocking(char *buf, int len)
-{
-  int got = buffered_recv(buf, len, MSG_DONTWAIT );
-  if (got < 0) {
-    ldout(msgr->cct, 10) << __func__ << " socket " << sd << " returned "
-                        << got << " " << cpp_strerror(errno) << dendl;
-    return -1;
-  }
-  if (got == 0) {
-    /* poll() said there was data, but we didn't read any - peer
-     * sent a FIN.  Maybe POLLRDHUP signals this, but this is
-     * standard socket behavior as documented by Stevens.
-     */
-    return -1;
-  }
-  return got;
-}
-
-int Pipe::tcp_write(const char *buf, int len)
-{
-  if (sd < 0)
-    return -1;
-  struct pollfd pfd;
-  pfd.fd = sd;
-  pfd.events = POLLOUT | POLLHUP | POLLNVAL | POLLERR;
-#if defined(__linux__)
-  pfd.events |= POLLRDHUP;
-#endif
-
-  if (msgr->cct->_conf->ms_inject_socket_failures && sd >= 0) {
-    if (rand() % msgr->cct->_conf->ms_inject_socket_failures == 0) {
-      ldout(msgr->cct, 0) << "injecting socket failure" << dendl;
-      ::shutdown(sd, SHUT_RDWR);
-    }
-  }
-
-  if (poll(&pfd, 1, -1) < 0)
-    return -1;
-
-  if (!(pfd.revents & POLLOUT))
-    return -1;
-
-  //lgeneric_dout(cct, DBL) << "tcp_write writing " << len << dendl;
-  assert(len > 0);
-  while (len > 0) {
-    int did = ::send( sd, buf, len, MSG_NOSIGNAL );
-    if (did < 0) {
-      //lgeneric_dout(cct, 1) << "tcp_write error did = " << did << " " << cpp_strerror(errno) << dendl;
-      //lgeneric_derr(cct, 1) << "tcp_write error did = " << did << " " << cpp_strerror(errno) << dendl;
-      return did;
-    }
-    len -= did;
-    buf += did;
-    //lgeneric_dout(cct, DBL) << "tcp_write did " << did << ", " << len << " left" << dendl;
-  }
-  return 0;
-}
diff --git a/src/msg/Pipe.h b/src/msg/Pipe.h
deleted file mode 100644 (file)
index b6d7432..0000000
+++ /dev/null
@@ -1,374 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software 
- * Foundation.  See file COPYING.
- * 
- */
-
-#ifndef CEPH_MSGR_PIPE_H
-#define CEPH_MSGR_PIPE_H
-
-#include "include/memory.h"
-
-#include "msg_types.h"
-#include "Messenger.h"
-#include "auth/AuthSessionHandler.h"
-#include "PipeConnection.h"
-
-
-class SimpleMessenger;
-class IncomingQueue;
-class DispatchQueue;
-
-  /**
-   * The Pipe is the most complex SimpleMessenger component. It gets
-   * two threads, one each for reading and writing on a socket it's handed
-   * at creation time, and is responsible for everything that happens on
-   * that socket. Besides message transmission, it's responsible for
-   * propagating socket errors to the SimpleMessenger and then sticking
-   * around in a state where it can provide enough data for the SimpleMessenger
-   * to provide reliable Message delivery when it manages to reconnect.
-   */
-  class Pipe : public RefCountedObject {
-    /**
-     * The Reader thread handles all reads off the socket -- not just
-     * Messages, but also acks and other protocol bits (excepting startup,
-     * when the Writer does a couple of reads).
-     * All the work is implemented in Pipe itself, of course.
-     */
-    class Reader : public Thread {
-      Pipe *pipe;
-    public:
-      Reader(Pipe *p) : pipe(p) {}
-      void *entry() { pipe->reader(); return 0; }
-    } reader_thread;
-    friend class Reader;
-
-    /**
-     * The Writer thread handles all writes to the socket (after startup).
-     * All the work is implemented in Pipe itself, of course.
-     */
-    class Writer : public Thread {
-      Pipe *pipe;
-    public:
-      Writer(Pipe *p) : pipe(p) {}
-      void *entry() { pipe->writer(); return 0; }
-    } writer_thread;
-    friend class Writer;
-
-    /**
-     * The DelayedDelivery is for injecting delays into Message delivery off
-     * the socket. It is only enabled if delays are requested, and if they
-     * are then it pulls Messages off the DelayQueue and puts them into the
-     * in_q (SimpleMessenger::dispatch_queue).
-     * Please note that this probably has issues with Pipe shutdown and
-     * replacement semantics. I've tried, but no guarantees.
-     */
-    class DelayedDelivery: public Thread {
-      Pipe *pipe;
-      std::deque< pair<utime_t,Message*> > delay_queue;
-      Mutex delay_lock;
-      Cond delay_cond;
-      int flush_count;
-      bool active_flush;
-      bool stop_delayed_delivery;
-      bool delay_dispatching; // we are in fast dispatch now
-      bool stop_fast_dispatching_flag; // we need to stop fast dispatching
-
-    public:
-      DelayedDelivery(Pipe *p)
-       : pipe(p),
-         delay_lock("Pipe::DelayedDelivery::delay_lock"), flush_count(0),
-         active_flush(false),
-         stop_delayed_delivery(false),
-         delay_dispatching(false),
-         stop_fast_dispatching_flag(false) { }
-      ~DelayedDelivery() {
-       discard();
-      }
-      void *entry();
-      void queue(utime_t release, Message *m) {
-       Mutex::Locker l(delay_lock);
-       delay_queue.push_back(make_pair(release, m));
-       delay_cond.Signal();
-      }
-      void discard();
-      void flush();
-      bool is_flushing() {
-        Mutex::Locker l(delay_lock);
-        return flush_count > 0 || active_flush;
-      }
-      void wait_for_flush() {
-        Mutex::Locker l(delay_lock);
-        while (flush_count > 0 || active_flush)
-          delay_cond.Wait(delay_lock);
-      }
-      void stop() {
-       delay_lock.Lock();
-       stop_delayed_delivery = true;
-       delay_cond.Signal();
-       delay_lock.Unlock();
-      }
-      void steal_for_pipe(Pipe *new_owner) {
-        Mutex::Locker l(delay_lock);
-        pipe = new_owner;
-      }
-      /**
-       * We need to stop fast dispatching before we need to stop putting
-       * normal messages into the DispatchQueue.
-       */
-      void stop_fast_dispatching();
-    } *delay_thread;
-    friend class DelayedDelivery;
-
-  public:
-    Pipe(SimpleMessenger *r, int st, PipeConnection *con);
-    ~Pipe();
-
-    SimpleMessenger *msgr;
-    uint64_t conn_id;
-    ostream& _pipe_prefix(std::ostream *_dout);
-
-    Pipe* get() {
-      return static_cast<Pipe*>(RefCountedObject::get());
-    }
-
-    char *recv_buf;
-    int recv_max_prefetch;
-    int recv_ofs;
-    int recv_len;
-
-    enum {
-      STATE_ACCEPTING,
-      STATE_CONNECTING,
-      STATE_OPEN,
-      STATE_STANDBY,
-      STATE_CLOSED,
-      STATE_CLOSING,
-      STATE_WAIT       // just wait for racing connection
-    };
-
-    static const char *get_state_name(int s) {
-      switch (s) {
-      case STATE_ACCEPTING: return "accepting";
-      case STATE_CONNECTING: return "connecting";
-      case STATE_OPEN: return "open";
-      case STATE_STANDBY: return "standby";
-      case STATE_CLOSED: return "closed";
-      case STATE_CLOSING: return "closing";
-      case STATE_WAIT: return "wait";
-      default: return "UNKNOWN";
-      }
-    }
-    const char *get_state_name() {
-      return get_state_name(state);
-    }
-
-  private:
-    int sd;
-    struct iovec msgvec[IOV_MAX];
-
-  public:
-    int port;
-    int peer_type;
-    entity_addr_t peer_addr;
-    Messenger::Policy policy;
-    
-    Mutex pipe_lock;
-    int state;
-    atomic_t state_closed; // non-zero iff state = STATE_CLOSED
-
-    // session_security handles any signatures or encryptions required for this pipe's msgs. PLR
-
-    ceph::shared_ptr<AuthSessionHandler> session_security;
-
-  protected:
-    friend class SimpleMessenger;
-    PipeConnectionRef connection_state;
-
-    utime_t backoff;         // backoff time
-
-    bool reader_running, reader_needs_join;
-    bool reader_dispatching; /// reader thread is dispatching without pipe_lock
-    bool writer_running;
-
-    map<int, list<Message*> > out_q;  // priority queue for outbound msgs
-    DispatchQueue *in_q;
-    list<Message*> sent;
-    Cond cond;
-    bool send_keepalive;
-    bool send_keepalive_ack;
-    utime_t keepalive_ack_stamp;
-    bool halt_delivery; //if a pipe's queue is destroyed, stop adding to it
-    
-    __u32 connect_seq, peer_global_seq;
-    uint64_t out_seq;
-    uint64_t in_seq, in_seq_acked;
-    
-    void set_socket_options();
-
-    int accept();   // server handshake
-    int connect();  // client handshake
-    void reader();
-    void writer();
-    void unlock_maybe_reap();
-
-    int randomize_out_seq();
-
-    int read_message(Message **pm,
-                    AuthSessionHandler *session_security_copy);
-    int write_message(ceph_msg_header& h, ceph_msg_footer& f, bufferlist& body);
-    /**
-     * Write the given data (of length len) to the Pipe's socket. This function
-     * will loop until all passed data has been written out.
-     * If more is set, the function will optimize socket writes
-     * for additional data (by passing the MSG_MORE flag, aka TCP_CORK).
-     *
-     * @param msg The msghdr to write out
-     * @param len The length of the data in msg
-     * @param more Should be set true if this is one part of a larger message
-     * @return 0, or -1 on failure (unrecoverable -- close the socket).
-     */
-    int do_sendmsg(struct msghdr *msg, int len, bool more=false);
-    int write_ack(uint64_t s);
-    int write_keepalive();
-    int write_keepalive2(char tag, const utime_t &t);
-
-    void fault(bool reader=false);
-
-    void was_session_reset();
-
-    /* Clean up sent list */
-    void handle_ack(uint64_t seq);
-
-    public:
-    Pipe(const Pipe& other);
-    const Pipe& operator=(const Pipe& other);
-
-    void start_reader();
-    void start_writer();
-    void maybe_start_delay_thread();
-    void join_reader();
-
-    // public constructors
-    static const Pipe& Server(int s);
-    static const Pipe& Client(const entity_addr_t& pi);
-
-    __u32 get_out_seq() { return out_seq; }
-
-    bool is_queued() { return !out_q.empty() || send_keepalive || send_keepalive_ack; }
-
-    entity_addr_t& get_peer_addr() { return peer_addr; }
-
-    void set_peer_addr(const entity_addr_t& a) {
-      if (&peer_addr != &a)  // shut up valgrind
-        peer_addr = a;
-      connection_state->set_peer_addr(a);
-    }
-    void set_peer_type(int t) {
-      peer_type = t;
-      connection_state->set_peer_type(t);
-    }
-
-    void register_pipe();
-    void unregister_pipe();
-    void join();
-    /// stop a Pipe by closing its socket and setting it to STATE_CLOSED
-    void stop();
-    /// stop() a Pipe if not already done, and wait for it to finish any
-    /// fast_dispatch in progress.
-    void stop_and_wait();
-
-    void _send(Message *m) {
-      assert(pipe_lock.is_locked());
-      out_q[m->get_priority()].push_back(m);
-      cond.Signal();
-    }
-    void _send_keepalive() {
-      assert(pipe_lock.is_locked());
-      send_keepalive = true;
-      cond.Signal();
-    }
-    Message *_get_next_outgoing() {
-      assert(pipe_lock.is_locked());
-      Message *m = 0;
-      while (!m && !out_q.empty()) {
-        map<int, list<Message*> >::reverse_iterator p = out_q.rbegin();
-        if (!p->second.empty()) {
-          m = p->second.front();
-          p->second.pop_front();
-        }
-        if (p->second.empty())
-          out_q.erase(p->first);
-      }
-      return m;
-    }
-
-    /// move all messages in the sent list back into the queue at the highest priority.
-    void requeue_sent();
-    /// discard messages requeued by requeued_sent() up to a given seq
-    void discard_requeued_up_to(uint64_t seq);
-    void discard_out_queue();
-
-    void shutdown_socket() {
-      recv_reset();
-      if (sd >= 0)
-        ::shutdown(sd, SHUT_RDWR);
-    }
-
-    void recv_reset() {
-      recv_len = 0;
-      recv_ofs = 0;
-    }
-    int do_recv(char *buf, size_t len, int flags);
-    int buffered_recv(char *buf, size_t len, int flags);
-    bool has_pending_data() { return recv_len > recv_ofs; }
-
-    /**
-     * do a blocking read of len bytes from socket
-     *
-     * @param buf buffer to read into
-     * @param len exact number of bytes to read
-     * @return 0 for success, or -1 on error
-     */
-    int tcp_read(char *buf, int len);
-
-    /**
-     * wait for bytes to become available on the socket
-     *
-     * @return 0 for success, or -1 on error
-     */
-    int tcp_read_wait();
-
-    /**
-     * non-blocking read of available bytes on socket
-     *
-     * This is expected to be used after tcp_read_wait(), and will return
-     * an error if there is no data on the socket to consume.
-     *
-     * @param buf buffer to read into
-     * @param len maximum number of bytes to read
-     * @return bytes read, or -1 on error or when there is no data
-     */
-    int tcp_read_nonblocking(char *buf, int len);
-
-    /**
-     * blocking write of bytes to socket
-     *
-     * @param buf buffer
-     * @param len number of bytes to write
-     * @return 0 for success, or -1 on error
-     */
-    int tcp_write(const char *buf, int len);
-
-  };
-
-
-#endif
diff --git a/src/msg/PipeConnection.cc b/src/msg/PipeConnection.cc
deleted file mode 100644 (file)
index 695b226..0000000
+++ /dev/null
@@ -1,91 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation.  See file COPYING.
- *
- */
-
-#include "Message.h"
-#include "Pipe.h"
-#include "SimpleMessenger.h"
-#include "PipeConnection.h"
-
-PipeConnection::~PipeConnection()
-{
-  if (pipe) {
-    pipe->put();
-    pipe = NULL;
-  }
-}
-
-Pipe* PipeConnection::get_pipe()
-{
-  Mutex::Locker l(lock);
-  if (pipe)
-    return pipe->get();
-  return NULL;
-}
-
-bool PipeConnection::try_get_pipe(Pipe **p)
-{
-  Mutex::Locker l(lock);
-  if (failed) {
-    *p = NULL;
-  } else {
-    if (pipe)
-      *p = pipe->get();
-    else
-      *p = NULL;
-  }
-  return !failed;
-}
-
-bool PipeConnection::clear_pipe(Pipe *old_p)
-{
-  Mutex::Locker l(lock);
-  if (old_p == pipe) {
-    pipe->put();
-    pipe = NULL;
-    failed = true;
-    return true;
-  }
-  return false;
-}
-
-void PipeConnection::reset_pipe(Pipe *p)
-{
-  Mutex::Locker l(lock);
-  if (pipe)
-    pipe->put();
-  pipe = p->get();
-}
-
-int PipeConnection::send_message(Message *m)
-{
-  assert(msgr);
-  return static_cast<SimpleMessenger*>(msgr)->send_message(m, this);
-}
-
-void PipeConnection::send_keepalive()
-{
-  static_cast<SimpleMessenger*>(msgr)->send_keepalive(this);
-}
-
-void PipeConnection::mark_down()
-{
-  if (msgr)
-    static_cast<SimpleMessenger*>(msgr)->mark_down(this);
-}
-
-void PipeConnection::mark_disposable()
-{
-  if (msgr)
-    static_cast<SimpleMessenger*>(msgr)->mark_disposable(this);
-}
diff --git a/src/msg/PipeConnection.h b/src/msg/PipeConnection.h
deleted file mode 100644 (file)
index c27a63c..0000000
+++ /dev/null
@@ -1,57 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2014 Red Hat
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation.  See file COPYING.
- *
- */
-
-#ifndef CEPH_MSG_PIPECONNECTION_H
-#define CEPH_MSG_PIPECONNECTION_H
-
-#include "Connection.h"
-class Pipe;
-
-struct PipeConnection : public Connection {
-  Pipe* pipe;
-
-  friend class boost::intrusive_ptr<PipeConnection>;
-  friend class Pipe;
-
-public:
-
-  PipeConnection(CephContext *cct, Messenger *m)
-    : Connection(cct, m),
-      pipe(NULL) { }
-
-  ~PipeConnection();
-
-  Pipe* get_pipe();
-
-  bool try_get_pipe(Pipe** p);
-
-  bool clear_pipe(Pipe* old_p);
-
-  void reset_pipe(Pipe* p);
-
-  bool is_connected() {
-    Mutex::Locker l(lock);
-    return pipe != NULL;
-  }
-
-  int send_message(Message *m);
-  void send_keepalive();
-  void mark_down();
-  void mark_disposable();
-
-}; /* PipeConnection */
-
-typedef boost::intrusive_ptr<PipeConnection> PipeConnectionRef;
-
-#endif
diff --git a/src/msg/SimpleMessenger.cc b/src/msg/SimpleMessenger.cc
deleted file mode 100644 (file)
index cf6bc73..0000000
+++ /dev/null
@@ -1,702 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software 
- * Foundation.  See file COPYING.
- * 
- */
-
-#include <errno.h>
-#include <iostream>
-#include <fstream>
-
-
-#include "SimpleMessenger.h"
-
-#include "common/config.h"
-#include "common/Timer.h"
-#include "common/errno.h"
-#include "auth/Crypto.h"
-#include "include/Spinlock.h"
-
-#define dout_subsys ceph_subsys_ms
-#undef dout_prefix
-#define dout_prefix _prefix(_dout, this)
-static ostream& _prefix(std::ostream *_dout, SimpleMessenger *msgr) {
-  return *_dout << "-- " << msgr->get_myaddr() << " ";
-}
-
-
-/*******************
- * SimpleMessenger
- */
-
-SimpleMessenger::SimpleMessenger(CephContext *cct, entity_name_t name,
-                                string mname, uint64_t _nonce)
-  : SimplePolicyMessenger(cct, name,mname, _nonce),
-    accepter(this, _nonce),
-    dispatch_queue(cct, this),
-    reaper_thread(this),
-    nonce(_nonce),
-    lock("SimpleMessenger::lock"), need_addr(true), did_bind(false),
-    global_seq(0),
-    cluster_protocol(0),
-    dispatch_throttler(cct, string("msgr_dispatch_throttler-") + mname,
-                      cct->_conf->ms_dispatch_throttle_bytes),
-    reaper_started(false), reaper_stop(false),
-    timeout(0),
-    local_connection(new PipeConnection(cct, this))
-{
-  ceph_spin_init(&global_seq_lock);
-  init_local_connection();
-}
-
-/**
- * Destroy the SimpleMessenger. Pretty simple since all the work is done
- * elsewhere.
- */
-SimpleMessenger::~SimpleMessenger()
-{
-  assert(!did_bind); // either we didn't bind or we shut down the Accepter
-  assert(rank_pipe.empty()); // we don't have any running Pipes.
-  assert(!reaper_started); // the reaper thread is stopped
-}
-
-void SimpleMessenger::ready()
-{
-  ldout(cct,10) << "ready " << get_myaddr() << dendl;
-  dispatch_queue.start();
-
-  lock.Lock();
-  if (did_bind)
-    accepter.start();
-  lock.Unlock();
-}
-
-
-int SimpleMessenger::shutdown()
-{
-  ldout(cct,10) << "shutdown " << get_myaddr() << dendl;
-  mark_down_all();
-  dispatch_queue.shutdown();
-
-  // break ref cycles on the loopback connection
-  local_connection->set_priv(NULL);
-  return 0;
-}
-
-int SimpleMessenger::_send_message(Message *m, const entity_inst_t& dest)
-{
-  // set envelope
-  m->get_header().src = get_myname();
-
-  if (!m->get_priority()) m->set_priority(get_default_send_priority());
-  ldout(cct,1) <<"--> " << dest.name << " "
-          << dest.addr << " -- " << *m
-         << " -- ?+" << m->get_data().length()
-         << " " << m 
-         << dendl;
-
-  if (dest.addr == entity_addr_t()) {
-    ldout(cct,0) << "send_message message " << *m
-                 << " with empty dest " << dest.addr << dendl;
-    m->put();
-    return -EINVAL;
-  }
-
-  lock.Lock();
-  Pipe *pipe = _lookup_pipe(dest.addr);
-  submit_message(m, (pipe ? pipe->connection_state.get() : NULL),
-                 dest.addr, dest.name.type(), true);
-  lock.Unlock();
-  return 0;
-}
-
-int SimpleMessenger::_send_message(Message *m, Connection *con)
-{
-  //set envelope
-  m->get_header().src = get_myname();
-
-  if (!m->get_priority()) m->set_priority(get_default_send_priority());
-
-  ldout(cct,1) << "--> " << con->get_peer_addr()
-      << " -- " << *m
-      << " -- ?+" << m->get_data().length()
-      << " " << m << " con " << con
-      << dendl;
-
-  submit_message(m, static_cast<PipeConnection*>(con),
-                con->get_peer_addr(), con->get_peer_type(), false);
-  return 0;
-}
-
-/**
- * If my_inst.addr doesn't have an IP set, this function
- * will fill it in from the passed addr. Otherwise it does nothing and returns.
- */
-void SimpleMessenger::set_addr_unknowns(entity_addr_t &addr)
-{
-  if (my_inst.addr.is_blank_ip()) {
-    int port = my_inst.addr.get_port();
-    my_inst.addr.addr = addr.addr;
-    my_inst.addr.set_port(port);
-    init_local_connection();
-  }
-}
-
-int SimpleMessenger::get_proto_version(int peer_type, bool connect)
-{
-  int my_type = my_inst.name.type();
-
-  // set reply protocol version
-  if (peer_type == my_type) {
-    // internal
-    return cluster_protocol;
-  } else {
-    // public
-    if (connect) {
-      switch (peer_type) {
-      case CEPH_ENTITY_TYPE_OSD: return CEPH_OSDC_PROTOCOL;
-      case CEPH_ENTITY_TYPE_MDS: return CEPH_MDSC_PROTOCOL;
-      case CEPH_ENTITY_TYPE_MON: return CEPH_MONC_PROTOCOL;
-      }
-    } else {
-      switch (my_type) {
-      case CEPH_ENTITY_TYPE_OSD: return CEPH_OSDC_PROTOCOL;
-      case CEPH_ENTITY_TYPE_MDS: return CEPH_MDSC_PROTOCOL;
-      case CEPH_ENTITY_TYPE_MON: return CEPH_MONC_PROTOCOL;
-      }
-    }
-  }
-  return 0;
-}
-
-
-
-
-
-
-
-/********************************************
- * SimpleMessenger
- */
-#undef dout_prefix
-#define dout_prefix _prefix(_dout, this)
-
-void SimpleMessenger::dispatch_throttle_release(uint64_t msize)
-{
-  if (msize) {
-    ldout(cct,10) << "dispatch_throttle_release " << msize << " to dispatch throttler "
-           << dispatch_throttler.get_current() << "/"
-           << dispatch_throttler.get_max() << dendl;
-    dispatch_throttler.put(msize);
-  }
-}
-
-void SimpleMessenger::reaper_entry()
-{
-  ldout(cct,10) << "reaper_entry start" << dendl;
-  lock.Lock();
-  while (!reaper_stop) {
-    reaper();  // may drop and retake the lock
-    if (reaper_stop)
-      break;
-    reaper_cond.Wait(lock);
-  }
-  lock.Unlock();
-  ldout(cct,10) << "reaper_entry done" << dendl;
-}
-
-/*
- * note: assumes lock is held
- */
-void SimpleMessenger::reaper()
-{
-  ldout(cct,10) << "reaper" << dendl;
-  assert(lock.is_locked());
-
-  while (!pipe_reap_queue.empty()) {
-    Pipe *p = pipe_reap_queue.front();
-    pipe_reap_queue.pop_front();
-    ldout(cct,10) << "reaper reaping pipe " << p << " " <<
-      p->get_peer_addr() << dendl;
-    p->pipe_lock.Lock();
-    p->discard_out_queue();
-    if (p->connection_state) {
-      // mark_down, mark_down_all, or fault() should have done this,
-      // or accept() may have switch the Connection to a different
-      // Pipe... but make sure!
-      bool cleared = p->connection_state->clear_pipe(p);
-      assert(!cleared);
-    }
-    p->pipe_lock.Unlock();
-    p->unregister_pipe();
-    assert(pipes.count(p));
-    pipes.erase(p);
-
-    // drop msgr lock while joining thread; the delay through could be
-    // trying to fast dispatch, preventing it from joining without
-    // blocking and deadlocking.
-    lock.Unlock();
-    p->join();
-    lock.Lock();
-
-    if (p->sd >= 0)
-      ::close(p->sd);
-    ldout(cct,10) << "reaper reaped pipe " << p << " " << p->get_peer_addr() << dendl;
-    p->put();
-    ldout(cct,10) << "reaper deleted pipe " << p << dendl;
-  }
-  ldout(cct,10) << "reaper done" << dendl;
-}
-
-void SimpleMessenger::queue_reap(Pipe *pipe)
-{
-  ldout(cct,10) << "queue_reap " << pipe << dendl;
-  lock.Lock();
-  pipe_reap_queue.push_back(pipe);
-  reaper_cond.Signal();
-  lock.Unlock();
-}
-
-
-
-int SimpleMessenger::bind(const entity_addr_t &bind_addr)
-{
-  lock.Lock();
-  if (started) {
-    ldout(cct,10) << "rank.bind already started" << dendl;
-    lock.Unlock();
-    return -1;
-  }
-  ldout(cct,10) << "rank.bind " << bind_addr << dendl;
-  lock.Unlock();
-
-  // bind to a socket
-  set<int> avoid_ports;
-  int r = accepter.bind(bind_addr, avoid_ports);
-  if (r >= 0)
-    did_bind = true;
-  return r;
-}
-
-int SimpleMessenger::rebind(const set<int>& avoid_ports)
-{
-  ldout(cct,1) << "rebind avoid " << avoid_ports << dendl;
-  assert(did_bind);
-  accepter.stop();
-  mark_down_all();
-  return accepter.rebind(avoid_ports);
-}
-
-int SimpleMessenger::start()
-{
-  lock.Lock();
-  ldout(cct,1) << "messenger.start" << dendl;
-
-  // register at least one entity, first!
-  assert(my_inst.name.type() >= 0);
-
-  assert(!started);
-  started = true;
-
-  if (!did_bind) {
-    my_inst.addr.nonce = nonce;
-    init_local_connection();
-  }
-
-  lock.Unlock();
-
-  reaper_started = true;
-  reaper_thread.create();
-  return 0;
-}
-
-Pipe *SimpleMessenger::add_accept_pipe(int sd)
-{
-  lock.Lock();
-  Pipe *p = new Pipe(this, Pipe::STATE_ACCEPTING, NULL);
-  p->sd = sd;
-  p->pipe_lock.Lock();
-  p->start_reader();
-  p->pipe_lock.Unlock();
-  pipes.insert(p);
-  accepting_pipes.insert(p);
-  lock.Unlock();
-  return p;
-}
-
-/* connect_rank
- * NOTE: assumes messenger.lock held.
- */
-Pipe *SimpleMessenger::connect_rank(const entity_addr_t& addr,
-                                   int type,
-                                   PipeConnection *con,
-                                   Message *first)
-{
-  assert(lock.is_locked());
-  assert(addr != my_inst.addr);
-  
-  ldout(cct,10) << "connect_rank to " << addr << ", creating pipe and registering" << dendl;
-  
-  // create pipe
-  Pipe *pipe = new Pipe(this, Pipe::STATE_CONNECTING,
-                       static_cast<PipeConnection*>(con));
-  pipe->pipe_lock.Lock();
-  pipe->set_peer_type(type);
-  pipe->set_peer_addr(addr);
-  pipe->policy = get_policy(type);
-  pipe->start_writer();
-  if (first)
-    pipe->_send(first);
-  pipe->pipe_lock.Unlock();
-  pipe->register_pipe();
-  pipes.insert(pipe);
-
-  return pipe;
-}
-
-
-
-
-
-
-AuthAuthorizer *SimpleMessenger::get_authorizer(int peer_type, bool force_new)
-{
-  return ms_deliver_get_authorizer(peer_type, force_new);
-}
-
-bool SimpleMessenger::verify_authorizer(Connection *con, int peer_type,
-                                       int protocol, bufferlist& authorizer, bufferlist& authorizer_reply,
-                                       bool& isvalid,CryptoKey& session_key)
-{
-  return ms_deliver_verify_authorizer(con, peer_type, protocol, authorizer, authorizer_reply, isvalid,session_key);
-}
-
-ConnectionRef SimpleMessenger::get_connection(const entity_inst_t& dest)
-{
-  Mutex::Locker l(lock);
-  if (my_inst.addr == dest.addr) {
-    // local
-    return local_connection;
-  }
-
-  // remote
-  while (true) {
-    Pipe *pipe = _lookup_pipe(dest.addr);
-    if (pipe) {
-      ldout(cct, 10) << "get_connection " << dest << " existing " << pipe << dendl;
-    } else {
-      pipe = connect_rank(dest.addr, dest.name.type(), NULL, NULL);
-      ldout(cct, 10) << "get_connection " << dest << " new " << pipe << dendl;
-    }
-    Mutex::Locker l(pipe->pipe_lock);
-    if (pipe->connection_state)
-      return pipe->connection_state;
-    // we failed too quickly!  retry.  FIXME.
-  }
-}
-
-ConnectionRef SimpleMessenger::get_loopback_connection()
-{
-  return local_connection;
-}
-
-void SimpleMessenger::submit_message(Message *m, PipeConnection *con,
-                                    const entity_addr_t& dest_addr, int dest_type,
-                                    bool already_locked)
-{
-  if (cct->_conf->ms_dump_on_send) {
-    m->encode(-1, true);
-    ldout(cct, 0) << "submit_message " << *m << "\n";
-    m->get_payload().hexdump(*_dout);
-    if (m->get_data().length() > 0) {
-      *_dout << " data:\n";
-      m->get_data().hexdump(*_dout);
-    }
-    *_dout << dendl;
-    m->clear_payload();
-  }
-
-  // existing connection?
-  if (con) {
-    Pipe *pipe = NULL;
-    bool ok = static_cast<PipeConnection*>(con)->try_get_pipe(&pipe);
-    if (!ok) {
-      ldout(cct,0) << "submit_message " << *m << " remote, " << dest_addr
-                  << ", failed lossy con, dropping message " << m << dendl;
-      m->put();
-      return;
-    }
-    while (pipe && ok) {
-      // we loop in case of a racing reconnect, either from us or them
-      pipe->pipe_lock.Lock(); // can't use a Locker because of the Pipe ref
-      if (pipe->state != Pipe::STATE_CLOSED) {
-       ldout(cct,20) << "submit_message " << *m << " remote, " << dest_addr << ", have pipe." << dendl;
-       pipe->_send(m);
-       pipe->pipe_lock.Unlock();
-       pipe->put();
-       return;
-      }
-      Pipe *current_pipe;
-      ok = con->try_get_pipe(&current_pipe);
-      pipe->pipe_lock.Unlock();
-      if (current_pipe == pipe) {
-       ldout(cct,20) << "submit_message " << *m << " remote, " << dest_addr
-                     << ", had pipe " << pipe << ", but it closed." << dendl;
-       pipe->put();
-       current_pipe->put();
-       m->put();
-       return;
-      } else {
-       pipe->put();
-       pipe = current_pipe;
-      }
-    }
-  }
-
-  // local?
-  if (my_inst.addr == dest_addr) {
-    // local
-    ldout(cct,20) << "submit_message " << *m << " local" << dendl;
-    dispatch_queue.local_delivery(m, m->get_priority());
-    return;
-  }
-
-  // remote, no existing pipe.
-  const Policy& policy = get_policy(dest_type);
-  if (policy.server) {
-    ldout(cct,20) << "submit_message " << *m << " remote, " << dest_addr << ", lossy server for target type "
-                 << ceph_entity_type_name(dest_type) << ", no session, dropping." << dendl;
-    m->put();
-  } else {
-    ldout(cct,20) << "submit_message " << *m << " remote, " << dest_addr << ", new pipe." << dendl;
-    if (!already_locked) {
-      /** We couldn't handle the Message without reference to global data, so
-       *  grab the lock and do it again. If we got here, we know it's a non-lossy
-       *  Connection, so we can use our existing pointer without doing another lookup. */
-      Mutex::Locker l(lock);
-      submit_message(m, con, dest_addr, dest_type, true);
-    } else {
-      connect_rank(dest_addr, dest_type, static_cast<PipeConnection*>(con), m);
-    }
-  }
-}
-
-int SimpleMessenger::send_keepalive(Connection *con)
-{
-  int ret = 0;
-  Pipe *pipe = static_cast<Pipe *>(
-    static_cast<PipeConnection*>(con)->get_pipe());
-  if (pipe) {
-    ldout(cct,20) << "send_keepalive con " << con << ", have pipe." << dendl;
-    assert(pipe->msgr == this);
-    pipe->pipe_lock.Lock();
-    pipe->_send_keepalive();
-    pipe->pipe_lock.Unlock();
-    pipe->put();
-  } else {
-    ldout(cct,0) << "send_keepalive con " << con << ", no pipe." << dendl;
-    ret = -EPIPE;
-  }
-  return ret;
-}
-
-
-
-void SimpleMessenger::wait()
-{
-  lock.Lock();
-  if (!started) {
-    lock.Unlock();
-    return;
-  }
-  lock.Unlock();
-
-  if(dispatch_queue.is_started()) {
-    ldout(cct,10) << "wait: waiting for dispatch queue" << dendl;
-    dispatch_queue.wait();
-    ldout(cct,10) << "wait: dispatch queue is stopped" << dendl;
-  }
-
-  // done!  clean up.
-  if (did_bind) {
-    ldout(cct,20) << "wait: stopping accepter thread" << dendl;
-    accepter.stop();
-    did_bind = false;
-    ldout(cct,20) << "wait: stopped accepter thread" << dendl;
-  }
-
-  if (reaper_started) {
-    ldout(cct,20) << "wait: stopping reaper thread" << dendl;
-    lock.Lock();
-    reaper_cond.Signal();
-    reaper_stop = true;
-    lock.Unlock();
-    reaper_thread.join();
-    reaper_started = false;
-    ldout(cct,20) << "wait: stopped reaper thread" << dendl;
-  }
-
-  // close+reap all pipes
-  lock.Lock();
-  {
-    ldout(cct,10) << "wait: closing pipes" << dendl;
-
-    while (!rank_pipe.empty()) {
-      Pipe *p = rank_pipe.begin()->second;
-      p->unregister_pipe();
-      p->pipe_lock.Lock();
-      p->stop_and_wait();
-      p->pipe_lock.Unlock();
-    }
-
-    reaper();
-    ldout(cct,10) << "wait: waiting for pipes " << pipes << " to close" << dendl;
-    while (!pipes.empty()) {
-      reaper_cond.Wait(lock);
-      reaper();
-    }
-  }
-  lock.Unlock();
-
-  ldout(cct,10) << "wait: done." << dendl;
-  ldout(cct,1) << "shutdown complete." << dendl;
-  started = false;
-}
-
-
-void SimpleMessenger::mark_down_all()
-{
-  ldout(cct,1) << "mark_down_all" << dendl;
-  lock.Lock();
-  for (set<Pipe*>::iterator q = accepting_pipes.begin(); q != accepting_pipes.end(); ++q) {
-    Pipe *p = *q;
-    ldout(cct,5) << "mark_down_all accepting_pipe " << p << dendl;
-    p->pipe_lock.Lock();
-    p->stop_and_wait();
-    PipeConnectionRef con = p->connection_state;
-    if (con && con->clear_pipe(p))
-      dispatch_queue.queue_reset(con.get());
-    p->pipe_lock.Unlock();
-  }
-  accepting_pipes.clear();
-
-  while (!rank_pipe.empty()) {
-    ceph::unordered_map<entity_addr_t,Pipe*>::iterator it = rank_pipe.begin();
-    Pipe *p = it->second;
-    ldout(cct,5) << "mark_down_all " << it->first << " " << p << dendl;
-    rank_pipe.erase(it);
-    p->unregister_pipe();
-    p->pipe_lock.Lock();
-    p->stop_and_wait();
-    PipeConnectionRef con = p->connection_state;
-    if (con && con->clear_pipe(p))
-      dispatch_queue.queue_reset(con.get());
-    p->pipe_lock.Unlock();
-  }
-  lock.Unlock();
-}
-
-void SimpleMessenger::mark_down(const entity_addr_t& addr)
-{
-  lock.Lock();
-  Pipe *p = _lookup_pipe(addr);
-  if (p) {
-    ldout(cct,1) << "mark_down " << addr << " -- " << p << dendl;
-    p->unregister_pipe();
-    p->pipe_lock.Lock();
-    p->stop_and_wait();
-    if (p->connection_state) {
-      // generate a reset event for the caller in this case, even
-      // though they asked for it, since this is the addr-based (and
-      // not Connection* based) interface
-      PipeConnectionRef con = p->connection_state;
-      if (con && con->clear_pipe(p))
-       dispatch_queue.queue_reset(con.get());
-    }
-    p->pipe_lock.Unlock();
-  } else {
-    ldout(cct,1) << "mark_down " << addr << " -- pipe dne" << dendl;
-  }
-  lock.Unlock();
-}
-
-void SimpleMessenger::mark_down(Connection *con)
-{
-  if (con == NULL)
-    return;
-  lock.Lock();
-  Pipe *p = static_cast<Pipe *>(static_cast<PipeConnection*>(con)->get_pipe());
-  if (p) {
-    ldout(cct,1) << "mark_down " << con << " -- " << p << dendl;
-    assert(p->msgr == this);
-    p->unregister_pipe();
-    p->pipe_lock.Lock();
-    p->stop_and_wait();
-    if (p->connection_state) {
-      // do not generate a reset event for the caller in this case,
-      // since they asked for it.
-      p->connection_state->clear_pipe(p);
-    }
-    p->pipe_lock.Unlock();
-    p->put();
-  } else {
-    ldout(cct,1) << "mark_down " << con << " -- pipe dne" << dendl;
-  }
-  lock.Unlock();
-}
-
-void SimpleMessenger::mark_disposable(Connection *con)
-{
-  lock.Lock();
-  Pipe *p = static_cast<Pipe *>(static_cast<PipeConnection*>(con)->get_pipe());
-  if (p) {
-    ldout(cct,1) << "mark_disposable " << con << " -- " << p << dendl;
-    assert(p->msgr == this);
-    p->pipe_lock.Lock();
-    p->policy.lossy = true;
-    p->pipe_lock.Unlock();
-    p->put();
-  } else {
-    ldout(cct,1) << "mark_disposable " << con << " -- pipe dne" << dendl;
-  }
-  lock.Unlock();
-}
-
-void SimpleMessenger::learned_addr(const entity_addr_t &peer_addr_for_me)
-{
-  // be careful here: multiple threads may block here, and readers of
-  // my_inst.addr do NOT hold any lock.
-
-  // this always goes from true -> false under the protection of the
-  // mutex.  if it is already false, we need not retake the mutex at
-  // all.
-  if (!need_addr)
-    return;
-
-  lock.Lock();
-  if (need_addr) {
-    entity_addr_t t = peer_addr_for_me;
-    t.set_port(my_inst.addr.get_port());
-    my_inst.addr.addr = t.addr;
-    ldout(cct,1) << "learned my addr " << my_inst.addr << dendl;
-    need_addr = false;
-    init_local_connection();
-  }
-  lock.Unlock();
-}
-
-void SimpleMessenger::init_local_connection()
-{
-  local_connection->peer_addr = my_inst.addr;
-  local_connection->peer_type = my_inst.name.type();
-  ms_deliver_handle_fast_connect(local_connection.get());
-}
diff --git a/src/msg/SimpleMessenger.h b/src/msg/SimpleMessenger.h
deleted file mode 100644 (file)
index 006b60b..0000000
+++ /dev/null
@@ -1,417 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software 
- * Foundation.  See file COPYING.
- * 
- */
-
-#ifndef CEPH_SIMPLEMESSENGER_H
-#define CEPH_SIMPLEMESSENGER_H
-
-#include "include/types.h"
-#include "include/xlist.h"
-
-#include <list>
-#include <map>
-using namespace std;
-#include "include/unordered_map.h"
-#include "include/unordered_set.h"
-
-#include "common/Mutex.h"
-#include "include/atomic.h"
-#include "common/Cond.h"
-#include "common/Thread.h"
-#include "common/Throttle.h"
-
-#include "SimplePolicyMessenger.h"
-#include "Message.h"
-#include "include/assert.h"
-#include "DispatchQueue.h"
-
-#include "Pipe.h"
-#include "Accepter.h"
-#include "include/Spinlock.h"
-
-/*
- * This class handles transmission and reception of messages. Generally
- * speaking, there are several major components:
- *
- * - Connection
- *    Each logical session is associated with a Connection.
- * - Pipe
- *    Each network connection is handled through a pipe, which handles
- *    the input and output of each message.  There is normally a 1:1
- *    relationship between Pipe and Connection, but logical sessions may
- *    get handed off between Pipes when sockets reconnect or during
- *    connection races.
- * - IncomingQueue
- *    Incoming messages are associated with an IncomingQueue, and there
- *    is one such queue associated with each Pipe.
- * - DispatchQueue
- *    IncomingQueues get queued in the DIspatchQueue, which is responsible
- *    for doing a round-robin sweep and processing them via a worker thread.
- * - SimpleMessenger
- *    It's the exterior class passed to the external message handler and
- *    most of the API details.
- *
- * Lock ordering:
- *
- *   SimpleMessenger::lock
- *       Pipe::pipe_lock
- *           DispatchQueue::lock
- *               IncomingQueue::lock
- */
-
-class SimpleMessenger : public SimplePolicyMessenger {
-  // First we have the public Messenger interface implementation...
-public:
-  /**
-   * Initialize the SimpleMessenger!
-   *
-   * @param cct The CephContext to use
-   * @param name The name to assign ourselves
-   * _nonce A unique ID to use for this SimpleMessenger. It should not
-   * be a value that will be repeated if the daemon restarts.
-   */
-  SimpleMessenger(CephContext *cct, entity_name_t name,
-                 string mname, uint64_t _nonce);
-
-  /**
-   * Destroy the SimpleMessenger. Pretty simple since all the work is done
-   * elsewhere.
-   */
-  virtual ~SimpleMessenger();
-
-  /** @defgroup Accessors
-   * @{
-   */
-  void set_addr_unknowns(entity_addr_t& addr);
-
-  int get_dispatch_queue_len() {
-    return dispatch_queue.get_queue_len();
-  }
-
-  double get_dispatch_queue_max_age(utime_t now) {
-    return dispatch_queue.get_max_age(now);
-  }
-  /** @} Accessors */
-
-  /**
-   * @defgroup Configuration functions
-   * @{
-   */
-  void set_cluster_protocol(int p) {
-    assert(!started && !did_bind);
-    cluster_protocol = p;
-  }
-
-  int bind(const entity_addr_t& bind_addr);
-  int rebind(const set<int>& avoid_ports);
-
-  /** @} Configuration functions */
-
-  /**
-   * @defgroup Startup/Shutdown
-   * @{
-   */
-  virtual int start();
-  virtual void wait();
-  virtual int shutdown();
-
-  /** @} // Startup/Shutdown */
-
-  /**
-   * @defgroup Messaging
-   * @{
-   */
-  virtual int send_message(Message *m, const entity_inst_t& dest) {
-    return _send_message(m, dest);
-  }
-
-  int send_message(Message *m, Connection *con) {
-    return _send_message(m, con);
-  }
-
-  /** @} // Messaging */
-
-  /**
-   * @defgroup Connection Management
-   * @{
-   */
-  virtual ConnectionRef get_connection(const entity_inst_t& dest);
-  virtual ConnectionRef get_loopback_connection();
-  int send_keepalive(Connection *con);
-  virtual void mark_down(const entity_addr_t& addr);
-  void mark_down(Connection *con);
-  void mark_disposable(Connection *con);
-  virtual void mark_down_all();
-  /** @} // Connection Management */
-protected:
-  /**
-   * @defgroup Messenger Interfaces
-   * @{
-   */
-  /**
-   * Start up the DispatchQueue thread once we have somebody to dispatch to.
-   */
-  virtual void ready();
-  /** @} // Messenger Interfaces */
-private:
-  /**
-   * @defgroup Inner classes
-   * @{
-   */
-
-public:
-  Accepter accepter;
-  DispatchQueue dispatch_queue;
-
-  friend class Accepter;
-
-  /**
-   * Register a new pipe for accept
-   *
-   * @param sd socket
-   */
-  Pipe *add_accept_pipe(int sd);
-
-  Connection *create_anon_connection() {
-    return new PipeConnection(cct, NULL);
-  }
-
-private:
-
-  /**
-   * A thread used to tear down Pipes when they're complete.
-   */
-  class ReaperThread : public Thread {
-    SimpleMessenger *msgr;
-  public:
-    ReaperThread(SimpleMessenger *m) : msgr(m) {}
-    void *entry() {
-      msgr->reaper_entry();
-      return 0;
-    }
-  } reaper_thread;
-
-  /**
-   * @} // Inner classes
-   */
-
-  /**
-   * @defgroup Utility functions
-   * @{
-   */
-
-  /**
-   * Create a Pipe associated with the given entity (of the given type).
-   * Initiate the connection. (This function returning does not guarantee
-   * connection success.)
-   *
-   * @param addr The address of the entity to connect to.
-   * @param type The peer type of the entity at the address.
-   * @param con An existing Connection to associate with the new Pipe. If
-   * NULL, it creates a new Connection.
-   * @param msg an initial message to queue on the new pipe
-   *
-   * @return a pointer to the newly-created Pipe. Caller does not own a
-   * reference; take one if you need it.
-   */
-  Pipe *connect_rank(const entity_addr_t& addr, int type, PipeConnection *con,
-                    Message *first);
-  /**
-   * Send a message, lazily or not.
-   * This just glues send_message together and passes
-   * the input on to submit_message.
-   */
-  int _send_message(Message *m, const entity_inst_t& dest);
-  /**
-   * Same as above, but for the Connection-based variants.
-   */
-  int _send_message(Message *m, Connection *con);
-  /**
-   * Queue up a Message for delivery to the entity specified
-   * by addr and dest_type.
-   * submit_message() is responsible for creating
-   * new Pipes (and closing old ones) as necessary.
-   *
-   * @param m The Message to queue up. This function eats a reference.
-   * @param con The existing Connection to use, or NULL if you don't know of one.
-   * @param addr The address to send the Message to.
-   * @param dest_type The peer type of the address we're sending to
-   * just drop silently under failure.
-   * @param already_locked If false, submit_message() will acquire the
-   * SimpleMessenger lock before accessing shared data structures; otherwise
-   * it will assume the lock is held. NOTE: if you are making a request
-   * without locking, you MUST have filled in the con with a valid pointer.
-   */
-  void submit_message(Message *m, PipeConnection *con,
-                     const entity_addr_t& addr, int dest_type,
-                     bool already_locked);
-  /**
-   * Look through the pipes in the pipe_reap_queue and tear them down.
-   */
-  void reaper();
-  /**
-   * @} // Utility functions
-   */
-
-  // SimpleMessenger stuff
-  /// approximately unique ID set by the Constructor for use in entity_addr_t
-  uint64_t nonce;
-  /// overall lock used for SimpleMessenger data structures
-  Mutex lock;
-  /// true, specifying we haven't learned our addr; set false when we find it.
-  // maybe this should be protected by the lock?
-  bool need_addr;
-
-public:
-  bool get_need_addr() const { return need_addr; }
-
-private:
-  /**
-   *  false; set to true if the SimpleMessenger bound to a specific address;
-   *  and set false again by Accepter::stop(). This isn't lock-protected
-   *  since you shouldn't be able to race the only writers.
-   */
-  bool did_bind;
-  /// counter for the global seq our connection protocol uses
-  __u32 global_seq;
-  /// lock to protect the global_seq
-  ceph_spinlock_t global_seq_lock;
-
-  /**
-   * hash map of addresses to Pipes
-   *
-   * NOTE: a Pipe* with state CLOSED may still be in the map but is considered
-   * invalid and can be replaced by anyone holding the msgr lock
-   */
-  ceph::unordered_map<entity_addr_t, Pipe*> rank_pipe;
-  /**
-   * list of pipes are in teh process of accepting
-   *
-   * These are not yet in the rank_pipe map.
-   */
-  set<Pipe*> accepting_pipes;
-  /// a set of all the Pipes we have which are somehow active
-  set<Pipe*>      pipes;
-  /// a list of Pipes we want to tear down
-  list<Pipe*>     pipe_reap_queue;
-
-  /// internal cluster protocol version, if any, for talking to entities of the same type.
-  int cluster_protocol;
-
-  /// Throttle preventing us from building up a big backlog waiting for dispatch
-  Throttle dispatch_throttler;
-
-  bool reaper_started, reaper_stop;
-  Cond reaper_cond;
-
-  /// This Cond is slept on by wait() and signaled by dispatch_entry()
-  Cond  wait_cond;
-
-  friend class Pipe;
-
-  Pipe *_lookup_pipe(const entity_addr_t& k) {
-    ceph::unordered_map<entity_addr_t, Pipe*>::iterator p = rank_pipe.find(k);
-    if (p == rank_pipe.end())
-      return NULL;
-    // see lock cribbing in Pipe::fault()
-    if (p->second->state_closed.read())
-      return NULL;
-    return p->second;
-  }
-
-public:
-
-  int timeout;
-
-  /// con used for sending messages to ourselves
-  ConnectionRef local_connection;
-
-  /**
-   * @defgroup SimpleMessenger internals
-   * @{
-   */
-
-  /**
-   * This wraps ms_deliver_get_authorizer. We use it for Pipe.
-   */
-  AuthAuthorizer *get_authorizer(int peer_type, bool force_new);
-  /**
-   * This wraps ms_deliver_verify_authorizer; we use it for Pipe.
-   */
-  bool verify_authorizer(Connection *con, int peer_type, int protocol, bufferlist& auth, bufferlist& auth_reply,
-                         bool& isvalid,CryptoKey& session_key);
-  /**
-   * Increment the global sequence for this SimpleMessenger and return it.
-   * This is for the connect protocol, although it doesn't hurt if somebody
-   * else calls it.
-   *
-   * @return a global sequence ID that nobody else has seen.
-   */
-  __u32 get_global_seq(__u32 old=0) {
-    ceph_spin_lock(&global_seq_lock);
-    if (old > global_seq)
-      global_seq = old;
-    __u32 ret = ++global_seq;
-    ceph_spin_unlock(&global_seq_lock);
-    return ret;
-  }
-  /**
-   * Get the protocol version we support for the given peer type: either
-   * a peer protocol (if it matches our own), the protocol version for the
-   * peer (if we're connecting), or our protocol version (if we're accepting).
-   */
-  int get_proto_version(int peer_type, bool connect);
-
-  /**
-   * Fill in the address and peer type for the local connection, which
-   * is used for delivering messages back to ourself.
-   */
-  void init_local_connection();
-  /**
-   * Tell the SimpleMessenger its full IP address.
-   *
-   * This is used by Pipes when connecting to other endpoints, and
-   * probably shouldn't be called by anybody else.
-   */
-  void learned_addr(const entity_addr_t& peer_addr_for_me);
-
-  /**
-   * Release memory accounting back to the dispatch throttler.
-   *
-   * @param msize The amount of memory to release.
-   */
-  void dispatch_throttle_release(uint64_t msize);
-
-  /**
-   * This function is used by the reaper thread. As long as nobody
-   * has set reaper_stop, it calls the reaper function, then
-   * waits to be signaled when it needs to reap again (or when it needs
-   * to stop).
-   */
-  void reaper_entry();
-  /**
-   * Add a pipe to the pipe_reap_queue, to be torn down on
-   * the next call to reaper().
-   * It should really only be the Pipe calling this, in our current
-   * implementation.
-   *
-   * @param pipe A Pipe which has stopped its threads and is
-   * ready to be torn down.
-   */
-  void queue_reap(Pipe *pipe);
-  /**
-   * @} // SimpleMessenger Internals
-   */
-} ;
-
-#endif /* CEPH_SIMPLEMESSENGER_H */
diff --git a/src/msg/simple/Accepter.cc b/src/msg/simple/Accepter.cc
new file mode 100644 (file)
index 0000000..7fb2baa
--- /dev/null
@@ -0,0 +1,255 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software 
+ * Foundation.  See file COPYING.
+ * 
+ */
+
+#include <sys/socket.h>
+#include <netinet/tcp.h>
+#include <sys/uio.h>
+#include <limits.h>
+#include <poll.h>
+
+#include "msg/Message.h"
+
+#include "Accepter.h"
+#include "Pipe.h"
+#include "SimpleMessenger.h"
+
+#include "common/debug.h"
+#include "common/errno.h"
+
+#define dout_subsys ceph_subsys_ms
+
+#undef dout_prefix
+#define dout_prefix *_dout << "accepter."
+
+
+/********************************************
+ * Accepter
+ */
+
+int Accepter::bind(const entity_addr_t &bind_addr, const set<int>& avoid_ports)
+{
+  const md_config_t *conf = msgr->cct->_conf;
+  // bind to a socket
+  ldout(msgr->cct,10) << "accepter.bind" << dendl;
+  
+  int family;
+  switch (bind_addr.get_family()) {
+  case AF_INET:
+  case AF_INET6:
+    family = bind_addr.get_family();
+    break;
+
+  default:
+    // bind_addr is empty
+    family = conf->ms_bind_ipv6 ? AF_INET6 : AF_INET;
+  }
+
+  /* socket creation */
+  listen_sd = ::socket(family, SOCK_STREAM, 0);
+  if (listen_sd < 0) {
+    lderr(msgr->cct) << "accepter.bind unable to create socket: "
+                    << cpp_strerror(errno) << dendl;
+    return -errno;
+  }
+
+  // use whatever user specified (if anything)
+  entity_addr_t listen_addr = bind_addr;
+  listen_addr.set_family(family);
+
+  /* bind to port */
+  int rc = -1;
+  if (listen_addr.get_port()) {
+    // specific port
+
+    // reuse addr+port when possible
+    int on = 1;
+    rc = ::setsockopt(listen_sd, SOL_SOCKET, SO_REUSEADDR, &on, sizeof(on));
+    if (rc < 0) {
+      lderr(msgr->cct) << "accepter.bind unable to setsockopt: "
+                        << cpp_strerror(errno) << dendl;
+      return -errno;
+    }
+
+    rc = ::bind(listen_sd, (struct sockaddr *) &listen_addr.ss_addr(), listen_addr.addr_size());
+    if (rc < 0) {
+      lderr(msgr->cct) << "accepter.bind unable to bind to " << listen_addr.ss_addr()
+                      << ": " << cpp_strerror(errno) << dendl;
+      return -errno;
+    }
+  } else {
+    // try a range of ports
+    for (int port = msgr->cct->_conf->ms_bind_port_min; port <= msgr->cct->_conf->ms_bind_port_max; port++) {
+      if (avoid_ports.count(port))
+       continue;
+      listen_addr.set_port(port);
+      rc = ::bind(listen_sd, (struct sockaddr *) &listen_addr.ss_addr(), listen_addr.addr_size());
+      if (rc == 0)
+       break;
+    }
+    if (rc < 0) {
+      lderr(msgr->cct) << "accepter.bind unable to bind to " << listen_addr.ss_addr()
+                      << " on any port in range " << msgr->cct->_conf->ms_bind_port_min
+                      << "-" << msgr->cct->_conf->ms_bind_port_max
+                      << ": " << cpp_strerror(errno)
+                      << dendl;
+      return -errno;
+    }
+    ldout(msgr->cct,10) << "accepter.bind bound on random port " << listen_addr << dendl;
+  }
+
+  // what port did we get?
+  socklen_t llen = sizeof(listen_addr.ss_addr());
+  rc = getsockname(listen_sd, (sockaddr*)&listen_addr.ss_addr(), &llen);
+  if (rc < 0) {
+    rc = -errno;
+    lderr(msgr->cct) << "accepter.bind failed getsockname: " << cpp_strerror(rc) << dendl;
+    return rc;
+  }
+  
+  ldout(msgr->cct,10) << "accepter.bind bound to " << listen_addr << dendl;
+
+  // listen!
+  rc = ::listen(listen_sd, 128);
+  if (rc < 0) {
+    rc = -errno;
+    lderr(msgr->cct) << "accepter.bind unable to listen on " << listen_addr
+                    << ": " << cpp_strerror(rc) << dendl;
+    return rc;
+  }
+  
+  msgr->set_myaddr(bind_addr);
+  if (bind_addr != entity_addr_t())
+    msgr->learned_addr(bind_addr);
+  else
+    assert(msgr->get_need_addr());  // should still be true.
+
+  if (msgr->get_myaddr().get_port() == 0) {
+    msgr->set_myaddr(listen_addr);
+  }
+  entity_addr_t addr = msgr->get_myaddr();
+  addr.nonce = nonce;
+  msgr->set_myaddr(addr);
+
+  msgr->init_local_connection();
+
+  ldout(msgr->cct,1) << "accepter.bind my_inst.addr is " << msgr->get_myaddr()
+                    << " need_addr=" << msgr->get_need_addr() << dendl;
+  return 0;
+}
+
+int Accepter::rebind(const set<int>& avoid_ports)
+{
+  ldout(msgr->cct,1) << "accepter.rebind avoid " << avoid_ports << dendl;
+  
+  entity_addr_t addr = msgr->get_myaddr();
+  set<int> new_avoid = avoid_ports;
+  new_avoid.insert(addr.get_port());
+  addr.set_port(0);
+
+  // adjust the nonce; we want our entity_addr_t to be truly unique.
+  nonce += 1000000;
+  msgr->my_inst.addr.nonce = nonce;
+  ldout(msgr->cct,10) << " new nonce " << nonce << " and inst " << msgr->my_inst << dendl;
+
+  ldout(msgr->cct,10) << " will try " << addr << " and avoid ports " << new_avoid << dendl;
+  int r = bind(addr, new_avoid);
+  if (r == 0)
+    start();
+  return r;
+}
+
+int Accepter::start()
+{
+  ldout(msgr->cct,1) << "accepter.start" << dendl;
+
+  // start thread
+  create();
+
+  return 0;
+}
+
+void *Accepter::entry()
+{
+  ldout(msgr->cct,10) << "accepter starting" << dendl;
+  
+  int errors = 0;
+
+  struct pollfd pfd;
+  pfd.fd = listen_sd;
+  pfd.events = POLLIN | POLLERR | POLLNVAL | POLLHUP;
+  while (!done) {
+    ldout(msgr->cct,20) << "accepter calling poll" << dendl;
+    int r = poll(&pfd, 1, -1);
+    if (r < 0)
+      break;
+    ldout(msgr->cct,20) << "accepter poll got " << r << dendl;
+
+    if (pfd.revents & (POLLERR | POLLNVAL | POLLHUP))
+      break;
+
+    ldout(msgr->cct,10) << "pfd.revents=" << pfd.revents << dendl;
+    if (done) break;
+
+    // accept
+    entity_addr_t addr;
+    socklen_t slen = sizeof(addr.ss_addr());
+    int sd = ::accept(listen_sd, (sockaddr*)&addr.ss_addr(), &slen);
+    if (sd >= 0) {
+      errors = 0;
+      ldout(msgr->cct,10) << "accepted incoming on sd " << sd << dendl;
+      
+      msgr->add_accept_pipe(sd);
+    } else {
+      ldout(msgr->cct,0) << "accepter no incoming connection?  sd = " << sd
+             << " errno " << errno << " " << cpp_strerror(errno) << dendl;
+      if (++errors > 4)
+       break;
+    }
+  }
+
+  ldout(msgr->cct,20) << "accepter closing" << dendl;
+  // don't close socket, in case we start up again?  blech.
+  if (listen_sd >= 0) {
+    ::close(listen_sd);
+    listen_sd = -1;
+  }
+  ldout(msgr->cct,10) << "accepter stopping" << dendl;
+  return 0;
+}
+
+void Accepter::stop()
+{
+  done = true;
+  ldout(msgr->cct,10) << "stop accepter" << dendl;
+
+  if (listen_sd >= 0) {
+    ::shutdown(listen_sd, SHUT_RDWR);
+  }
+
+  // wait for thread to stop before closing the socket, to avoid
+  // racing against fd re-use.
+  if (is_started()) {
+    join();
+  }
+
+  if (listen_sd >= 0) {
+    ::close(listen_sd);
+    listen_sd = -1;
+  }
+  done = false;
+}
+
+
+
+
diff --git a/src/msg/simple/Accepter.h b/src/msg/simple/Accepter.h
new file mode 100644 (file)
index 0000000..4b1421f
--- /dev/null
@@ -0,0 +1,44 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software 
+ * Foundation.  See file COPYING.
+ * 
+ */
+
+#ifndef CEPH_MSG_ACCEPTER_H
+#define CEPH_MSG_ACCEPTER_H
+
+#include "msg/msg_types.h"
+#include "common/Thread.h"
+
+class SimpleMessenger;
+
+/**
+ * If the SimpleMessenger binds to a specific address, the Accepter runs
+ * and listens for incoming connections.
+ */
+class Accepter : public Thread {
+  SimpleMessenger *msgr;
+  bool done;
+  int listen_sd;
+  uint64_t nonce;
+
+public:
+  Accepter(SimpleMessenger *r, uint64_t n) : msgr(r), done(false), listen_sd(-1), nonce(n) {}
+    
+  void *entry();
+  void stop();
+  int bind(const entity_addr_t &bind_addr, const set<int>& avoid_ports);
+  int rebind(const set<int>& avoid_port);
+  int start();
+};
+
+
+#endif
diff --git a/src/msg/simple/DispatchQueue.cc b/src/msg/simple/DispatchQueue.cc
new file mode 100644 (file)
index 0000000..12c4846
--- /dev/null
@@ -0,0 +1,243 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software 
+ * Foundation.  See file COPYING.
+ * 
+ */
+
+#include "msg/Message.h"
+#include "DispatchQueue.h"
+#include "SimpleMessenger.h"
+#include "common/ceph_context.h"
+
+#define dout_subsys ceph_subsys_ms
+#include "common/debug.h"
+
+
+/*******************
+ * DispatchQueue
+ */
+
+#undef dout_prefix
+#define dout_prefix *_dout << "-- " << msgr->get_myaddr() << " "
+
+double DispatchQueue::get_max_age(utime_t now) {
+  Mutex::Locker l(lock);
+  if (marrival.empty())
+    return 0;
+  else
+    return (now - marrival.begin()->first);
+}
+
+uint64_t DispatchQueue::pre_dispatch(Message *m)
+{
+  ldout(cct,1) << "<== " << m->get_source_inst()
+              << " " << m->get_seq()
+              << " ==== " << *m
+              << " ==== " << m->get_payload().length()
+              << "+" << m->get_middle().length()
+              << "+" << m->get_data().length()
+              << " (" << m->get_footer().front_crc << " "
+              << m->get_footer().middle_crc
+              << " " << m->get_footer().data_crc << ")"
+              << " " << m << " con " << m->get_connection()
+              << dendl;
+  uint64_t msize = m->get_dispatch_throttle_size();
+  m->set_dispatch_throttle_size(0); // clear it out, in case we requeue this message.
+  return msize;
+}
+
+void DispatchQueue::post_dispatch(Message *m, uint64_t msize)
+{
+  msgr->dispatch_throttle_release(msize);
+  ldout(cct,20) << "done calling dispatch on " << m << dendl;
+}
+
+bool DispatchQueue::can_fast_dispatch(Message *m)
+{
+  return msgr->ms_can_fast_dispatch(m);
+}
+
+void DispatchQueue::fast_dispatch(Message *m)
+{
+  uint64_t msize = pre_dispatch(m);
+  msgr->ms_fast_dispatch(m);
+  post_dispatch(m, msize);
+}
+
+void DispatchQueue::fast_preprocess(Message *m)
+{
+  msgr->ms_fast_preprocess(m);
+}
+
+void DispatchQueue::enqueue(Message *m, int priority, uint64_t id)
+{
+
+  Mutex::Locker l(lock);
+  ldout(cct,20) << "queue " << m << " prio " << priority << dendl;
+  add_arrival(m);
+  if (priority >= CEPH_MSG_PRIO_LOW) {
+    mqueue.enqueue_strict(
+        id, priority, QueueItem(m));
+  } else {
+    mqueue.enqueue(
+        id, priority, m->get_cost(), QueueItem(m));
+  }
+  cond.Signal();
+}
+
+void DispatchQueue::local_delivery(Message *m, int priority)
+{
+  m->set_connection(msgr->local_connection.get());
+  m->set_recv_stamp(ceph_clock_now(msgr->cct));
+  Mutex::Locker l(local_delivery_lock);
+  if (local_messages.empty())
+    local_delivery_cond.Signal();
+  local_messages.push_back(make_pair(m, priority));
+  return;
+}
+
+void DispatchQueue::run_local_delivery()
+{
+  local_delivery_lock.Lock();
+  while (true) {
+    if (stop_local_delivery)
+      break;
+    if (local_messages.empty()) {
+      local_delivery_cond.Wait(local_delivery_lock);
+      continue;
+    }
+    pair<Message *, int> mp = local_messages.front();
+    local_messages.pop_front();
+    local_delivery_lock.Unlock();
+    Message *m = mp.first;
+    int priority = mp.second;
+    fast_preprocess(m);
+    if (can_fast_dispatch(m)) {
+      fast_dispatch(m);
+    } else {
+      Mutex::Locker l(lock);
+      add_arrival(m);
+      if (priority >= CEPH_MSG_PRIO_LOW) {
+        mqueue.enqueue_strict(
+            0, priority, QueueItem(m));
+      } else {
+        mqueue.enqueue(
+            0, priority, m->get_cost(), QueueItem(m));
+      }
+      cond.Signal();
+    }
+    local_delivery_lock.Lock();
+  }
+  local_delivery_lock.Unlock();
+}
+
+/*
+ * This function delivers incoming messages to the Messenger.
+ * Pipes with messages are kept in queues; when beginning a message
+ * delivery the highest-priority queue is selected, the pipe from the
+ * front of the queue is removed, and its message read. If the pipe
+ * has remaining messages at that priority level, it is re-placed on to the
+ * end of the queue. If the queue is empty; it's removed.
+ * The message is then delivered and the process starts again.
+ */
+void DispatchQueue::entry()
+{
+  lock.Lock();
+  while (true) {
+    while (!mqueue.empty()) {
+      QueueItem qitem = mqueue.dequeue();
+      if (!qitem.is_code())
+       remove_arrival(qitem.get_message());
+      lock.Unlock();
+
+      if (qitem.is_code()) {
+       switch (qitem.get_code()) {
+       case D_BAD_REMOTE_RESET:
+         msgr->ms_deliver_handle_remote_reset(qitem.get_connection());
+         break;
+       case D_CONNECT:
+         msgr->ms_deliver_handle_connect(qitem.get_connection());
+         break;
+       case D_ACCEPT:
+         msgr->ms_deliver_handle_accept(qitem.get_connection());
+         break;
+       case D_BAD_RESET:
+         msgr->ms_deliver_handle_reset(qitem.get_connection());
+         break;
+       default:
+         assert(0);
+       }
+      } else {
+       Message *m = qitem.get_message();
+       if (stop) {
+         ldout(cct,10) << " stop flag set, discarding " << m << " " << *m << dendl;
+         m->put();
+       } else {
+         uint64_t msize = pre_dispatch(m);
+         msgr->ms_deliver_dispatch(m);
+         post_dispatch(m, msize);
+       }
+      }
+
+      lock.Lock();
+    }
+    if (stop)
+      break;
+
+    // wait for something to be put on queue
+    cond.Wait(lock);
+  }
+  lock.Unlock();
+}
+
+void DispatchQueue::discard_queue(uint64_t id) {
+  Mutex::Locker l(lock);
+  list<QueueItem> removed;
+  mqueue.remove_by_class(id, &removed);
+  for (list<QueueItem>::iterator i = removed.begin();
+       i != removed.end();
+       ++i) {
+    assert(!(i->is_code())); // We don't discard id 0, ever!
+    Message *m = i->get_message();
+    remove_arrival(m);
+    msgr->dispatch_throttle_release(m->get_dispatch_throttle_size());
+    m->put();
+  }
+}
+
+void DispatchQueue::start()
+{
+  assert(!stop);
+  assert(!dispatch_thread.is_started());
+  dispatch_thread.create();
+  local_delivery_thread.create();
+}
+
+void DispatchQueue::wait()
+{
+  local_delivery_thread.join();
+  dispatch_thread.join();
+}
+
+void DispatchQueue::shutdown()
+{
+  // stop my local delivery thread
+  local_delivery_lock.Lock();
+  stop_local_delivery = true;
+  local_delivery_cond.Signal();
+  local_delivery_lock.Unlock();
+
+  // stop my dispatch thread
+  lock.Lock();
+  stop = true;
+  cond.Signal();
+  lock.Unlock();
+}
diff --git a/src/msg/simple/DispatchQueue.h b/src/msg/simple/DispatchQueue.h
new file mode 100644 (file)
index 0000000..5fe17dc
--- /dev/null
@@ -0,0 +1,208 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software 
+ * Foundation.  See file COPYING.
+ * 
+ */
+
+#ifndef CEPH_DISPATCHQUEUE_H
+#define CEPH_DISPATCHQUEUE_H
+
+#include <map>
+#include <boost/intrusive_ptr.hpp>
+#include "include/assert.h"
+#include "include/xlist.h"
+#include "include/atomic.h"
+#include "common/Mutex.h"
+#include "common/Cond.h"
+#include "common/Thread.h"
+#include "common/RefCountedObj.h"
+#include "common/PrioritizedQueue.h"
+
+class CephContext;
+class DispatchQueue;
+class Pipe;
+class SimpleMessenger;
+class Message;
+struct Connection;
+
+/**
+ * The DispatchQueue contains all the Pipes which have Messages
+ * they want to be dispatched, carefully organized by Message priority
+ * and permitted to deliver in a round-robin fashion.
+ * See SimpleMessenger::dispatch_entry for details.
+ */
+class DispatchQueue {
+  class QueueItem {
+    int type;
+    ConnectionRef con;
+    MessageRef m;
+  public:
+    QueueItem(Message *m) : type(-1), con(0), m(m) {}
+    QueueItem(int type, Connection *con) : type(type), con(con), m(0) {}
+    bool is_code() const {
+      return type != -1;
+    }
+    int get_code () {
+      assert(is_code());
+      return type;
+    }
+    Message *get_message() {
+      assert(!is_code());
+      return m.get();
+    }
+    Connection *get_connection() {
+      assert(is_code());
+      return con.get();
+    }
+  };
+    
+  CephContext *cct;
+  SimpleMessenger *msgr;
+  Mutex lock;
+  Cond cond;
+
+  PrioritizedQueue<QueueItem, uint64_t> mqueue;
+
+  set<pair<double, Message*> > marrival;
+  map<Message *, set<pair<double, Message*> >::iterator> marrival_map;
+  void add_arrival(Message *m) {
+    marrival_map.insert(
+      make_pair(
+       m,
+       marrival.insert(make_pair(m->get_recv_stamp(), m)).first
+       )
+      );
+  }
+  void remove_arrival(Message *m) {
+    map<Message *, set<pair<double, Message*> >::iterator>::iterator i =
+      marrival_map.find(m);
+    assert(i != marrival_map.end());
+    marrival.erase(i->second);
+    marrival_map.erase(i);
+  }
+
+  uint64_t next_pipe_id;
+    
+  enum { D_CONNECT = 1, D_ACCEPT, D_BAD_REMOTE_RESET, D_BAD_RESET, D_NUM_CODES };
+
+  /**
+   * The DispatchThread runs dispatch_entry to empty out the dispatch_queue.
+   */
+  class DispatchThread : public Thread {
+    DispatchQueue *dq;
+  public:
+    DispatchThread(DispatchQueue *dq) : dq(dq) {}
+    void *entry() {
+      dq->entry();
+      return 0;
+    }
+  } dispatch_thread;
+
+  Mutex local_delivery_lock;
+  Cond local_delivery_cond;
+  bool stop_local_delivery;
+  list<pair<Message *, int> > local_messages;
+  class LocalDeliveryThread : public Thread {
+    DispatchQueue *dq;
+  public:
+    LocalDeliveryThread(DispatchQueue *dq) : dq(dq) {}
+    void *entry() {
+      dq->run_local_delivery();
+      return 0;
+    }
+  } local_delivery_thread;
+
+  uint64_t pre_dispatch(Message *m);
+  void post_dispatch(Message *m, uint64_t msize);
+
+  public:
+  bool stop;
+  void local_delivery(Message *m, int priority);
+  void run_local_delivery();
+
+  double get_max_age(utime_t now);
+
+  int get_queue_len() {
+    Mutex::Locker l(lock);
+    return mqueue.length();
+  }
+    
+  void queue_connect(Connection *con) {
+    Mutex::Locker l(lock);
+    if (stop)
+      return;
+    mqueue.enqueue_strict(
+      0,
+      CEPH_MSG_PRIO_HIGHEST,
+      QueueItem(D_CONNECT, con));
+    cond.Signal();
+  }
+  void queue_accept(Connection *con) {
+    Mutex::Locker l(lock);
+    if (stop)
+      return;
+    mqueue.enqueue_strict(
+      0,
+      CEPH_MSG_PRIO_HIGHEST,
+      QueueItem(D_ACCEPT, con));
+    cond.Signal();
+  }
+  void queue_remote_reset(Connection *con) {
+    Mutex::Locker l(lock);
+    if (stop)
+      return;
+    mqueue.enqueue_strict(
+      0,
+      CEPH_MSG_PRIO_HIGHEST,
+      QueueItem(D_BAD_REMOTE_RESET, con));
+    cond.Signal();
+  }
+  void queue_reset(Connection *con) {
+    Mutex::Locker l(lock);
+    if (stop)
+      return;
+    mqueue.enqueue_strict(
+      0,
+      CEPH_MSG_PRIO_HIGHEST,
+      QueueItem(D_BAD_RESET, con));
+    cond.Signal();
+  }
+
+  bool can_fast_dispatch(Message *m);
+  void fast_dispatch(Message *m);
+  void fast_preprocess(Message *m);
+  void enqueue(Message *m, int priority, uint64_t id);
+  void discard_queue(uint64_t id);
+  uint64_t get_id() {
+    Mutex::Locker l(lock);
+    return next_pipe_id++;
+  }
+  void start();
+  void entry();
+  void wait();
+  void shutdown();
+  bool is_started() {return dispatch_thread.is_started();}
+
+  DispatchQueue(CephContext *cct, SimpleMessenger *msgr)
+    : cct(cct), msgr(msgr),
+      lock("SimpleMessenger::DispatchQeueu::lock"), 
+      mqueue(cct->_conf->ms_pq_max_tokens_per_priority,
+            cct->_conf->ms_pq_min_cost),
+      next_pipe_id(1),
+      dispatch_thread(this),
+      local_delivery_lock("SimpleMessenger::DispatchQueue::local_delivery_lock"),
+      stop_local_delivery(false),
+      local_delivery_thread(this),
+      stop(false)
+    {}
+};
+
+#endif
diff --git a/src/msg/simple/Pipe.cc b/src/msg/simple/Pipe.cc
new file mode 100644 (file)
index 0000000..4f38605
--- /dev/null
@@ -0,0 +1,2488 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software 
+ * Foundation.  See file COPYING.
+ * 
+ */
+
+#include <sys/socket.h>
+#include <netinet/tcp.h>
+#include <sys/uio.h>
+#include <limits.h>
+#include <poll.h>
+
+#include "msg/Message.h"
+#include "Pipe.h"
+#include "SimpleMessenger.h"
+
+#include "common/debug.h"
+#include "common/errno.h"
+
+// Below included to get encode_encrypt(); That probably should be in Crypto.h, instead
+
+#include "auth/Crypto.h"
+#include "auth/cephx/CephxProtocol.h"
+#include "auth/AuthSessionHandler.h"
+
+// Constant to limit starting sequence number to 2^31.  Nothing special about it, just a big number.  PLR
+#define SEQ_MASK  0x7fffffff 
+#define dout_subsys ceph_subsys_ms
+
+#undef dout_prefix
+#define dout_prefix _pipe_prefix(_dout)
+ostream& Pipe::_pipe_prefix(std::ostream *_dout) {
+  return *_dout << "-- " << msgr->get_myinst().addr << " >> " << peer_addr << " pipe(" << this
+               << " sd=" << sd << " :" << port
+               << " s=" << state
+               << " pgs=" << peer_global_seq
+               << " cs=" << connect_seq
+               << " l=" << policy.lossy
+               << " c=" << connection_state
+               << ").";
+}
+
+/*
+ * This optimization may not be available on all platforms (e.g. OSX).
+ * Apparently a similar approach based on TCP_CORK can be used.
+ */
+#ifndef MSG_MORE
+# define MSG_MORE 0
+#endif
+
+/*
+ * On BSD SO_NOSIGPIPE can be set via setsockopt to block SIGPIPE.
+ */
+#ifndef MSG_NOSIGNAL
+# define MSG_NOSIGNAL 0
+# ifdef SO_NOSIGPIPE
+#  define CEPH_USE_SO_NOSIGPIPE
+# else
+#  error "Cannot block SIGPIPE!"
+# endif
+#endif
+
+/**************************************
+ * Pipe
+ */
+
+Pipe::Pipe(SimpleMessenger *r, int st, PipeConnection *con)
+  : RefCountedObject(r->cct),
+    reader_thread(this),
+    writer_thread(this),
+    delay_thread(NULL),
+    msgr(r),
+    conn_id(r->dispatch_queue.get_id()),
+    recv_ofs(0),
+    recv_len(0),
+    sd(-1), port(0),
+    peer_type(-1),
+    pipe_lock("SimpleMessenger::Pipe::pipe_lock"),
+    state(st),
+    connection_state(NULL),
+    reader_running(false), reader_needs_join(false),
+    reader_dispatching(false),
+    writer_running(false),
+    in_q(&(r->dispatch_queue)),
+    send_keepalive(false),
+    send_keepalive_ack(false),
+    connect_seq(0), peer_global_seq(0),
+    out_seq(0), in_seq(0), in_seq_acked(0) {
+  if (con) {
+    connection_state = con;
+    connection_state->reset_pipe(this);
+  } else {
+    connection_state = new PipeConnection(msgr->cct, msgr);
+    connection_state->pipe = get();
+  }
+
+  if (randomize_out_seq()) {
+    lsubdout(msgr->cct,ms,15) << "Pipe(): Could not get random bytes to set seq number for session reset; set seq number to " << out_seq << dendl;
+  }
+    
+
+  msgr->timeout = msgr->cct->_conf->ms_tcp_read_timeout * 1000; //convert to ms
+  if (msgr->timeout == 0)
+    msgr->timeout = -1;
+
+  recv_max_prefetch = msgr->cct->_conf->ms_tcp_prefetch_max_size;
+  recv_buf = new char[recv_max_prefetch];
+}
+
+Pipe::~Pipe()
+{
+  assert(out_q.empty());
+  assert(sent.empty());
+  delete delay_thread;
+  delete[] recv_buf;
+}
+
+void Pipe::handle_ack(uint64_t seq)
+{
+  lsubdout(msgr->cct, ms, 15) << "reader got ack seq " << seq << dendl;
+  // trim sent list
+  while (!sent.empty() &&
+        sent.front()->get_seq() <= seq) {
+    Message *m = sent.front();
+    sent.pop_front();
+    lsubdout(msgr->cct, ms, 10) << "reader got ack seq "
+                               << seq << " >= " << m->get_seq() << " on " << m << " " << *m << dendl;
+    m->put();
+  }
+}
+
+void Pipe::start_reader()
+{
+  assert(pipe_lock.is_locked());
+  assert(!reader_running);
+  if (reader_needs_join) {
+    reader_thread.join();
+    reader_needs_join = false;
+  }
+  reader_running = true;
+  reader_thread.create(msgr->cct->_conf->ms_rwthread_stack_bytes);
+}
+
+void Pipe::maybe_start_delay_thread()
+{
+  if (!delay_thread &&
+      msgr->cct->_conf->ms_inject_delay_type.find(ceph_entity_type_name(connection_state->peer_type)) != string::npos) {
+    lsubdout(msgr->cct, ms, 1) << "setting up a delay queue on Pipe " << this << dendl;
+    delay_thread = new DelayedDelivery(this);
+    delay_thread->create();
+  }
+}
+
+void Pipe::start_writer()
+{
+  assert(pipe_lock.is_locked());
+  assert(!writer_running);
+  writer_running = true;
+  writer_thread.create(msgr->cct->_conf->ms_rwthread_stack_bytes);
+}
+
+void Pipe::join_reader()
+{
+  if (!reader_running)
+    return;
+  cond.Signal();
+  pipe_lock.Unlock();
+  reader_thread.join();
+  pipe_lock.Lock();
+  reader_needs_join = false;
+}
+
+void Pipe::DelayedDelivery::discard()
+{
+  lgeneric_subdout(pipe->msgr->cct, ms, 20) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::discard" << dendl;
+  Mutex::Locker l(delay_lock);
+  while (!delay_queue.empty()) {
+    Message *m = delay_queue.front().second;
+    pipe->msgr->dispatch_throttle_release(m->get_dispatch_throttle_size());
+    m->put();
+    delay_queue.pop_front();
+  }
+}
+
+void Pipe::DelayedDelivery::flush()
+{
+  lgeneric_subdout(pipe->msgr->cct, ms, 20) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::flush" << dendl;
+  Mutex::Locker l(delay_lock);
+  flush_count = delay_queue.size();
+  delay_cond.Signal();
+}
+
+void *Pipe::DelayedDelivery::entry()
+{
+  Mutex::Locker locker(delay_lock);
+  lgeneric_subdout(pipe->msgr->cct, ms, 20) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::entry start" << dendl;
+
+  while (!stop_delayed_delivery) {
+    if (delay_queue.empty()) {
+      lgeneric_subdout(pipe->msgr->cct, ms, 30) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::entry sleeping on delay_cond because delay queue is empty" << dendl;
+      delay_cond.Wait(delay_lock);
+      continue;
+    }
+    utime_t release = delay_queue.front().first;
+    Message *m = delay_queue.front().second;
+    string delay_msg_type = pipe->msgr->cct->_conf->ms_inject_delay_msg_type;
+    if (!flush_count &&
+        (release > ceph_clock_now(pipe->msgr->cct) &&
+         (delay_msg_type.empty() || m->get_type_name() == delay_msg_type))) {
+      lgeneric_subdout(pipe->msgr->cct, ms, 10) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::entry sleeping on delay_cond until " << release << dendl;
+      delay_cond.WaitUntil(delay_lock, release);
+      continue;
+    }
+    lgeneric_subdout(pipe->msgr->cct, ms, 10) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::entry dequeuing message " << m << " for delivery, past " << release << dendl;
+    delay_queue.pop_front();
+    if (flush_count > 0) {
+      --flush_count;
+      active_flush = true;
+    }
+    if (pipe->in_q->can_fast_dispatch(m)) {
+      if (!stop_fast_dispatching_flag) {
+        delay_dispatching = true;
+        delay_lock.Unlock();
+        pipe->in_q->fast_dispatch(m);
+        delay_lock.Lock();
+        delay_dispatching = false;
+        if (stop_fast_dispatching_flag) {
+          // we need to let the stopping thread proceed
+          delay_cond.Signal();
+          delay_lock.Unlock();
+          delay_lock.Lock();
+        }
+      }
+    } else {
+      pipe->in_q->enqueue(m, m->get_priority(), pipe->conn_id);
+    }
+    active_flush = false;
+  }
+  lgeneric_subdout(pipe->msgr->cct, ms, 20) << pipe->_pipe_prefix(_dout) << "DelayedDelivery::entry stop" << dendl;
+  return NULL;
+}
+
+void Pipe::DelayedDelivery::stop_fast_dispatching() {
+  Mutex::Locker l(delay_lock);
+  stop_fast_dispatching_flag = true;
+  // we can't block if we're the delay thread; see Pipe::stop_and_wait()
+  while (delay_dispatching && !am_self())
+    delay_cond.Wait(delay_lock);
+}
+
+
+int Pipe::accept()
+{
+  ldout(msgr->cct,10) << "accept" << dendl;
+  assert(pipe_lock.is_locked());
+  assert(state == STATE_ACCEPTING);
+
+  pipe_lock.Unlock();
+
+  // vars
+  bufferlist addrs;
+  entity_addr_t socket_addr;
+  socklen_t len;
+  int r;
+  char banner[strlen(CEPH_BANNER)+1];
+  bufferlist addrbl;
+  ceph_msg_connect connect;
+  ceph_msg_connect_reply reply;
+  Pipe *existing = 0;
+  bufferptr bp;
+  bufferlist authorizer, authorizer_reply;
+  bool authorizer_valid;
+  uint64_t feat_missing;
+  bool replaced = false;
+  // this variable denotes if the connection attempt from peer is a hard 
+  // reset or not, it is true if there is an existing connection and the
+  // connection sequence from peer is equal to zero
+  bool is_reset_from_peer = false;
+  CryptoKey session_key;
+  int removed; // single-use down below
+
+  // this should roughly mirror pseudocode at
+  //  http://ceph.newdream.net/wiki/Messaging_protocol
+  int reply_tag = 0;
+  uint64_t existing_seq = -1;
+
+  // used for reading in the remote acked seq on connect
+  uint64_t newly_acked_seq = 0;
+
+  recv_reset();
+
+  set_socket_options();
+
+  // announce myself.
+  r = tcp_write(CEPH_BANNER, strlen(CEPH_BANNER));
+  if (r < 0) {
+    ldout(msgr->cct,10) << "accept couldn't write banner" << dendl;
+    goto fail_unlocked;
+  }
+
+  // and my addr
+  ::encode(msgr->my_inst.addr, addrs);
+
+  port = msgr->my_inst.addr.get_port();
+
+  // and peer's socket addr (they might not know their ip)
+  len = sizeof(socket_addr.ss_addr());
+  r = ::getpeername(sd, (sockaddr*)&socket_addr.ss_addr(), &len);
+  if (r < 0) {
+    ldout(msgr->cct,0) << "accept failed to getpeername " << cpp_strerror(errno) << dendl;
+    goto fail_unlocked;
+  }
+  ::encode(socket_addr, addrs);
+
+  r = tcp_write(addrs.c_str(), addrs.length());
+  if (r < 0) {
+    ldout(msgr->cct,10) << "accept couldn't write my+peer addr" << dendl;
+    goto fail_unlocked;
+  }
+
+  ldout(msgr->cct,1) << "accept sd=" << sd << " " << socket_addr << dendl;
+  
+  // identify peer
+  if (tcp_read(banner, strlen(CEPH_BANNER)) < 0) {
+    ldout(msgr->cct,10) << "accept couldn't read banner" << dendl;
+    goto fail_unlocked;
+  }
+  if (memcmp(banner, CEPH_BANNER, strlen(CEPH_BANNER))) {
+    banner[strlen(CEPH_BANNER)] = 0;
+    ldout(msgr->cct,1) << "accept peer sent bad banner '" << banner << "' (should be '" << CEPH_BANNER << "')" << dendl;
+    goto fail_unlocked;
+  }
+  {
+    bufferptr tp(sizeof(peer_addr));
+    addrbl.push_back(tp);
+  }
+  if (tcp_read(addrbl.c_str(), addrbl.length()) < 0) {
+    ldout(msgr->cct,10) << "accept couldn't read peer_addr" << dendl;
+    goto fail_unlocked;
+  }
+  {
+    bufferlist::iterator ti = addrbl.begin();
+    ::decode(peer_addr, ti);
+  }
+
+  ldout(msgr->cct,10) << "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.addr = socket_addr.addr;
+    peer_addr.set_port(port);
+    ldout(msgr->cct,0) << "accept peer addr is really " << peer_addr
+           << " (socket is " << socket_addr << ")" << dendl;
+  }
+  set_peer_addr(peer_addr);  // so that connection_state gets set up
+  
+  while (1) {
+    if (tcp_read((char*)&connect, sizeof(connect)) < 0) {
+      ldout(msgr->cct,10) << "accept couldn't read connect" << dendl;
+      goto fail_unlocked;
+    }
+
+    // sanitize features
+    connect.features = ceph_sanitize_features(connect.features);
+
+    authorizer.clear();
+    if (connect.authorizer_len) {
+      bp = buffer::create(connect.authorizer_len);
+      if (tcp_read(bp.c_str(), connect.authorizer_len) < 0) {
+        ldout(msgr->cct,10) << "accept couldn't read connect authorizer" << dendl;
+        goto fail_unlocked;
+      }
+      authorizer.push_back(bp);
+      authorizer_reply.clear();
+    }
+
+    ldout(msgr->cct,20) << "accept got peer connect_seq " << connect.connect_seq
+            << " global_seq " << connect.global_seq
+            << dendl;
+    
+    msgr->lock.Lock();   // FIXME
+    pipe_lock.Lock();
+    if (msgr->dispatch_queue.stop)
+      goto shutting_down;
+    if (state != STATE_ACCEPTING) {
+      goto shutting_down;
+    }
+
+    // note peer's type, flags
+    set_peer_type(connect.host_type);
+    policy = msgr->get_policy(connect.host_type);
+    ldout(msgr->cct,10) << "accept of host_type " << connect.host_type
+                       << ", policy.lossy=" << policy.lossy
+                       << " policy.server=" << policy.server
+                       << " policy.standby=" << policy.standby
+                       << " policy.resetcheck=" << policy.resetcheck
+                       << dendl;
+
+    memset(&reply, 0, sizeof(reply));
+    reply.protocol_version = msgr->get_proto_version(peer_type, false);
+    msgr->lock.Unlock();
+
+    // mismatch?
+    ldout(msgr->cct,10) << "accept my proto " << reply.protocol_version
+            << ", their proto " << connect.protocol_version << dendl;
+    if (connect.protocol_version != reply.protocol_version) {
+      reply.tag = CEPH_MSGR_TAG_BADPROTOVER;
+      goto 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) {
+       if (msgr->cct->_conf->cephx_require_signatures ||
+           msgr->cct->_conf->cephx_cluster_require_signatures) {
+         ldout(msgr->cct,10) << "using cephx, requiring MSG_AUTH feature bit for cluster" << dendl;
+         policy.features_required |= CEPH_FEATURE_MSG_AUTH;
+       }
+      } else {
+       if (msgr->cct->_conf->cephx_require_signatures ||
+           msgr->cct->_conf->cephx_service_require_signatures) {
+         ldout(msgr->cct,10) << "using cephx, requiring MSG_AUTH feature bit for service" << dendl;
+         policy.features_required |= CEPH_FEATURE_MSG_AUTH;
+       }
+      }
+    }
+
+    feat_missing = policy.features_required & ~(uint64_t)connect.features;
+    if (feat_missing) {
+      ldout(msgr->cct,1) << "peer missing required features " << std::hex << feat_missing << std::dec << dendl;
+      reply.tag = CEPH_MSGR_TAG_FEATURES;
+      goto reply;
+    }
+    
+    // Check the authorizer.  If not good, bail out.
+
+    pipe_lock.Unlock();
+
+    if (!msgr->verify_authorizer(connection_state.get(), peer_type, connect.authorizer_protocol, authorizer,
+                                authorizer_reply, authorizer_valid, session_key) ||
+       !authorizer_valid) {
+      ldout(msgr->cct,0) << "accept: got bad authorizer" << dendl;
+      pipe_lock.Lock();
+      if (state != STATE_ACCEPTING)
+       goto shutting_down_msgr_unlocked;
+      reply.tag = CEPH_MSGR_TAG_BADAUTHORIZER;
+      session_security.reset();
+      goto reply;
+    } 
+
+    // We've verified the authorizer for this pipe, so set up the session security structure.  PLR
+
+    ldout(msgr->cct,10) << "accept:  setting up session_security." << dendl;
+
+    msgr->lock.Lock();
+    pipe_lock.Lock();
+    if (msgr->dispatch_queue.stop)
+      goto shutting_down;
+    if (state != STATE_ACCEPTING)
+      goto shutting_down;
+    
+    // existing?
+    existing = msgr->_lookup_pipe(peer_addr);
+    if (existing) {
+      existing->pipe_lock.Lock(true);  // skip lockdep check (we are locking a second Pipe here)
+
+      if (connect.global_seq < existing->peer_global_seq) {
+       ldout(msgr->cct,10) << "accept existing " << existing << ".gseq " << existing->peer_global_seq
+                << " > " << connect.global_seq << ", RETRY_GLOBAL" << dendl;
+       reply.tag = CEPH_MSGR_TAG_RETRY_GLOBAL;
+       reply.global_seq = existing->peer_global_seq;  // so we can send it below..
+       existing->pipe_lock.Unlock();
+       msgr->lock.Unlock();
+       goto reply;
+      } else {
+       ldout(msgr->cct,10) << "accept existing " << existing << ".gseq " << existing->peer_global_seq
+                << " <= " << connect.global_seq << ", looks ok" << dendl;
+      }
+      
+      if (existing->policy.lossy) {
+       ldout(msgr->cct,0) << "accept replacing existing (lossy) channel (new one lossy="
+               << policy.lossy << ")" << dendl;
+       existing->was_session_reset();
+       goto replace;
+      }
+
+      ldout(msgr->cct,0) << "accept connect_seq " << connect.connect_seq
+                        << " vs existing " << existing->connect_seq
+                        << " state " << existing->get_state_name() << dendl;
+
+      if (connect.connect_seq == 0 && existing->connect_seq > 0) {
+       ldout(msgr->cct,0) << "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;
+      }
+
+      if (connect.connect_seq < existing->connect_seq) {
+       // old attempt, or we sent READY but they didn't get it.
+       ldout(msgr->cct,10) << "accept existing " << existing << ".cseq " << existing->connect_seq
+                           << " > " << connect.connect_seq << ", RETRY_SESSION" << dendl;
+       goto retry_session;
+      }
+
+      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(msgr->cct,10) << "accept connection race, existing " << existing
+                             << ".cseq " << existing->connect_seq
+                             << " == " << connect.connect_seq
+                             << ", OPEN|STANDBY, RETRY_SESSION" << dendl;
+         goto retry_session;
+       }
+
+       // connection race?
+       if (peer_addr < msgr->my_inst.addr ||
+           existing->policy.server) {
+         // incoming wins
+         ldout(msgr->cct,10) << "accept connection race, existing " << existing << ".cseq " << existing->connect_seq
+                  << " == " << connect.connect_seq << ", or we are server, replacing my attempt" << dendl;
+         if (!(existing->state == STATE_CONNECTING ||
+               existing->state == STATE_WAIT))
+           lderr(msgr->cct) << "accept race bad state, would replace, existing="
+                            << existing->get_state_name()
+                            << " " << existing << ".cseq=" << existing->connect_seq
+                            << " == " << connect.connect_seq
+                            << dendl;
+         assert(existing->state == STATE_CONNECTING ||
+                existing->state == STATE_WAIT);
+         goto replace;
+       } else {
+         // our existing outgoing wins
+         ldout(msgr->cct,10) << "accept connection race, existing " << existing << ".cseq " << existing->connect_seq
+                  << " == " << connect.connect_seq << ", sending WAIT" << dendl;
+         assert(peer_addr > msgr->my_inst.addr);
+         if (!(existing->state == STATE_CONNECTING))
+           lderr(msgr->cct) << "accept race bad state, would send wait, existing="
+                            << existing->get_state_name()
+                            << " " << existing << ".cseq=" << existing->connect_seq
+                            << " == " << connect.connect_seq
+                            << dendl;
+         assert(existing->state == STATE_CONNECTING);
+         // make sure our outgoing connection will follow through
+         existing->_send_keepalive();
+         reply.tag = CEPH_MSGR_TAG_WAIT;
+         existing->pipe_lock.Unlock();
+         msgr->lock.Unlock();
+         goto reply;
+       }
+      }
+
+      assert(connect.connect_seq > existing->connect_seq);
+      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(msgr->cct,0) << "accept we reset (peer sent cseq " << connect.connect_seq 
+                << ", " << existing << ".cseq = " << existing->connect_seq
+                << "), sending RESETSESSION" << dendl;
+       reply.tag = CEPH_MSGR_TAG_RESETSESSION;
+       msgr->lock.Unlock();
+       existing->pipe_lock.Unlock();
+       goto reply;
+      }
+
+      // reconnect
+      ldout(msgr->cct,10) << "accept peer sent cseq " << connect.connect_seq
+              << " > " << existing->connect_seq << dendl;
+      goto replace;
+    } // existing
+    else if (policy.resetcheck && connect.connect_seq > 0) {
+      // we reset, and they are opening a new session
+      ldout(msgr->cct,0) << "accept we reset (peer sent cseq " << connect.connect_seq << "), sending RESETSESSION" << dendl;
+      msgr->lock.Unlock();
+      reply.tag = CEPH_MSGR_TAG_RESETSESSION;
+      goto reply;
+    } else {
+      // new session
+      ldout(msgr->cct,10) << "accept new session" << dendl;
+      existing = NULL;
+      goto open;
+    }
+    assert(0);
+
+  retry_session:
+    assert(existing->pipe_lock.is_locked());
+    assert(pipe_lock.is_locked());
+    reply.tag = CEPH_MSGR_TAG_RETRY_SESSION;
+    reply.connect_seq = existing->connect_seq + 1;
+    existing->pipe_lock.Unlock();
+    msgr->lock.Unlock();
+    goto reply;    
+
+  reply:
+    assert(pipe_lock.is_locked());
+    reply.features = ((uint64_t)connect.features & policy.features_supported) | policy.features_required;
+    reply.authorizer_len = authorizer_reply.length();
+    pipe_lock.Unlock();
+    r = tcp_write((char*)&reply, sizeof(reply));
+    if (r < 0)
+      goto fail_unlocked;
+    if (reply.authorizer_len) {
+      r = tcp_write(authorizer_reply.c_str(), authorizer_reply.length());
+      if (r < 0)
+       goto fail_unlocked;
+    }
+  }
+  
+ replace:
+  assert(existing->pipe_lock.is_locked());
+  assert(pipe_lock.is_locked());
+  // 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;
+    existing_seq = existing->in_seq;
+  }
+  ldout(msgr->cct,10) << "accept replacing " << existing << dendl;
+  existing->stop();
+  existing->unregister_pipe();
+  replaced = true;
+
+  if (existing->policy.lossy) {
+    // disconnect from the Connection
+    assert(existing->connection_state);
+    if (existing->connection_state->clear_pipe(existing))
+      msgr->dispatch_queue.queue_reset(existing->connection_state.get());
+  } else {
+    // queue a reset on the new connection, which we're dumping for the old
+    msgr->dispatch_queue.queue_reset(connection_state.get());
+
+    // drop my Connection, and take a ref to the existing one. do not
+    // clear existing->connection_state, since read_message and
+    // write_message both dereference it without pipe_lock.
+    connection_state = existing->connection_state;
+
+    // make existing Connection reference us
+    connection_state->reset_pipe(this);
+
+    if (existing->delay_thread) {
+      existing->delay_thread->steal_for_pipe(this);
+      delay_thread = existing->delay_thread;
+      existing->delay_thread = NULL;
+      delay_thread->flush();
+    }
+
+    // steal incoming queue
+    uint64_t replaced_conn_id = conn_id;
+    conn_id = existing->conn_id;
+    existing->conn_id = replaced_conn_id;
+
+    // reset the in_seq if this is a hard reset from peer,
+    // otherwise we respect our original connection's value
+    in_seq = is_reset_from_peer ? 0 : existing->in_seq;
+    in_seq_acked = in_seq;
+
+    // steal outgoing queue and out_seq
+    existing->requeue_sent();
+    out_seq = existing->out_seq;
+    ldout(msgr->cct,10) << "accept re-queuing on out_seq " << out_seq << " in_seq " << in_seq << dendl;
+    for (map<int, list<Message*> >::iterator p = existing->out_q.begin();
+         p != existing->out_q.end();
+         ++p)
+      out_q[p->first].splice(out_q[p->first].begin(), p->second);
+  }
+  existing->stop_and_wait();
+  existing->pipe_lock.Unlock();
+
+ open:
+  // open
+  assert(pipe_lock.is_locked());
+  connect_seq = connect.connect_seq + 1;
+  peer_global_seq = connect.global_seq;
+  assert(state == STATE_ACCEPTING);
+  state = STATE_OPEN;
+  ldout(msgr->cct,10) << "accept success, connect_seq = " << connect_seq << ", sending READY" << dendl;
+
+  // send READY reply
+  reply.tag = (reply_tag ? reply_tag : CEPH_MSGR_TAG_READY);
+  reply.features = policy.features_supported;
+  reply.global_seq = 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;
+
+  connection_state->set_features((uint64_t)reply.features & (uint64_t)connect.features);
+  ldout(msgr->cct,10) << "accept features " << connection_state->get_features() << dendl;
+
+  session_security.reset(
+      get_auth_session_handler(msgr->cct,
+                              connect.authorizer_protocol,
+                              session_key,
+                              connection_state->get_features()));
+
+  // notify
+  msgr->dispatch_queue.queue_accept(connection_state.get());
+  msgr->ms_deliver_handle_fast_accept(connection_state.get());
+
+  // ok!
+  if (msgr->dispatch_queue.stop)
+    goto shutting_down;
+  removed = msgr->accepting_pipes.erase(this);
+  assert(removed == 1);
+  register_pipe();
+  msgr->lock.Unlock();
+  pipe_lock.Unlock();
+
+  r = tcp_write((char*)&reply, sizeof(reply));
+  if (r < 0) {
+    goto fail_registered;
+  }
+
+  if (reply.authorizer_len) {
+    r = tcp_write(authorizer_reply.c_str(), authorizer_reply.length());
+    if (r < 0) {
+      goto fail_registered;
+    }
+  }
+
+  if (reply_tag == CEPH_MSGR_TAG_SEQ) {
+    if (tcp_write((char*)&existing_seq, sizeof(existing_seq)) < 0) {
+      ldout(msgr->cct,2) << "accept write error on in_seq" << dendl;
+      goto fail_registered;
+    }
+    if (tcp_read((char*)&newly_acked_seq, sizeof(newly_acked_seq)) < 0) {
+      ldout(msgr->cct,2) << "accept read error on newly_acked_seq" << dendl;
+      goto fail_registered;
+    }
+  }
+
+  pipe_lock.Lock();
+  discard_requeued_up_to(newly_acked_seq);
+  if (state != STATE_CLOSED) {
+    ldout(msgr->cct,10) << "accept starting writer, state " << get_state_name() << dendl;
+    start_writer();
+  }
+  ldout(msgr->cct,20) << "accept done" << dendl;
+
+  maybe_start_delay_thread();
+
+  return 0;   // success.
+
+ fail_registered:
+  ldout(msgr->cct, 10) << "accept fault after register" << dendl;
+
+  if (msgr->cct->_conf->ms_inject_internal_delays) {
+    ldout(msgr->cct, 10) << " sleep for " << msgr->cct->_conf->ms_inject_internal_delays << dendl;
+    utime_t t;
+    t.set_from_double(msgr->cct->_conf->ms_inject_internal_delays);
+    t.sleep();
+  }
+
+ fail_unlocked:
+  pipe_lock.Lock();
+  if (state != STATE_CLOSED) {
+    bool queued = is_queued();
+    ldout(msgr->cct, 10) << "  queued = " << (int)queued << dendl;
+    if (queued) {
+      state = policy.server ? STATE_STANDBY : STATE_CONNECTING;
+    } else if (replaced) {
+      state = STATE_STANDBY;
+    } else {
+      state = STATE_CLOSED;
+      state_closed.set(1);
+    }
+    fault();
+    if (queued || replaced)
+      start_writer();
+  }
+  return -1;
+
+ shutting_down:
+  msgr->lock.Unlock();
+ shutting_down_msgr_unlocked:
+  assert(pipe_lock.is_locked());
+
+  if (msgr->cct->_conf->ms_inject_internal_delays) {
+    ldout(msgr->cct, 10) << " sleep for " << msgr->cct->_conf->ms_inject_internal_delays << dendl;
+    utime_t t;
+    t.set_from_double(msgr->cct->_conf->ms_inject_internal_delays);
+    t.sleep();
+  }
+
+  state = STATE_CLOSED;
+  state_closed.set(1);
+  fault();
+  return -1;
+}
+
+void Pipe::set_socket_options()
+{
+  // disable Nagle algorithm?
+  if (msgr->cct->_conf->ms_tcp_nodelay) {
+    int flag = 1;
+    int r = ::setsockopt(sd, IPPROTO_TCP, TCP_NODELAY, (char*)&flag, sizeof(flag));
+    if (r < 0) {
+      r = -errno;
+      ldout(msgr->cct,0) << "couldn't set TCP_NODELAY: " << cpp_strerror(r) << dendl;
+    }
+  }
+  if (msgr->cct->_conf->ms_tcp_rcvbuf) {
+    int size = msgr->cct->_conf->ms_tcp_rcvbuf;
+    int r = ::setsockopt(sd, SOL_SOCKET, SO_RCVBUF, (void*)&size, sizeof(size));
+    if (r < 0)  {
+      r = -errno;
+      ldout(msgr->cct,0) << "couldn't set SO_RCVBUF to " << size << ": " << cpp_strerror(r) << dendl;
+    }
+  }
+
+  // block ESIGPIPE
+#ifdef CEPH_USE_SO_NOSIGPIPE
+  int val = 1;
+  int r = ::setsockopt(sd, SOL_SOCKET, SO_NOSIGPIPE, (void*)&val, sizeof(val));
+  if (r) {
+    r = -errno;
+    ldout(msgr->cct,0) << "couldn't set SO_NOSIGPIPE: " << cpp_strerror(r) << dendl;
+  }
+#endif
+}
+
+int Pipe::connect()
+{
+  bool got_bad_auth = false;
+
+  ldout(msgr->cct,10) << "connect " << connect_seq << dendl;
+  assert(pipe_lock.is_locked());
+
+  __u32 cseq = connect_seq;
+  __u32 gseq = msgr->get_global_seq();
+
+  // stop reader thrad
+  join_reader();
+
+  pipe_lock.Unlock();
+  
+  char tag = -1;
+  int rc;
+  struct msghdr msg;
+  struct iovec msgvec[2];
+  int msglen;
+  char banner[strlen(CEPH_BANNER) + 1];  // extra byte makes coverity happy
+  entity_addr_t paddr;
+  entity_addr_t peer_addr_for_me, socket_addr;
+  AuthAuthorizer *authorizer = NULL;
+  bufferlist addrbl, myaddrbl;
+  const md_config_t *conf = msgr->cct->_conf;
+
+  // close old socket.  this is safe because we stopped the reader thread above.
+  if (sd >= 0)
+    ::close(sd);
+
+  // create socket?
+  sd = ::socket(peer_addr.get_family(), SOCK_STREAM, 0);
+  if (sd < 0) {
+    lderr(msgr->cct) << "connect couldn't created socket " << cpp_strerror(errno) << dendl;
+    goto fail;
+  }
+
+  recv_reset();
+  // connect!
+  ldout(msgr->cct,10) << "connecting to " << peer_addr << dendl;
+  rc = ::connect(sd, (sockaddr*)&peer_addr.addr, peer_addr.addr_size());
+  if (rc < 0) {
+    ldout(msgr->cct,2) << "connect error " << peer_addr
+            << ", " << cpp_strerror(errno) << dendl;
+    goto fail;
+  }
+
+  set_socket_options();
+
+  // verify banner
+  // FIXME: this should be non-blocking, or in some other way verify the banner as we get it.
+  if (tcp_read((char*)&banner, strlen(CEPH_BANNER)) < 0) {
+    ldout(msgr->cct,2) << "connect couldn't read banner, " << cpp_strerror(errno) << dendl;
+    goto fail;
+  }
+  if (memcmp(banner, CEPH_BANNER, strlen(CEPH_BANNER))) {
+    ldout(msgr->cct,0) << "connect protocol error (bad banner) on peer " << peer_addr << dendl;
+    goto fail;
+  }
+
+  memset(&msg, 0, sizeof(msg));
+  msgvec[0].iov_base = banner;
+  msgvec[0].iov_len = strlen(CEPH_BANNER);
+  msg.msg_iov = msgvec;
+  msg.msg_iovlen = 1;
+  msglen = msgvec[0].iov_len;
+  if (do_sendmsg(&msg, msglen)) {
+    ldout(msgr->cct,2) << "connect couldn't write my banner, " << cpp_strerror(errno) << dendl;
+    goto fail;
+  }
+
+  // identify peer
+  {
+    bufferptr p(sizeof(paddr) * 2);
+    addrbl.push_back(p);
+  }
+  if (tcp_read(addrbl.c_str(), addrbl.length()) < 0) {
+    ldout(msgr->cct,2) << "connect couldn't read peer addrs, " << cpp_strerror(errno) << dendl;
+    goto fail;
+  }
+  {
+    bufferlist::iterator p = addrbl.begin();
+    ::decode(paddr, p);
+    ::decode(peer_addr_for_me, p);
+    port = peer_addr_for_me.get_port();
+  }
+
+  ldout(msgr->cct,20) << "connect read peer addr " << paddr << " on socket " << sd << dendl;
+  if (peer_addr != paddr) {
+    if (paddr.is_blank_ip() &&
+       peer_addr.get_port() == paddr.get_port() &&
+       peer_addr.get_nonce() == paddr.get_nonce()) {
+      ldout(msgr->cct,0) << "connect claims to be " 
+             << paddr << " not " << peer_addr << " - presumably this is the same node!" << dendl;
+    } else {
+      ldout(msgr->cct,0) << "connect claims to be " 
+             << paddr << " not " << peer_addr << " - wrong node!" << dendl;
+      goto fail;
+    }
+  }
+
+  ldout(msgr->cct,20) << "connect peer addr for me is " << peer_addr_for_me << dendl;
+
+  msgr->learned_addr(peer_addr_for_me);
+
+  ::encode(msgr->my_inst.addr, myaddrbl);
+
+  memset(&msg, 0, sizeof(msg));
+  msgvec[0].iov_base = myaddrbl.c_str();
+  msgvec[0].iov_len = myaddrbl.length();
+  msg.msg_iov = msgvec;
+  msg.msg_iovlen = 1;
+  msglen = msgvec[0].iov_len;
+  if (do_sendmsg(&msg, msglen)) {
+    ldout(msgr->cct,2) << "connect couldn't write my addr, " << cpp_strerror(errno) << dendl;
+    goto fail;
+  }
+  ldout(msgr->cct,10) << "connect sent my addr " << msgr->my_inst.addr << dendl;
+
+
+  while (1) {
+    delete authorizer;
+    authorizer = msgr->get_authorizer(peer_type, false);
+    bufferlist authorizer_reply;
+
+    ceph_msg_connect connect;
+    connect.features = policy.features_supported;
+    connect.host_type = msgr->get_myinst().name.type();
+    connect.global_seq = gseq;
+    connect.connect_seq = cseq;
+    connect.protocol_version = msgr->get_proto_version(peer_type, true);
+    connect.authorizer_protocol = authorizer ? authorizer->protocol : 0;
+    connect.authorizer_len = authorizer ? authorizer->bl.length() : 0;
+    if (authorizer) 
+      ldout(msgr->cct,10) << "connect.authorizer_len=" << connect.authorizer_len
+              << " protocol=" << connect.authorizer_protocol << dendl;
+    connect.flags = 0;
+    if (policy.lossy)
+      connect.flags |= CEPH_MSG_CONNECT_LOSSY;  // this is fyi, actually, server decides!
+    memset(&msg, 0, sizeof(msg));
+    msgvec[0].iov_base = (char*)&connect;
+    msgvec[0].iov_len = sizeof(connect);
+    msg.msg_iov = msgvec;
+    msg.msg_iovlen = 1;
+    msglen = msgvec[0].iov_len;
+    if (authorizer) {
+      msgvec[1].iov_base = authorizer->bl.c_str();
+      msgvec[1].iov_len = authorizer->bl.length();
+      msg.msg_iovlen++;
+      msglen += msgvec[1].iov_len;
+    }
+
+    ldout(msgr->cct,10) << "connect sending gseq=" << gseq << " cseq=" << cseq
+            << " proto=" << connect.protocol_version << dendl;
+    if (do_sendmsg(&msg, msglen)) {
+      ldout(msgr->cct,2) << "connect couldn't write gseq, cseq, " << cpp_strerror(errno) << dendl;
+      goto fail;
+    }
+
+    ldout(msgr->cct,20) << "connect wrote (self +) cseq, waiting for reply" << dendl;
+    ceph_msg_connect_reply reply;
+    if (tcp_read((char*)&reply, sizeof(reply)) < 0) {
+      ldout(msgr->cct,2) << "connect read reply " << cpp_strerror(errno) << dendl;
+      goto fail;
+    }
+
+    // sanitize features
+    reply.features = ceph_sanitize_features(reply.features);
+
+    ldout(msgr->cct,20) << "connect got reply tag " << (int)reply.tag
+                       << " connect_seq " << reply.connect_seq
+                       << " global_seq " << reply.global_seq
+                       << " proto " << reply.protocol_version
+                       << " flags " << (int)reply.flags
+                       << " features " << reply.features
+                       << dendl;
+
+    authorizer_reply.clear();
+
+    if (reply.authorizer_len) {
+      ldout(msgr->cct,10) << "reply.authorizer_len=" << reply.authorizer_len << dendl;
+      bufferptr bp = buffer::create(reply.authorizer_len);
+      if (tcp_read(bp.c_str(), reply.authorizer_len) < 0) {
+        ldout(msgr->cct,10) << "connect couldn't read connect authorizer_reply" << dendl;
+       goto fail;
+      }
+      authorizer_reply.push_back(bp);
+    }
+
+    if (authorizer) {
+      bufferlist::iterator iter = authorizer_reply.begin();
+      if (!authorizer->verify_reply(iter)) {
+        ldout(msgr->cct,0) << "failed verifying authorize reply" << dendl;
+       goto fail;
+      }
+    }
+
+    if (conf->ms_inject_internal_delays) {
+      ldout(msgr->cct, 10) << " sleep for " << msgr->cct->_conf->ms_inject_internal_delays << dendl;
+      utime_t t;
+      t.set_from_double(msgr->cct->_conf->ms_inject_internal_delays);
+      t.sleep();
+    }
+
+    pipe_lock.Lock();
+    if (state != STATE_CONNECTING) {
+      ldout(msgr->cct,0) << "connect got RESETSESSION but no longer connecting" << dendl;
+      goto stop_locked;
+    }
+
+    if (reply.tag == CEPH_MSGR_TAG_FEATURES) {
+      ldout(msgr->cct,0) << "connect protocol feature mismatch, my " << std::hex
+             << connect.features << " < peer " << reply.features
+             << " missing " << (reply.features & ~policy.features_supported)
+             << std::dec << dendl;
+      goto fail_locked;
+    }
+
+    if (reply.tag == CEPH_MSGR_TAG_BADPROTOVER) {
+      ldout(msgr->cct,0) << "connect protocol version mismatch, my " << connect.protocol_version
+             << " != " << reply.protocol_version << dendl;
+      goto fail_locked;
+    }
+
+    if (reply.tag == CEPH_MSGR_TAG_BADAUTHORIZER) {
+      ldout(msgr->cct,0) << "connect got BADAUTHORIZER" << dendl;
+      if (got_bad_auth)
+        goto stop_locked;
+      got_bad_auth = true;
+      pipe_lock.Unlock();
+      delete authorizer;
+      authorizer = msgr->get_authorizer(peer_type, true);  // try harder
+      continue;
+    }
+    if (reply.tag == CEPH_MSGR_TAG_RESETSESSION) {
+      ldout(msgr->cct,0) << "connect got RESETSESSION" << dendl;
+      was_session_reset();
+      cseq = 0;
+      pipe_lock.Unlock();
+      continue;
+    }
+    if (reply.tag == CEPH_MSGR_TAG_RETRY_GLOBAL) {
+      gseq = msgr->get_global_seq(reply.global_seq);
+      ldout(msgr->cct,10) << "connect got RETRY_GLOBAL " << reply.global_seq
+              << " chose new " << gseq << dendl;
+      pipe_lock.Unlock();
+      continue;
+    }
+    if (reply.tag == CEPH_MSGR_TAG_RETRY_SESSION) {
+      assert(reply.connect_seq > connect_seq);
+      ldout(msgr->cct,10) << "connect got RETRY_SESSION " << connect_seq
+              << " -> " << reply.connect_seq << dendl;
+      cseq = connect_seq = reply.connect_seq;
+      pipe_lock.Unlock();
+      continue;
+    }
+
+    if (reply.tag == CEPH_MSGR_TAG_WAIT) {
+      ldout(msgr->cct,3) << "connect got WAIT (connection race)" << dendl;
+      state = STATE_WAIT;
+      goto stop_locked;
+    }
+
+    if (reply.tag == CEPH_MSGR_TAG_READY ||
+        reply.tag == CEPH_MSGR_TAG_SEQ) {
+      uint64_t feat_missing = policy.features_required & ~(uint64_t)reply.features;
+      if (feat_missing) {
+       ldout(msgr->cct,1) << "missing required features " << std::hex << feat_missing << std::dec << dendl;
+       goto fail_locked;
+      }
+
+      if (reply.tag == CEPH_MSGR_TAG_SEQ) {
+        ldout(msgr->cct,10) << "got CEPH_MSGR_TAG_SEQ, reading acked_seq and writing in_seq" << dendl;
+        uint64_t newly_acked_seq = 0;
+        if (tcp_read((char*)&newly_acked_seq, sizeof(newly_acked_seq)) < 0) {
+          ldout(msgr->cct,2) << "connect read error on newly_acked_seq" << dendl;
+          goto fail_locked;
+        }
+       ldout(msgr->cct,2) << " got newly_acked_seq " << newly_acked_seq
+                          << " vs out_seq " << out_seq << dendl;
+       while (newly_acked_seq > out_seq) {
+         Message *m = _get_next_outgoing();
+         assert(m);
+         ldout(msgr->cct,2) << " discarding previously sent " << m->get_seq()
+                            << " " << *m << dendl;
+         assert(m->get_seq() <= newly_acked_seq);
+         m->put();
+         ++out_seq;
+       }
+        if (tcp_write((char*)&in_seq, sizeof(in_seq)) < 0) {
+          ldout(msgr->cct,2) << "connect write error on in_seq" << dendl;
+          goto fail_locked;
+        }
+      }
+
+      // hooray!
+      peer_global_seq = reply.global_seq;
+      policy.lossy = reply.flags & CEPH_MSG_CONNECT_LOSSY;
+      state = STATE_OPEN;
+      connect_seq = cseq + 1;
+      assert(connect_seq == reply.connect_seq);
+      backoff = utime_t();
+      connection_state->set_features((uint64_t)reply.features & (uint64_t)connect.features);
+      ldout(msgr->cct,10) << "connect success " << connect_seq << ", lossy = " << policy.lossy
+              << ", features " << connection_state->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(msgr->cct,
+                                    authorizer->protocol,
+                                    authorizer->session_key,
+                                    connection_state->get_features()));
+      }  else {
+        // We have no authorizer, so we shouldn't be applying security to messages in this pipe.  PLR
+       session_security.reset();
+      }
+
+      msgr->dispatch_queue.queue_connect(connection_state.get());
+      msgr->ms_deliver_handle_fast_connect(connection_state.get());
+      
+      if (!reader_running) {
+       ldout(msgr->cct,20) << "connect starting reader" << dendl;
+       start_reader();
+      }
+      maybe_start_delay_thread();
+      delete authorizer;
+      return 0;
+    }
+    
+    // protocol error
+    ldout(msgr->cct,0) << "connect got bad tag " << (int)tag << dendl;
+    goto fail_locked;
+  }
+
+ fail:
+  if (conf->ms_inject_internal_delays) {
+    ldout(msgr->cct, 10) << " sleep for " << msgr->cct->_conf->ms_inject_internal_delays << dendl;
+    utime_t t;
+    t.set_from_double(msgr->cct->_conf->ms_inject_internal_delays);
+    t.sleep();
+  }
+
+  pipe_lock.Lock();
+ fail_locked:
+  if (state == STATE_CONNECTING)
+    fault();
+  else
+    ldout(msgr->cct,3) << "connect fault, but state = " << get_state_name()
+                      << " != connecting, stopping" << dendl;
+
+ stop_locked:
+  delete authorizer;
+  return -1;
+}
+
+void Pipe::register_pipe()
+{
+  ldout(msgr->cct,10) << "register_pipe" << dendl;
+  assert(msgr->lock.is_locked());
+  Pipe *existing = msgr->_lookup_pipe(peer_addr);
+  assert(existing == NULL);
+  msgr->rank_pipe[peer_addr] = this;
+}
+
+void Pipe::unregister_pipe()
+{
+  assert(msgr->lock.is_locked());
+  ceph::unordered_map<entity_addr_t,Pipe*>::iterator p = msgr->rank_pipe.find(peer_addr);
+  if (p != msgr->rank_pipe.end() && p->second == this) {
+    ldout(msgr->cct,10) << "unregister_pipe" << dendl;
+    msgr->rank_pipe.erase(p);
+  } else {
+    ldout(msgr->cct,10) << "unregister_pipe - not registered" << dendl;
+    msgr->accepting_pipes.erase(this);  // somewhat overkill, but safe.
+  }
+}
+
+void Pipe::join()
+{
+  ldout(msgr->cct, 20) << "join" << dendl;
+  if (writer_thread.is_started())
+    writer_thread.join();
+  if (reader_thread.is_started())
+    reader_thread.join();
+  if (delay_thread) {
+    ldout(msgr->cct, 20) << "joining delay_thread" << dendl;
+    delay_thread->stop();
+    delay_thread->join();
+  }
+}
+
+void Pipe::requeue_sent()
+{
+  if (sent.empty())
+    return;
+
+  list<Message*>& rq = out_q[CEPH_MSG_PRIO_HIGHEST];
+  while (!sent.empty()) {
+    Message *m = sent.back();
+    sent.pop_back();
+    ldout(msgr->cct,10) << "requeue_sent " << *m << " for resend seq " << out_seq
+                       << " (" << m->get_seq() << ")" << dendl;
+    rq.push_front(m);
+    out_seq--;
+  }
+}
+
+void Pipe::discard_requeued_up_to(uint64_t seq)
+{
+  ldout(msgr->cct, 10) << "discard_requeued_up_to " << seq << dendl;
+  if (out_q.count(CEPH_MSG_PRIO_HIGHEST) == 0)
+    return;
+  list<Message*>& rq = out_q[CEPH_MSG_PRIO_HIGHEST];
+  while (!rq.empty()) {
+    Message *m = rq.front();
+    if (m->get_seq() == 0 || m->get_seq() > seq)
+      break;
+    ldout(msgr->cct,10) << "discard_requeued_up_to " << *m << " for resend seq " << out_seq
+                       << " <= " << seq << ", discarding" << dendl;
+    m->put();
+    rq.pop_front();
+    out_seq++;
+  }
+  if (rq.empty())
+    out_q.erase(CEPH_MSG_PRIO_HIGHEST);
+}
+
+/*
+ * Tears down the Pipe's message queues, and removes them from the DispatchQueue
+ * Must hold pipe_lock prior to calling.
+ */
+void Pipe::discard_out_queue()
+{
+  ldout(msgr->cct,10) << "discard_queue" << dendl;
+
+  for (list<Message*>::iterator p = sent.begin(); p != sent.end(); ++p) {
+    ldout(msgr->cct,20) << "  discard " << *p << dendl;
+    (*p)->put();
+  }
+  sent.clear();
+  for (map<int,list<Message*> >::iterator p = out_q.begin(); p != out_q.end(); ++p)
+    for (list<Message*>::iterator r = p->second.begin(); r != p->second.end(); ++r) {
+      ldout(msgr->cct,20) << "  discard " << *r << dendl;
+      (*r)->put();
+    }
+  out_q.clear();
+}
+
+void Pipe::fault(bool onread)
+{
+  const md_config_t *conf = msgr->cct->_conf;
+  assert(pipe_lock.is_locked());
+  cond.Signal();
+
+  if (onread && state == STATE_CONNECTING) {
+    ldout(msgr->cct,10) << "fault already connecting, reader shutting down" << dendl;
+    return;
+  }
+  
+  ldout(msgr->cct,2) << "fault " << cpp_strerror(errno) << dendl;
+
+  if (state == STATE_CLOSED ||
+      state == STATE_CLOSING) {
+    ldout(msgr->cct,10) << "fault already closed|closing" << dendl;
+    if (connection_state->clear_pipe(this))
+      msgr->dispatch_queue.queue_reset(connection_state.get());
+    return;
+  }
+
+  shutdown_socket();
+
+  // lossy channel?
+  if (policy.lossy && state != STATE_CONNECTING) {
+    ldout(msgr->cct,10) << "fault on lossy channel, failing" << dendl;
+
+    stop();
+
+    // crib locks, blech.  note that Pipe is now STATE_CLOSED and the
+    // rank_pipe entry is ignored by others.
+    pipe_lock.Unlock();
+
+    if (conf->ms_inject_internal_delays) {
+      ldout(msgr->cct, 10) << " sleep for " << msgr->cct->_conf->ms_inject_internal_delays << dendl;
+      utime_t t;
+      t.set_from_double(msgr->cct->_conf->ms_inject_internal_delays);
+      t.sleep();
+    }
+
+    msgr->lock.Lock();
+    pipe_lock.Lock();
+    unregister_pipe();
+    msgr->lock.Unlock();
+
+    in_q->discard_queue(conn_id);
+    if (delay_thread)
+      delay_thread->discard();
+    discard_out_queue();
+
+    // disconnect from Connection, and mark it failed.  future messages
+    // will be dropped.
+    assert(connection_state);
+    if (connection_state->clear_pipe(this))
+      msgr->dispatch_queue.queue_reset(connection_state.get());
+    return;
+  }
+
+  // queue delayed items immediately
+  if (delay_thread)
+    delay_thread->flush();
+
+  // requeue sent items
+  requeue_sent();
+
+  if (policy.standby && !is_queued()) {
+    ldout(msgr->cct,0) << "fault with nothing to send, going to standby" << dendl;
+    state = STATE_STANDBY;
+    return;
+  }
+
+  if (state != STATE_CONNECTING) {
+    if (policy.server) {
+      ldout(msgr->cct,0) << "fault, server, going to standby" << dendl;
+      state = STATE_STANDBY;
+    } else {
+      ldout(msgr->cct,0) << "fault, initiating reconnect" << dendl;
+      connect_seq++;
+      state = STATE_CONNECTING;
+    }
+    backoff = utime_t();
+  } else if (backoff == utime_t()) {
+    ldout(msgr->cct,0) << "fault" << dendl;
+    backoff.set_from_double(conf->ms_initial_backoff);
+  } else {
+    ldout(msgr->cct,10) << "fault waiting " << backoff << dendl;
+    cond.WaitInterval(msgr->cct, pipe_lock, backoff);
+    backoff += backoff;
+    if (backoff > conf->ms_max_backoff)
+      backoff.set_from_double(conf->ms_max_backoff);
+    ldout(msgr->cct,10) << "fault done waiting or woke up" << dendl;
+  }
+}
+
+int Pipe::randomize_out_seq()
+{
+  if (connection_state->get_features() & CEPH_FEATURE_MSG_AUTH) {
+    // Set out_seq to a random value, so CRC won't be predictable.   Don't bother checking seq_error
+    // here.  We'll check it on the call.  PLR
+    int seq_error = get_random_bytes((char *)&out_seq, sizeof(out_seq));
+    out_seq &= SEQ_MASK;
+    lsubdout(msgr->cct, ms, 10) << "randomize_out_seq " << out_seq << dendl;
+    return seq_error;
+  } else {
+    // previously, seq #'s always started at 0.
+    out_seq = 0;
+    return 0;
+  }
+}
+
+void Pipe::was_session_reset()
+{
+  assert(pipe_lock.is_locked());
+
+  ldout(msgr->cct,10) << "was_session_reset" << dendl;
+  in_q->discard_queue(conn_id);
+  if (delay_thread)
+    delay_thread->discard();
+  discard_out_queue();
+
+  msgr->dispatch_queue.queue_remote_reset(connection_state.get());
+
+  if (randomize_out_seq()) {
+    lsubdout(msgr->cct,ms,15) << "was_session_reset(): Could not get random bytes to set seq number for session reset; set seq number to " << out_seq << dendl;
+  }
+
+  in_seq = 0;
+  connect_seq = 0;
+}
+
+void Pipe::stop()
+{
+  ldout(msgr->cct,10) << "stop" << dendl;
+  assert(pipe_lock.is_locked());
+  state = STATE_CLOSED;
+  state_closed.set(1);
+  cond.Signal();
+  shutdown_socket();
+}
+
+void Pipe::stop_and_wait()
+{
+  if (state != STATE_CLOSED)
+    stop();
+  
+  // HACK: we work around an annoying deadlock here.  If the fast
+  // dispatch method calls mark_down() on itself, it can block here
+  // waiting for the reader_dispatching flag to clear... which will
+  // clearly never happen.  Avoid the situation by skipping the wait
+  // if we are marking our *own* connect down. Do the same for the
+  // delayed dispatch thread.
+  if (delay_thread) {
+    delay_thread->stop_fast_dispatching();
+  }
+  while (reader_running &&
+        reader_dispatching &&
+        !reader_thread.am_self())
+    cond.Wait(pipe_lock);
+}
+
+/* read msgs from socket.
+ * also, server.
+ */
+void Pipe::reader()
+{
+  pipe_lock.Lock();
+
+  if (state == STATE_ACCEPTING) {
+    accept();
+    assert(pipe_lock.is_locked());
+  }
+
+  // loop.
+  while (state != STATE_CLOSED &&
+        state != STATE_CONNECTING) {
+    assert(pipe_lock.is_locked());
+
+    // sleep if (re)connecting
+    if (state == STATE_STANDBY) {
+      ldout(msgr->cct,20) << "reader sleeping during reconnect|standby" << dendl;
+      cond.Wait(pipe_lock);
+      continue;
+    }
+
+    // get a reference to the AuthSessionHandler while we have the pipe_lock
+    ceph::shared_ptr<AuthSessionHandler> auth_handler = session_security;
+
+    pipe_lock.Unlock();
+
+    char tag = -1;
+    ldout(msgr->cct,20) << "reader reading tag..." << dendl;
+    if (tcp_read((char*)&tag, 1) < 0) {
+      pipe_lock.Lock();
+      ldout(msgr->cct,2) << "reader couldn't read tag, " << cpp_strerror(errno) << dendl;
+      fault(true);
+      continue;
+    }
+
+    if (tag == CEPH_MSGR_TAG_KEEPALIVE) {
+      ldout(msgr->cct,20) << "reader got KEEPALIVE" << dendl;
+      pipe_lock.Lock();
+      continue;
+    }
+    if (tag == CEPH_MSGR_TAG_KEEPALIVE2) {
+      ldout(msgr->cct,30) << "reader got KEEPALIVE2 tag ..." << dendl;
+      ceph_timespec t;
+      int rc = tcp_read((char*)&t, sizeof(t));
+      pipe_lock.Lock();
+      if (rc < 0) {
+       ldout(msgr->cct,2) << "reader couldn't read KEEPALIVE2 stamp "
+                          << cpp_strerror(errno) << dendl;
+       fault(true);
+      } else {
+       send_keepalive_ack = true;
+       keepalive_ack_stamp = utime_t(t);
+       ldout(msgr->cct,20) << "reader got KEEPALIVE2 " << keepalive_ack_stamp
+                           << dendl;
+       cond.Signal();
+      }
+      continue;
+    }
+    if (tag == CEPH_MSGR_TAG_KEEPALIVE2_ACK) {
+      ldout(msgr->cct,20) << "reader got KEEPALIVE_ACK" << dendl;
+      struct ceph_timespec t;
+      int rc = tcp_read((char*)&t, sizeof(t));
+      pipe_lock.Lock();
+      if (rc < 0) {
+       ldout(msgr->cct,2) << "reader couldn't read KEEPALIVE2 stamp " << cpp_strerror(errno) << dendl;
+       fault(true);
+      } else {
+       connection_state->last_keepalive_ack = utime_t(t);
+      }
+      continue;
+    }
+
+    // open ...
+    if (tag == CEPH_MSGR_TAG_ACK) {
+      ldout(msgr->cct,20) << "reader got ACK" << dendl;
+      ceph_le64 seq;
+      int rc = tcp_read((char*)&seq, sizeof(seq));
+      pipe_lock.Lock();
+      if (rc < 0) {
+       ldout(msgr->cct,2) << "reader couldn't read ack seq, " << cpp_strerror(errno) << dendl;
+       fault(true);
+      } else if (state != STATE_CLOSED) {
+        handle_ack(seq);
+      }
+      continue;
+    }
+
+    else if (tag == CEPH_MSGR_TAG_MSG) {
+      ldout(msgr->cct,20) << "reader got MSG" << dendl;
+      Message *m = 0;
+      int r = read_message(&m, auth_handler.get());
+
+      pipe_lock.Lock();
+      
+      if (!m) {
+       if (r < 0)
+         fault(true);
+       continue;
+      }
+
+      if (state == STATE_CLOSED ||
+         state == STATE_CONNECTING) {
+       msgr->dispatch_throttle_release(m->get_dispatch_throttle_size());
+       m->put();
+       continue;
+      }
+
+      // 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.
+      if (m->get_seq() <= in_seq) {
+       ldout(msgr->cct,0) << "reader got old message "
+               << m->get_seq() << " <= " << in_seq << " " << m << " " << *m
+               << ", discarding" << dendl;
+       msgr->dispatch_throttle_release(m->get_dispatch_throttle_size());
+       m->put();
+       if (connection_state->has_feature(CEPH_FEATURE_RECONNECT_SEQ) &&
+           msgr->cct->_conf->ms_die_on_old_message)
+         assert(0 == "old msgs despite reconnect_seq feature");
+       continue;
+      }
+
+      m->set_connection(connection_state.get());
+
+      // note last received message.
+      in_seq = m->get_seq();
+
+      cond.Signal();  // wake up writer, to ack this
+      
+      ldout(msgr->cct,10) << "reader got message "
+              << m->get_seq() << " " << m << " " << *m
+              << dendl;
+      in_q->fast_preprocess(m);
+
+      if (delay_thread) {
+        utime_t release;
+        if (rand() % 10000 < msgr->cct->_conf->ms_inject_delay_probability * 10000.0) {
+          release = m->get_recv_stamp();
+          release += msgr->cct->_conf->ms_inject_delay_max * (double)(rand() % 10000) / 10000.0;
+          lsubdout(msgr->cct, ms, 1) << "queue_received will delay until " << release << " on " << m << " " << *m << dendl;
+        }
+        delay_thread->queue(release, m);
+      } else {
+        if (in_q->can_fast_dispatch(m)) {
+         reader_dispatching = true;
+          pipe_lock.Unlock();
+          in_q->fast_dispatch(m);
+          pipe_lock.Lock();
+         reader_dispatching = false;
+         if (state == STATE_CLOSED) // there might be somebody waiting
+           cond.Signal();
+        } else {
+          in_q->enqueue(m, m->get_priority(), conn_id);
+        }
+      }
+    }
+    
+    else if (tag == CEPH_MSGR_TAG_CLOSE) {
+      ldout(msgr->cct,20) << "reader got CLOSE" << dendl;
+      pipe_lock.Lock();
+      if (state == STATE_CLOSING) {
+       state = STATE_CLOSED;
+       state_closed.set(1);
+      } else {
+       state = STATE_CLOSING;
+      }
+      cond.Signal();
+      break;
+    }
+    else {
+      ldout(msgr->cct,0) << "reader bad tag " << (int)tag << dendl;
+      pipe_lock.Lock();
+      fault(true);
+    }
+  }
+
+  // reap?
+  reader_running = false;
+  reader_needs_join = true;
+  unlock_maybe_reap();
+  ldout(msgr->cct,10) << "reader done" << dendl;
+}
+
+/* write msgs to socket.
+ * also, client.
+ */
+void Pipe::writer()
+{
+  pipe_lock.Lock();
+  while (state != STATE_CLOSED) {// && state != STATE_WAIT) {
+    ldout(msgr->cct,10) << "writer: state = " << get_state_name()
+                       << " policy.server=" << policy.server << dendl;
+
+    // standby?
+    if (is_queued() && state == STATE_STANDBY && !policy.server) {
+      connect_seq++;
+      state = STATE_CONNECTING;
+    }
+
+    // connect?
+    if (state == STATE_CONNECTING) {
+      assert(!policy.server);
+      connect();
+      continue;
+    }
+    
+    if (state == STATE_CLOSING) {
+      // write close tag
+      ldout(msgr->cct,20) << "writer writing CLOSE tag" << dendl;
+      char tag = CEPH_MSGR_TAG_CLOSE;
+      state = STATE_CLOSED;
+      state_closed.set(1);
+      pipe_lock.Unlock();
+      if (sd) {
+       int r = ::write(sd, &tag, 1);
+       // we can ignore r, actually; we don't care if this succeeds.
+       r++; r = 0; // placate gcc
+      }
+      pipe_lock.Lock();
+      continue;
+    }
+
+    if (state != STATE_CONNECTING && state != STATE_WAIT && state != STATE_STANDBY &&
+       (is_queued() || in_seq > in_seq_acked)) {
+
+      // keepalive?
+      if (send_keepalive) {
+       int rc;
+       if (connection_state->has_feature(CEPH_FEATURE_MSGR_KEEPALIVE2)) {
+         pipe_lock.Unlock();
+         rc = write_keepalive2(CEPH_MSGR_TAG_KEEPALIVE2,
+                               ceph_clock_now(msgr->cct));
+       } else {
+         pipe_lock.Unlock();
+         rc = write_keepalive();
+       }
+       pipe_lock.Lock();
+       if (rc < 0) {
+         ldout(msgr->cct,2) << "writer couldn't write keepalive[2], "
+                            << cpp_strerror(errno) << dendl;
+         fault();
+         continue;
+       }
+       send_keepalive = false;
+      }
+      if (send_keepalive_ack) {
+       utime_t t = keepalive_ack_stamp;
+       pipe_lock.Unlock();
+       int rc = write_keepalive2(CEPH_MSGR_TAG_KEEPALIVE2_ACK, t);
+       pipe_lock.Lock();
+       if (rc < 0) {
+         ldout(msgr->cct,2) << "writer couldn't write keepalive_ack, " << cpp_strerror(errno) << dendl;
+         fault();
+         continue;
+       }
+       send_keepalive_ack = false;
+      }
+
+      // send ack?
+      if (in_seq > in_seq_acked) {
+       uint64_t send_seq = in_seq;
+       pipe_lock.Unlock();
+       int rc = write_ack(send_seq);
+       pipe_lock.Lock();
+       if (rc < 0) {
+         ldout(msgr->cct,2) << "writer couldn't write ack, " << cpp_strerror(errno) << dendl;
+         fault();
+         continue;
+       }
+       in_seq_acked = send_seq;
+      }
+
+      // grab outgoing message
+      Message *m = _get_next_outgoing();
+      if (m) {
+       m->set_seq(++out_seq);
+       if (!policy.lossy) {
+         // put on sent list
+         sent.push_back(m); 
+         m->get();
+       }
+
+       // associate message with Connection (for benefit of encode_payload)
+       m->set_connection(connection_state.get());
+
+       uint64_t features = connection_state->get_features();
+
+       if (m->empty_payload())
+         ldout(msgr->cct,20) << "writer encoding " << m->get_seq() << " features " << features
+                             << " " << m << " " << *m << dendl;
+       else
+         ldout(msgr->cct,20) << "writer half-reencoding " << m->get_seq() << " features " << features
+                             << " " << m << " " << *m << dendl;
+
+       // encode and copy out of *m
+       m->encode(features, !msgr->cct->_conf->ms_nocrc);
+
+       // prepare everything
+       ceph_msg_header& header = m->get_header();
+       ceph_msg_footer& footer = m->get_footer();
+
+       // Now that we have all the crcs calculated, handle the
+       // digital signature for the message, if the pipe 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(msgr->cct, 20) << "writer no session security" << dendl;
+       } else {
+         if (session_security->sign_message(m)) {
+           ldout(msgr->cct, 20) << "writer failed to sign seq # " << header.seq
+                                << "): sig = " << footer.sig << dendl;
+         } else {
+           ldout(msgr->cct, 20) << "writer signed seq # " << header.seq
+                                << "): sig = " << footer.sig << dendl;
+         }
+       }
+
+       bufferlist blist = m->get_payload();
+       blist.append(m->get_middle());
+       blist.append(m->get_data());
+
+        pipe_lock.Unlock();
+
+        ldout(msgr->cct,20) << "writer sending " << m->get_seq() << " " << m << dendl;
+       int rc = write_message(header, footer, blist);
+
+       pipe_lock.Lock();
+       if (rc < 0) {
+          ldout(msgr->cct,1) << "writer error sending " << m << ", "
+                 << cpp_strerror(errno) << dendl;
+         fault();
+        }
+       m->put();
+      }
+      continue;
+    }
+    
+    // wait
+    ldout(msgr->cct,20) << "writer sleeping" << dendl;
+    cond.Wait(pipe_lock);
+  }
+  
+  ldout(msgr->cct,20) << "writer finishing" << dendl;
+
+  // reap?
+  writer_running = false;
+  unlock_maybe_reap();
+  ldout(msgr->cct,10) << "writer done" << dendl;
+}
+
+void Pipe::unlock_maybe_reap()
+{
+  if (!reader_running && !writer_running) {
+    shutdown_socket();
+    pipe_lock.Unlock();
+    if (delay_thread && delay_thread->is_flushing()) {
+      delay_thread->wait_for_flush();
+    }
+    msgr->queue_reap(this);
+  } else {
+    pipe_lock.Unlock();
+  }
+}
+
+static void alloc_aligned_buffer(bufferlist& data, unsigned len, unsigned off)
+{
+  // create a buffer to read into that matches the data alignment
+  unsigned left = len;
+  if (off & ~CEPH_PAGE_MASK) {
+    // head
+    unsigned head = 0;
+    head = MIN(CEPH_PAGE_SIZE - (off & ~CEPH_PAGE_MASK), left);
+    bufferptr bp = buffer::create(head);
+    data.push_back(bp);
+    left -= head;
+  }
+  unsigned middle = left & CEPH_PAGE_MASK;
+  if (middle > 0) {
+    bufferptr bp = buffer::create_page_aligned(middle);
+    data.push_back(bp);
+    left -= middle;
+  }
+  if (left) {
+    bufferptr bp = buffer::create(left);
+    data.push_back(bp);
+  }
+}
+
+int Pipe::read_message(Message **pm, AuthSessionHandler* auth_handler)
+{
+  int ret = -1;
+  // envelope
+  //ldout(msgr->cct,10) << "receiver.read_message from sd " << sd  << dendl;
+  
+  ceph_msg_header header; 
+  ceph_msg_footer footer;
+  __u32 header_crc;
+  
+  if (connection_state->has_feature(CEPH_FEATURE_NOSRCADDR)) {
+    if (tcp_read((char*)&header, sizeof(header)) < 0)
+      return -1;
+    header_crc = ceph_crc32c(0, (unsigned char *)&header, sizeof(header) - sizeof(header.crc));
+  } else {
+    ceph_msg_header_old oldheader;
+    if (tcp_read((char*)&oldheader, sizeof(oldheader)) < 0)
+      return -1;
+    // this is fugly
+    memcpy(&header, &oldheader, sizeof(header));
+    header.src = oldheader.src.name;
+    header.reserved = oldheader.reserved;
+    header.crc = oldheader.crc;
+    header_crc = ceph_crc32c(0, (unsigned char *)&oldheader, sizeof(oldheader) - sizeof(oldheader.crc));
+  }
+
+  ldout(msgr->cct,20) << "reader got envelope type=" << header.type
+           << " src " << entity_name_t(header.src)
+           << " front=" << header.front_len
+          << " data=" << header.data_len
+          << " off " << header.data_off
+           << dendl;
+
+  // verify header crc
+  if (header_crc != header.crc) {
+    ldout(msgr->cct,0) << "reader got bad header crc " << header_crc << " != " << header.crc << dendl;
+    return -1;
+  }
+
+  bufferlist front, middle, data;
+  int front_len, middle_len;
+  unsigned data_len, data_off;
+  int aborted;
+  Message *message;
+  utime_t recv_stamp = ceph_clock_now(msgr->cct);
+
+  if (policy.throttler_messages) {
+    ldout(msgr->cct,10) << "reader wants " << 1 << " message from policy throttler "
+                       << policy.throttler_messages->get_current() << "/"
+                       << policy.throttler_messages->get_max() << dendl;
+    policy.throttler_messages->get();
+  }
+
+  uint64_t message_size = header.front_len + header.middle_len + header.data_len;
+  if (message_size) {
+    if (policy.throttler_bytes) {
+      ldout(msgr->cct,10) << "reader wants " << message_size << " bytes from policy throttler "
+              << policy.throttler_bytes->get_current() << "/"
+              << policy.throttler_bytes->get_max() << dendl;
+      policy.throttler_bytes->get(message_size);
+    }
+
+    // throttle total bytes waiting for dispatch.  do this _after_ the
+    // policy throttle, as this one does not deadlock (unless dispatch
+    // blocks indefinitely, which it shouldn't).  in contrast, the
+    // policy throttle carries for the lifetime of the message.
+    ldout(msgr->cct,10) << "reader wants " << message_size << " from dispatch throttler "
+            << msgr->dispatch_throttler.get_current() << "/"
+            << msgr->dispatch_throttler.get_max() << dendl;
+    msgr->dispatch_throttler.get(message_size);
+  }
+
+  utime_t throttle_stamp = ceph_clock_now(msgr->cct);
+
+  // read front
+  front_len = header.front_len;
+  if (front_len) {
+    bufferptr bp = buffer::create(front_len);
+    if (tcp_read(bp.c_str(), front_len) < 0)
+      goto out_dethrottle;
+    front.push_back(bp);
+    ldout(msgr->cct,20) << "reader got front " << front.length() << dendl;
+  }
+
+  // read middle
+  middle_len = header.middle_len;
+  if (middle_len) {
+    bufferptr bp = buffer::create(middle_len);
+    if (tcp_read(bp.c_str(), middle_len) < 0)
+      goto out_dethrottle;
+    middle.push_back(bp);
+    ldout(msgr->cct,20) << "reader got middle " << middle.length() << dendl;
+  }
+
+
+  // read data
+  data_len = le32_to_cpu(header.data_len);
+  data_off = le32_to_cpu(header.data_off);
+  if (data_len) {
+    unsigned offset = 0;
+    unsigned left = data_len;
+
+    bufferlist newbuf, rxbuf;
+    bufferlist::iterator blp;
+    int rxbuf_version = 0;
+       
+    while (left > 0) {
+      // wait for data
+      if (tcp_read_wait() < 0)
+       goto out_dethrottle;
+
+      // get a buffer
+      connection_state->lock.Lock();
+      map<ceph_tid_t,pair<bufferlist,int> >::iterator p = connection_state->rx_buffers.find(header.tid);
+      if (p != connection_state->rx_buffers.end()) {
+       if (rxbuf.length() == 0 || p->second.second != rxbuf_version) {
+         ldout(msgr->cct,10) << "reader seleting rx buffer v " << p->second.second
+                  << " at offset " << offset
+                  << " len " << p->second.first.length() << dendl;
+         rxbuf = p->second.first;
+         rxbuf_version = p->second.second;
+         // make sure it's big enough
+         if (rxbuf.length() < data_len)
+           rxbuf.push_back(buffer::create(data_len - rxbuf.length()));
+         blp = p->second.first.begin();
+         blp.advance(offset);
+       }
+      } else {
+       if (!newbuf.length()) {
+         ldout(msgr->cct,20) << "reader allocating new rx buffer at offset " << offset << dendl;
+         alloc_aligned_buffer(newbuf, data_len, data_off);
+         blp = newbuf.begin();
+         blp.advance(offset);
+       }
+      }
+      bufferptr bp = blp.get_current_ptr();
+      int read = MIN(bp.length(), left);
+      ldout(msgr->cct,20) << "reader reading nonblocking into " << (void*)bp.c_str() << " len " << bp.length() << dendl;
+      int got = tcp_read_nonblocking(bp.c_str(), read);
+      ldout(msgr->cct,30) << "reader read " << got << " of " << read << dendl;
+      connection_state->lock.Unlock();
+      if (got < 0)
+       goto out_dethrottle;
+      if (got > 0) {
+       blp.advance(got);
+       data.append(bp, 0, got);
+       offset += got;
+       left -= got;
+      } // else we got a signal or something; just loop.
+    }
+  }
+
+  // footer
+  if (connection_state->has_feature(CEPH_FEATURE_MSG_AUTH)) {
+    if (tcp_read((char*)&footer, sizeof(footer)) < 0)
+      goto out_dethrottle;
+  } else {
+    ceph_msg_footer_old old_footer;
+    if (tcp_read((char*)&old_footer, sizeof(old_footer)) < 0)
+      goto out_dethrottle;
+    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;
+  }
+  
+  aborted = (footer.flags & CEPH_MSG_FOOTER_COMPLETE) == 0;
+  ldout(msgr->cct,10) << "aborted = " << aborted << dendl;
+  if (aborted) {
+    ldout(msgr->cct,0) << "reader got " << front.length() << " + " << middle.length() << " + " << data.length()
+           << " byte message.. ABORTED" << dendl;
+    ret = 0;
+    goto out_dethrottle;
+  }
+
+  ldout(msgr->cct,20) << "reader got " << front.length() << " + " << middle.length() << " + " << data.length()
+          << " byte message" << dendl;
+  message = decode_message(msgr->cct, header, footer, front, middle, data);
+  if (!message) {
+    ret = -EINVAL;
+    goto out_dethrottle;
+  }
+
+  //
+  //  Check the signature if one should be present.  A zero return indicates success. PLR
+  //
+
+  if (auth_handler == NULL) {
+    ldout(msgr->cct, 10) << "No session security set" << dendl;
+  } else {
+    if (auth_handler->check_message_signature(message)) {
+      ldout(msgr->cct, 0) << "Signature check failed" << dendl;
+      ret = -EINVAL;
+      goto out_dethrottle;
+    } 
+  }
+
+  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(message_size);
+
+  message->set_recv_stamp(recv_stamp);
+  message->set_throttle_stamp(throttle_stamp);
+  message->set_recv_complete_stamp(ceph_clock_now(msgr->cct));
+
+  *pm = message;
+  return 0;
+
+ out_dethrottle:
+  // release bytes reserved from the throttlers on failure
+  if (policy.throttler_messages) {
+    ldout(msgr->cct,10) << "reader releasing " << 1 << " message to policy throttler "
+                       << policy.throttler_messages->get_current() << "/"
+                       << policy.throttler_messages->get_max() << dendl;
+    policy.throttler_messages->put();
+  }
+  if (message_size) {
+    if (policy.throttler_bytes) {
+      ldout(msgr->cct,10) << "reader releasing " << message_size << " bytes to policy throttler "
+                         << policy.throttler_bytes->get_current() << "/"
+                         << policy.throttler_bytes->get_max() << dendl;
+      policy.throttler_bytes->put(message_size);
+    }
+
+    msgr->dispatch_throttle_release(message_size);
+  }
+  return ret;
+}
+
+int Pipe::do_sendmsg(struct msghdr *msg, int len, bool more)
+{
+  while (len > 0) {
+    if (0) { // sanity
+      int l = 0;
+      for (unsigned i=0; i<msg->msg_iovlen; i++)
+       l += msg->msg_iov[i].iov_len;
+      assert(l == len);
+    }
+
+    int r = ::sendmsg(sd, msg, MSG_NOSIGNAL | (more ? MSG_MORE : 0));
+    if (r == 0) 
+      ldout(msgr->cct,10) << "do_sendmsg hmm do_sendmsg got r==0!" << dendl;
+    if (r < 0) { 
+      ldout(msgr->cct,1) << "do_sendmsg error " << cpp_strerror(errno) << dendl;
+      return -1;
+    }
+    if (state == STATE_CLOSED) {
+      ldout(msgr->cct,10) << "do_sendmsg oh look, state == CLOSED, giving up" << dendl;
+      errno = EINTR;
+      return -1; // close enough
+    }
+
+    len -= r;
+    if (len == 0) break;
+    
+    // hrmph.  trim r bytes off the front of our message.
+    ldout(msgr->cct,20) << "do_sendmsg short write did " << r << ", still have " << len << dendl;
+    while (r > 0) {
+      if (msg->msg_iov[0].iov_len <= (size_t)r) {
+       // lose this whole item
+       //ldout(msgr->cct,30) << "skipping " << msg->msg_iov[0].iov_len << ", " << (msg->msg_iovlen-1) << " v, " << r << " left" << dendl;
+       r -= msg->msg_iov[0].iov_len;
+       msg->msg_iov++;
+       msg->msg_iovlen--;
+      } else {
+       // partial!
+       //ldout(msgr->cct,30) << "adjusting " << msg->msg_iov[0].iov_len << ", " << msg->msg_iovlen << " v, " << r << " left" << dendl;
+       msg->msg_iov[0].iov_base = (char *)msg->msg_iov[0].iov_base + r;
+       msg->msg_iov[0].iov_len -= r;
+       break;
+      }
+    }
+  }
+  return 0;
+}
+
+
+int Pipe::write_ack(uint64_t seq)
+{
+  ldout(msgr->cct,10) << "write_ack " << seq << dendl;
+
+  char c = CEPH_MSGR_TAG_ACK;
+  ceph_le64 s;
+  s = seq;
+
+  struct msghdr msg;
+  memset(&msg, 0, sizeof(msg));
+  struct iovec msgvec[2];
+  msgvec[0].iov_base = &c;
+  msgvec[0].iov_len = 1;
+  msgvec[1].iov_base = &s;
+  msgvec[1].iov_len = sizeof(s);
+  msg.msg_iov = msgvec;
+  msg.msg_iovlen = 2;
+  
+  if (do_sendmsg(&msg, 1 + sizeof(s), true) < 0)
+    return -1; 
+  return 0;
+}
+
+int Pipe::write_keepalive()
+{
+  ldout(msgr->cct,10) << "write_keepalive" << dendl;
+
+  char c = CEPH_MSGR_TAG_KEEPALIVE;
+
+  struct msghdr msg;
+  memset(&msg, 0, sizeof(msg));
+  struct iovec msgvec[2];
+  msgvec[0].iov_base = &c;
+  msgvec[0].iov_len = 1;
+  msg.msg_iov = msgvec;
+  msg.msg_iovlen = 1;
+  
+  if (do_sendmsg(&msg, 1) < 0)
+    return -1; 
+  return 0;
+}
+
+int Pipe::write_keepalive2(char tag, const utime_t& t)
+{
+  ldout(msgr->cct,10) << "write_keepalive2 " << (int)tag << " " << t << dendl;
+  struct ceph_timespec ts;
+  t.encode_timeval(&ts);
+  struct msghdr msg;
+  memset(&msg, 0, sizeof(msg));
+  struct iovec msgvec[2];
+  msgvec[0].iov_base = &tag;
+  msgvec[0].iov_len = 1;
+  msgvec[1].iov_base = &ts;
+  msgvec[1].iov_len = sizeof(ts);
+  msg.msg_iov = msgvec;
+  msg.msg_iovlen = 2;
+
+  if (do_sendmsg(&msg, 1 + sizeof(ts)) < 0)
+    return -1;
+  return 0;
+}
+
+
+int Pipe::write_message(ceph_msg_header& header, ceph_msg_footer& footer, bufferlist& blist)
+{
+  int ret;
+
+  // set up msghdr and iovecs
+  struct msghdr msg;
+  memset(&msg, 0, sizeof(msg));
+  msg.msg_iov = msgvec;
+  int msglen = 0;
+  
+  // send tag
+  char tag = CEPH_MSGR_TAG_MSG;
+  msgvec[msg.msg_iovlen].iov_base = &tag;
+  msgvec[msg.msg_iovlen].iov_len = 1;
+  msglen++;
+  msg.msg_iovlen++;
+
+  // send envelope
+  ceph_msg_header_old oldheader;
+  if (connection_state->has_feature(CEPH_FEATURE_NOSRCADDR)) {
+    msgvec[msg.msg_iovlen].iov_base = (char*)&header;
+    msgvec[msg.msg_iovlen].iov_len = sizeof(header);
+    msglen += sizeof(header);
+    msg.msg_iovlen++;
+  } else {
+    memcpy(&oldheader, &header, sizeof(header));
+    oldheader.src.name = header.src;
+    oldheader.src.addr = connection_state->get_peer_addr();
+    oldheader.orig_src = oldheader.src;
+    oldheader.reserved = header.reserved;
+    oldheader.crc = ceph_crc32c(0, (unsigned char*)&oldheader,
+                               sizeof(oldheader) - sizeof(oldheader.crc));
+    msgvec[msg.msg_iovlen].iov_base = (char*)&oldheader;
+    msgvec[msg.msg_iovlen].iov_len = sizeof(oldheader);
+    msglen += sizeof(oldheader);
+    msg.msg_iovlen++;
+  }
+
+  // payload (front+data)
+  list<bufferptr>::const_iterator pb = blist.buffers().begin();
+  int b_off = 0;  // carry-over buffer offset, if any
+  int bl_pos = 0; // blist pos
+  int left = blist.length();
+
+  while (left > 0) {
+    int donow = MIN(left, (int)pb->length()-b_off);
+    if (donow == 0) {
+      ldout(msgr->cct,0) << "donow = " << donow << " left " << left << " pb->length " << pb->length()
+             << " b_off " << b_off << dendl;
+    }
+    assert(donow > 0);
+    ldout(msgr->cct,30) << " bl_pos " << bl_pos << " b_off " << b_off
+            << " leftinchunk " << left
+            << " buffer len " << pb->length()
+            << " writing " << donow 
+            << dendl;
+    
+    if (msg.msg_iovlen >= IOV_MAX-2) {
+      if (do_sendmsg(&msg, msglen, true))
+       goto fail;
+      
+      // and restart the iov
+      msg.msg_iov = msgvec;
+      msg.msg_iovlen = 0;
+      msglen = 0;
+    }
+    
+    msgvec[msg.msg_iovlen].iov_base = (void*)(pb->c_str()+b_off);
+    msgvec[msg.msg_iovlen].iov_len = donow;
+    msglen += donow;
+    msg.msg_iovlen++;
+    
+    left -= donow;
+    assert(left >= 0);
+    b_off += donow;
+    bl_pos += donow;
+    if (left == 0)
+      break;
+    while (b_off == (int)pb->length()) {
+      ++pb;
+      b_off = 0;
+    }
+  }
+  assert(left == 0);
+
+  // send footer; if receiver doesn't support signatures, use the old footer format
+
+  ceph_msg_footer_old old_footer;
+  if (connection_state->has_feature(CEPH_FEATURE_MSG_AUTH)) {
+    msgvec[msg.msg_iovlen].iov_base = (void*)&footer;
+    msgvec[msg.msg_iovlen].iov_len = sizeof(footer);
+    msglen += sizeof(footer);
+    msg.msg_iovlen++;
+  } else {
+    old_footer.front_crc = footer.front_crc;   
+    old_footer.middle_crc = footer.middle_crc;   
+    old_footer.data_crc = footer.data_crc;   
+    old_footer.flags = footer.flags;   
+    msgvec[msg.msg_iovlen].iov_base = (char*)&old_footer;
+    msgvec[msg.msg_iovlen].iov_len = sizeof(old_footer);
+    msglen += sizeof(old_footer);
+    msg.msg_iovlen++;
+  }
+
+  // send
+  if (do_sendmsg(&msg, msglen))
+    goto fail;
+
+  ret = 0;
+
+ out:
+  return ret;
+
+ fail:
+  ret = -1;
+  goto out;
+}
+
+
+int Pipe::tcp_read(char *buf, int len)
+{
+  if (sd < 0)
+    return -1;
+
+  while (len > 0) {
+
+    if (msgr->cct->_conf->ms_inject_socket_failures && sd >= 0) {
+      if (rand() % msgr->cct->_conf->ms_inject_socket_failures == 0) {
+       ldout(msgr->cct, 0) << "injecting socket failure" << dendl;
+       ::shutdown(sd, SHUT_RDWR);
+      }
+    }
+
+    if (tcp_read_wait() < 0)
+      return -1;
+
+    int got = tcp_read_nonblocking(buf, len);
+
+    if (got < 0)
+      return -1;
+
+    len -= got;
+    buf += got;
+    //lgeneric_dout(cct, DBL) << "tcp_read got " << got << ", " << len << " left" << dendl;
+  }
+  return len;
+}
+
+int Pipe::tcp_read_wait()
+{
+  if (sd < 0)
+    return -1;
+  struct pollfd pfd;
+  short evmask;
+  pfd.fd = sd;
+  pfd.events = POLLIN;
+#if defined(__linux__)
+  pfd.events |= POLLRDHUP;
+#endif
+
+  if (has_pending_data())
+    return 0;
+
+  if (poll(&pfd, 1, msgr->timeout) <= 0)
+    return -1;
+
+  evmask = POLLERR | POLLHUP | POLLNVAL;
+#if defined(__linux__)
+  evmask |= POLLRDHUP;
+#endif
+  if (pfd.revents & evmask)
+    return -1;
+
+  if (!(pfd.revents & POLLIN))
+    return -1;
+
+  return 0;
+}
+
+int Pipe::do_recv(char *buf, size_t len, int flags)
+{
+again:
+  int got = ::recv( sd, buf, len, flags );
+  if (got < 0) {
+    if (errno == EAGAIN || errno == EINTR) {
+      goto again;
+    }
+    ldout(msgr->cct, 10) << __func__ << " socket " << sd << " returned "
+                    << got << " " << cpp_strerror(errno) << dendl;
+    return -1;
+  }
+  if (got == 0) {
+    return -1;
+  }
+  return got;
+}
+
+int Pipe::buffered_recv(char *buf, size_t len, int flags)
+{
+  int left = len;
+  int total_recv = 0;
+  if (recv_len > recv_ofs) {
+    int to_read = MIN(recv_len - recv_ofs, left);
+    memcpy(buf, &recv_buf[recv_ofs], to_read);
+    recv_ofs += to_read;
+    left -= to_read;
+    if (left == 0) {
+      return to_read;
+    }
+    buf += to_read;
+    total_recv += to_read;
+  }
+
+  /* nothing left in the prefetch buffer */
+
+  if (len > (size_t)recv_max_prefetch) {
+    /* this was a large read, we don't prefetch for these */
+    int ret = do_recv(buf, left, flags );
+    if (ret < 0) {
+      if (total_recv > 0)
+        return total_recv;
+      return ret;
+    }
+    total_recv += ret;
+    return total_recv;
+  }
+
+
+  int got = do_recv(recv_buf, recv_max_prefetch, flags);
+  if (got <= 0) {
+    if (total_recv > 0)
+      return total_recv;
+
+    return got;
+  }
+
+  recv_len = got;
+  got = MIN(left, got);
+  memcpy(buf, recv_buf, got);
+  recv_ofs = got;
+  total_recv += got;
+  return total_recv;
+}
+
+int Pipe::tcp_read_nonblocking(char *buf, int len)
+{
+  int got = buffered_recv(buf, len, MSG_DONTWAIT );
+  if (got < 0) {
+    ldout(msgr->cct, 10) << __func__ << " socket " << sd << " returned "
+                        << got << " " << cpp_strerror(errno) << dendl;
+    return -1;
+  }
+  if (got == 0) {
+    /* poll() said there was data, but we didn't read any - peer
+     * sent a FIN.  Maybe POLLRDHUP signals this, but this is
+     * standard socket behavior as documented by Stevens.
+     */
+    return -1;
+  }
+  return got;
+}
+
+int Pipe::tcp_write(const char *buf, int len)
+{
+  if (sd < 0)
+    return -1;
+  struct pollfd pfd;
+  pfd.fd = sd;
+  pfd.events = POLLOUT | POLLHUP | POLLNVAL | POLLERR;
+#if defined(__linux__)
+  pfd.events |= POLLRDHUP;
+#endif
+
+  if (msgr->cct->_conf->ms_inject_socket_failures && sd >= 0) {
+    if (rand() % msgr->cct->_conf->ms_inject_socket_failures == 0) {
+      ldout(msgr->cct, 0) << "injecting socket failure" << dendl;
+      ::shutdown(sd, SHUT_RDWR);
+    }
+  }
+
+  if (poll(&pfd, 1, -1) < 0)
+    return -1;
+
+  if (!(pfd.revents & POLLOUT))
+    return -1;
+
+  //lgeneric_dout(cct, DBL) << "tcp_write writing " << len << dendl;
+  assert(len > 0);
+  while (len > 0) {
+    int did = ::send( sd, buf, len, MSG_NOSIGNAL );
+    if (did < 0) {
+      //lgeneric_dout(cct, 1) << "tcp_write error did = " << did << " " << cpp_strerror(errno) << dendl;
+      //lgeneric_derr(cct, 1) << "tcp_write error did = " << did << " " << cpp_strerror(errno) << dendl;
+      return did;
+    }
+    len -= did;
+    buf += did;
+    //lgeneric_dout(cct, DBL) << "tcp_write did " << did << ", " << len << " left" << dendl;
+  }
+  return 0;
+}
diff --git a/src/msg/simple/Pipe.h b/src/msg/simple/Pipe.h
new file mode 100644 (file)
index 0000000..3f33d31
--- /dev/null
@@ -0,0 +1,374 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software 
+ * Foundation.  See file COPYING.
+ * 
+ */
+
+#ifndef CEPH_MSGR_PIPE_H
+#define CEPH_MSGR_PIPE_H
+
+#include "include/memory.h"
+#include "auth/AuthSessionHandler.h"
+
+#include "msg/msg_types.h"
+#include "msg/Messenger.h"
+#include "PipeConnection.h"
+
+
+class SimpleMessenger;
+class IncomingQueue;
+class DispatchQueue;
+
+  /**
+   * The Pipe is the most complex SimpleMessenger component. It gets
+   * two threads, one each for reading and writing on a socket it's handed
+   * at creation time, and is responsible for everything that happens on
+   * that socket. Besides message transmission, it's responsible for
+   * propagating socket errors to the SimpleMessenger and then sticking
+   * around in a state where it can provide enough data for the SimpleMessenger
+   * to provide reliable Message delivery when it manages to reconnect.
+   */
+  class Pipe : public RefCountedObject {
+    /**
+     * The Reader thread handles all reads off the socket -- not just
+     * Messages, but also acks and other protocol bits (excepting startup,
+     * when the Writer does a couple of reads).
+     * All the work is implemented in Pipe itself, of course.
+     */
+    class Reader : public Thread {
+      Pipe *pipe;
+    public:
+      Reader(Pipe *p) : pipe(p) {}
+      void *entry() { pipe->reader(); return 0; }
+    } reader_thread;
+    friend class Reader;
+
+    /**
+     * The Writer thread handles all writes to the socket (after startup).
+     * All the work is implemented in Pipe itself, of course.
+     */
+    class Writer : public Thread {
+      Pipe *pipe;
+    public:
+      Writer(Pipe *p) : pipe(p) {}
+      void *entry() { pipe->writer(); return 0; }
+    } writer_thread;
+    friend class Writer;
+
+    /**
+     * The DelayedDelivery is for injecting delays into Message delivery off
+     * the socket. It is only enabled if delays are requested, and if they
+     * are then it pulls Messages off the DelayQueue and puts them into the
+     * in_q (SimpleMessenger::dispatch_queue).
+     * Please note that this probably has issues with Pipe shutdown and
+     * replacement semantics. I've tried, but no guarantees.
+     */
+    class DelayedDelivery: public Thread {
+      Pipe *pipe;
+      std::deque< pair<utime_t,Message*> > delay_queue;
+      Mutex delay_lock;
+      Cond delay_cond;
+      int flush_count;
+      bool active_flush;
+      bool stop_delayed_delivery;
+      bool delay_dispatching; // we are in fast dispatch now
+      bool stop_fast_dispatching_flag; // we need to stop fast dispatching
+
+    public:
+      DelayedDelivery(Pipe *p)
+       : pipe(p),
+         delay_lock("Pipe::DelayedDelivery::delay_lock"), flush_count(0),
+         active_flush(false),
+         stop_delayed_delivery(false),
+         delay_dispatching(false),
+         stop_fast_dispatching_flag(false) { }
+      ~DelayedDelivery() {
+       discard();
+      }
+      void *entry();
+      void queue(utime_t release, Message *m) {
+       Mutex::Locker l(delay_lock);
+       delay_queue.push_back(make_pair(release, m));
+       delay_cond.Signal();
+      }
+      void discard();
+      void flush();
+      bool is_flushing() {
+        Mutex::Locker l(delay_lock);
+        return flush_count > 0 || active_flush;
+      }
+      void wait_for_flush() {
+        Mutex::Locker l(delay_lock);
+        while (flush_count > 0 || active_flush)
+          delay_cond.Wait(delay_lock);
+      }
+      void stop() {
+       delay_lock.Lock();
+       stop_delayed_delivery = true;
+       delay_cond.Signal();
+       delay_lock.Unlock();
+      }
+      void steal_for_pipe(Pipe *new_owner) {
+        Mutex::Locker l(delay_lock);
+        pipe = new_owner;
+      }
+      /**
+       * We need to stop fast dispatching before we need to stop putting
+       * normal messages into the DispatchQueue.
+       */
+      void stop_fast_dispatching();
+    } *delay_thread;
+    friend class DelayedDelivery;
+
+  public:
+    Pipe(SimpleMessenger *r, int st, PipeConnection *con);
+    ~Pipe();
+
+    SimpleMessenger *msgr;
+    uint64_t conn_id;
+    ostream& _pipe_prefix(std::ostream *_dout);
+
+    Pipe* get() {
+      return static_cast<Pipe*>(RefCountedObject::get());
+    }
+
+    char *recv_buf;
+    int recv_max_prefetch;
+    int recv_ofs;
+    int recv_len;
+
+    enum {
+      STATE_ACCEPTING,
+      STATE_CONNECTING,
+      STATE_OPEN,
+      STATE_STANDBY,
+      STATE_CLOSED,
+      STATE_CLOSING,
+      STATE_WAIT       // just wait for racing connection
+    };
+
+    static const char *get_state_name(int s) {
+      switch (s) {
+      case STATE_ACCEPTING: return "accepting";
+      case STATE_CONNECTING: return "connecting";
+      case STATE_OPEN: return "open";
+      case STATE_STANDBY: return "standby";
+      case STATE_CLOSED: return "closed";
+      case STATE_CLOSING: return "closing";
+      case STATE_WAIT: return "wait";
+      default: return "UNKNOWN";
+      }
+    }
+    const char *get_state_name() {
+      return get_state_name(state);
+    }
+
+  private:
+    int sd;
+    struct iovec msgvec[IOV_MAX];
+
+  public:
+    int port;
+    int peer_type;
+    entity_addr_t peer_addr;
+    Messenger::Policy policy;
+    
+    Mutex pipe_lock;
+    int state;
+    atomic_t state_closed; // non-zero iff state = STATE_CLOSED
+
+    // session_security handles any signatures or encryptions required for this pipe's msgs. PLR
+
+    ceph::shared_ptr<AuthSessionHandler> session_security;
+
+  protected:
+    friend class SimpleMessenger;
+    PipeConnectionRef connection_state;
+
+    utime_t backoff;         // backoff time
+
+    bool reader_running, reader_needs_join;
+    bool reader_dispatching; /// reader thread is dispatching without pipe_lock
+    bool writer_running;
+
+    map<int, list<Message*> > out_q;  // priority queue for outbound msgs
+    DispatchQueue *in_q;
+    list<Message*> sent;
+    Cond cond;
+    bool send_keepalive;
+    bool send_keepalive_ack;
+    utime_t keepalive_ack_stamp;
+    bool halt_delivery; //if a pipe's queue is destroyed, stop adding to it
+    
+    __u32 connect_seq, peer_global_seq;
+    uint64_t out_seq;
+    uint64_t in_seq, in_seq_acked;
+    
+    void set_socket_options();
+
+    int accept();   // server handshake
+    int connect();  // client handshake
+    void reader();
+    void writer();
+    void unlock_maybe_reap();
+
+    int randomize_out_seq();
+
+    int read_message(Message **pm,
+                    AuthSessionHandler *session_security_copy);
+    int write_message(ceph_msg_header& h, ceph_msg_footer& f, bufferlist& body);
+    /**
+     * Write the given data (of length len) to the Pipe's socket. This function
+     * will loop until all passed data has been written out.
+     * If more is set, the function will optimize socket writes
+     * for additional data (by passing the MSG_MORE flag, aka TCP_CORK).
+     *
+     * @param msg The msghdr to write out
+     * @param len The length of the data in msg
+     * @param more Should be set true if this is one part of a larger message
+     * @return 0, or -1 on failure (unrecoverable -- close the socket).
+     */
+    int do_sendmsg(struct msghdr *msg, int len, bool more=false);
+    int write_ack(uint64_t s);
+    int write_keepalive();
+    int write_keepalive2(char tag, const utime_t &t);
+
+    void fault(bool reader=false);
+
+    void was_session_reset();
+
+    /* Clean up sent list */
+    void handle_ack(uint64_t seq);
+
+    public:
+    Pipe(const Pipe& other);
+    const Pipe& operator=(const Pipe& other);
+
+    void start_reader();
+    void start_writer();
+    void maybe_start_delay_thread();
+    void join_reader();
+
+    // public constructors
+    static const Pipe& Server(int s);
+    static const Pipe& Client(const entity_addr_t& pi);
+
+    __u32 get_out_seq() { return out_seq; }
+
+    bool is_queued() { return !out_q.empty() || send_keepalive || send_keepalive_ack; }
+
+    entity_addr_t& get_peer_addr() { return peer_addr; }
+
+    void set_peer_addr(const entity_addr_t& a) {
+      if (&peer_addr != &a)  // shut up valgrind
+        peer_addr = a;
+      connection_state->set_peer_addr(a);
+    }
+    void set_peer_type(int t) {
+      peer_type = t;
+      connection_state->set_peer_type(t);
+    }
+
+    void register_pipe();
+    void unregister_pipe();
+    void join();
+    /// stop a Pipe by closing its socket and setting it to STATE_CLOSED
+    void stop();
+    /// stop() a Pipe if not already done, and wait for it to finish any
+    /// fast_dispatch in progress.
+    void stop_and_wait();
+
+    void _send(Message *m) {
+      assert(pipe_lock.is_locked());
+      out_q[m->get_priority()].push_back(m);
+      cond.Signal();
+    }
+    void _send_keepalive() {
+      assert(pipe_lock.is_locked());
+      send_keepalive = true;
+      cond.Signal();
+    }
+    Message *_get_next_outgoing() {
+      assert(pipe_lock.is_locked());
+      Message *m = 0;
+      while (!m && !out_q.empty()) {
+        map<int, list<Message*> >::reverse_iterator p = out_q.rbegin();
+        if (!p->second.empty()) {
+          m = p->second.front();
+          p->second.pop_front();
+        }
+        if (p->second.empty())
+          out_q.erase(p->first);
+      }
+      return m;
+    }
+
+    /// move all messages in the sent list back into the queue at the highest priority.
+    void requeue_sent();
+    /// discard messages requeued by requeued_sent() up to a given seq
+    void discard_requeued_up_to(uint64_t seq);
+    void discard_out_queue();
+
+    void shutdown_socket() {
+      recv_reset();
+      if (sd >= 0)
+        ::shutdown(sd, SHUT_RDWR);
+    }
+
+    void recv_reset() {
+      recv_len = 0;
+      recv_ofs = 0;
+    }
+    int do_recv(char *buf, size_t len, int flags);
+    int buffered_recv(char *buf, size_t len, int flags);
+    bool has_pending_data() { return recv_len > recv_ofs; }
+
+    /**
+     * do a blocking read of len bytes from socket
+     *
+     * @param buf buffer to read into
+     * @param len exact number of bytes to read
+     * @return 0 for success, or -1 on error
+     */
+    int tcp_read(char *buf, int len);
+
+    /**
+     * wait for bytes to become available on the socket
+     *
+     * @return 0 for success, or -1 on error
+     */
+    int tcp_read_wait();
+
+    /**
+     * non-blocking read of available bytes on socket
+     *
+     * This is expected to be used after tcp_read_wait(), and will return
+     * an error if there is no data on the socket to consume.
+     *
+     * @param buf buffer to read into
+     * @param len maximum number of bytes to read
+     * @return bytes read, or -1 on error or when there is no data
+     */
+    int tcp_read_nonblocking(char *buf, int len);
+
+    /**
+     * blocking write of bytes to socket
+     *
+     * @param buf buffer
+     * @param len number of bytes to write
+     * @return 0 for success, or -1 on error
+     */
+    int tcp_write(const char *buf, int len);
+
+  };
+
+
+#endif
diff --git a/src/msg/simple/PipeConnection.cc b/src/msg/simple/PipeConnection.cc
new file mode 100644 (file)
index 0000000..f4db374
--- /dev/null
@@ -0,0 +1,91 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation.  See file COPYING.
+ *
+ */
+
+#include "msg/Message.h"
+#include "Pipe.h"
+#include "SimpleMessenger.h"
+#include "PipeConnection.h"
+
+PipeConnection::~PipeConnection()
+{
+  if (pipe) {
+    pipe->put();
+    pipe = NULL;
+  }
+}
+
+Pipe* PipeConnection::get_pipe()
+{
+  Mutex::Locker l(lock);
+  if (pipe)
+    return pipe->get();
+  return NULL;
+}
+
+bool PipeConnection::try_get_pipe(Pipe **p)
+{
+  Mutex::Locker l(lock);
+  if (failed) {
+    *p = NULL;
+  } else {
+    if (pipe)
+      *p = pipe->get();
+    else
+      *p = NULL;
+  }
+  return !failed;
+}
+
+bool PipeConnection::clear_pipe(Pipe *old_p)
+{
+  Mutex::Locker l(lock);
+  if (old_p == pipe) {
+    pipe->put();
+    pipe = NULL;
+    failed = true;
+    return true;
+  }
+  return false;
+}
+
+void PipeConnection::reset_pipe(Pipe *p)
+{
+  Mutex::Locker l(lock);
+  if (pipe)
+    pipe->put();
+  pipe = p->get();
+}
+
+int PipeConnection::send_message(Message *m)
+{
+  assert(msgr);
+  return static_cast<SimpleMessenger*>(msgr)->send_message(m, this);
+}
+
+void PipeConnection::send_keepalive()
+{
+  static_cast<SimpleMessenger*>(msgr)->send_keepalive(this);
+}
+
+void PipeConnection::mark_down()
+{
+  if (msgr)
+    static_cast<SimpleMessenger*>(msgr)->mark_down(this);
+}
+
+void PipeConnection::mark_disposable()
+{
+  if (msgr)
+    static_cast<SimpleMessenger*>(msgr)->mark_disposable(this);
+}
diff --git a/src/msg/simple/PipeConnection.h b/src/msg/simple/PipeConnection.h
new file mode 100644 (file)
index 0000000..cbe2677
--- /dev/null
@@ -0,0 +1,58 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2014 Red Hat
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation.  See file COPYING.
+ *
+ */
+
+#ifndef CEPH_MSG_PIPECONNECTION_H
+#define CEPH_MSG_PIPECONNECTION_H
+
+#include "msg/Connection.h"
+
+class Pipe;
+
+struct PipeConnection : public Connection {
+  Pipe* pipe;
+
+  friend class boost::intrusive_ptr<PipeConnection>;
+  friend class Pipe;
+
+public:
+
+  PipeConnection(CephContext *cct, Messenger *m)
+    : Connection(cct, m),
+      pipe(NULL) { }
+
+  ~PipeConnection();
+
+  Pipe* get_pipe();
+
+  bool try_get_pipe(Pipe** p);
+
+  bool clear_pipe(Pipe* old_p);
+
+  void reset_pipe(Pipe* p);
+
+  bool is_connected() {
+    Mutex::Locker l(lock);
+    return pipe != NULL;
+  }
+
+  int send_message(Message *m);
+  void send_keepalive();
+  void mark_down();
+  void mark_disposable();
+
+}; /* PipeConnection */
+
+typedef boost::intrusive_ptr<PipeConnection> PipeConnectionRef;
+
+#endif
diff --git a/src/msg/simple/SimpleMessenger.cc b/src/msg/simple/SimpleMessenger.cc
new file mode 100644 (file)
index 0000000..cf6bc73
--- /dev/null
@@ -0,0 +1,702 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software 
+ * Foundation.  See file COPYING.
+ * 
+ */
+
+#include <errno.h>
+#include <iostream>
+#include <fstream>
+
+
+#include "SimpleMessenger.h"
+
+#include "common/config.h"
+#include "common/Timer.h"
+#include "common/errno.h"
+#include "auth/Crypto.h"
+#include "include/Spinlock.h"
+
+#define dout_subsys ceph_subsys_ms
+#undef dout_prefix
+#define dout_prefix _prefix(_dout, this)
+static ostream& _prefix(std::ostream *_dout, SimpleMessenger *msgr) {
+  return *_dout << "-- " << msgr->get_myaddr() << " ";
+}
+
+
+/*******************
+ * SimpleMessenger
+ */
+
+SimpleMessenger::SimpleMessenger(CephContext *cct, entity_name_t name,
+                                string mname, uint64_t _nonce)
+  : SimplePolicyMessenger(cct, name,mname, _nonce),
+    accepter(this, _nonce),
+    dispatch_queue(cct, this),
+    reaper_thread(this),
+    nonce(_nonce),
+    lock("SimpleMessenger::lock"), need_addr(true), did_bind(false),
+    global_seq(0),
+    cluster_protocol(0),
+    dispatch_throttler(cct, string("msgr_dispatch_throttler-") + mname,
+                      cct->_conf->ms_dispatch_throttle_bytes),
+    reaper_started(false), reaper_stop(false),
+    timeout(0),
+    local_connection(new PipeConnection(cct, this))
+{
+  ceph_spin_init(&global_seq_lock);
+  init_local_connection();
+}
+
+/**
+ * Destroy the SimpleMessenger. Pretty simple since all the work is done
+ * elsewhere.
+ */
+SimpleMessenger::~SimpleMessenger()
+{
+  assert(!did_bind); // either we didn't bind or we shut down the Accepter
+  assert(rank_pipe.empty()); // we don't have any running Pipes.
+  assert(!reaper_started); // the reaper thread is stopped
+}
+
+void SimpleMessenger::ready()
+{
+  ldout(cct,10) << "ready " << get_myaddr() << dendl;
+  dispatch_queue.start();
+
+  lock.Lock();
+  if (did_bind)
+    accepter.start();
+  lock.Unlock();
+}
+
+
+int SimpleMessenger::shutdown()
+{
+  ldout(cct,10) << "shutdown " << get_myaddr() << dendl;
+  mark_down_all();
+  dispatch_queue.shutdown();
+
+  // break ref cycles on the loopback connection
+  local_connection->set_priv(NULL);
+  return 0;
+}
+
+int SimpleMessenger::_send_message(Message *m, const entity_inst_t& dest)
+{
+  // set envelope
+  m->get_header().src = get_myname();
+
+  if (!m->get_priority()) m->set_priority(get_default_send_priority());
+  ldout(cct,1) <<"--> " << dest.name << " "
+          << dest.addr << " -- " << *m
+         << " -- ?+" << m->get_data().length()
+         << " " << m 
+         << dendl;
+
+  if (dest.addr == entity_addr_t()) {
+    ldout(cct,0) << "send_message message " << *m
+                 << " with empty dest " << dest.addr << dendl;
+    m->put();
+    return -EINVAL;
+  }
+
+  lock.Lock();
+  Pipe *pipe = _lookup_pipe(dest.addr);
+  submit_message(m, (pipe ? pipe->connection_state.get() : NULL),
+                 dest.addr, dest.name.type(), true);
+  lock.Unlock();
+  return 0;
+}
+
+int SimpleMessenger::_send_message(Message *m, Connection *con)
+{
+  //set envelope
+  m->get_header().src = get_myname();
+
+  if (!m->get_priority()) m->set_priority(get_default_send_priority());
+
+  ldout(cct,1) << "--> " << con->get_peer_addr()
+      << " -- " << *m
+      << " -- ?+" << m->get_data().length()
+      << " " << m << " con " << con
+      << dendl;
+
+  submit_message(m, static_cast<PipeConnection*>(con),
+                con->get_peer_addr(), con->get_peer_type(), false);
+  return 0;
+}
+
+/**
+ * If my_inst.addr doesn't have an IP set, this function
+ * will fill it in from the passed addr. Otherwise it does nothing and returns.
+ */
+void SimpleMessenger::set_addr_unknowns(entity_addr_t &addr)
+{
+  if (my_inst.addr.is_blank_ip()) {
+    int port = my_inst.addr.get_port();
+    my_inst.addr.addr = addr.addr;
+    my_inst.addr.set_port(port);
+    init_local_connection();
+  }
+}
+
+int SimpleMessenger::get_proto_version(int peer_type, bool connect)
+{
+  int my_type = my_inst.name.type();
+
+  // set reply protocol version
+  if (peer_type == my_type) {
+    // internal
+    return cluster_protocol;
+  } else {
+    // public
+    if (connect) {
+      switch (peer_type) {
+      case CEPH_ENTITY_TYPE_OSD: return CEPH_OSDC_PROTOCOL;
+      case CEPH_ENTITY_TYPE_MDS: return CEPH_MDSC_PROTOCOL;
+      case CEPH_ENTITY_TYPE_MON: return CEPH_MONC_PROTOCOL;
+      }
+    } else {
+      switch (my_type) {
+      case CEPH_ENTITY_TYPE_OSD: return CEPH_OSDC_PROTOCOL;
+      case CEPH_ENTITY_TYPE_MDS: return CEPH_MDSC_PROTOCOL;
+      case CEPH_ENTITY_TYPE_MON: return CEPH_MONC_PROTOCOL;
+      }
+    }
+  }
+  return 0;
+}
+
+
+
+
+
+
+
+/********************************************
+ * SimpleMessenger
+ */
+#undef dout_prefix
+#define dout_prefix _prefix(_dout, this)
+
+void SimpleMessenger::dispatch_throttle_release(uint64_t msize)
+{
+  if (msize) {
+    ldout(cct,10) << "dispatch_throttle_release " << msize << " to dispatch throttler "
+           << dispatch_throttler.get_current() << "/"
+           << dispatch_throttler.get_max() << dendl;
+    dispatch_throttler.put(msize);
+  }
+}
+
+void SimpleMessenger::reaper_entry()
+{
+  ldout(cct,10) << "reaper_entry start" << dendl;
+  lock.Lock();
+  while (!reaper_stop) {
+    reaper();  // may drop and retake the lock
+    if (reaper_stop)
+      break;
+    reaper_cond.Wait(lock);
+  }
+  lock.Unlock();
+  ldout(cct,10) << "reaper_entry done" << dendl;
+}
+
+/*
+ * note: assumes lock is held
+ */
+void SimpleMessenger::reaper()
+{
+  ldout(cct,10) << "reaper" << dendl;
+  assert(lock.is_locked());
+
+  while (!pipe_reap_queue.empty()) {
+    Pipe *p = pipe_reap_queue.front();
+    pipe_reap_queue.pop_front();
+    ldout(cct,10) << "reaper reaping pipe " << p << " " <<
+      p->get_peer_addr() << dendl;
+    p->pipe_lock.Lock();
+    p->discard_out_queue();
+    if (p->connection_state) {
+      // mark_down, mark_down_all, or fault() should have done this,
+      // or accept() may have switch the Connection to a different
+      // Pipe... but make sure!
+      bool cleared = p->connection_state->clear_pipe(p);
+      assert(!cleared);
+    }
+    p->pipe_lock.Unlock();
+    p->unregister_pipe();
+    assert(pipes.count(p));
+    pipes.erase(p);
+
+    // drop msgr lock while joining thread; the delay through could be
+    // trying to fast dispatch, preventing it from joining without
+    // blocking and deadlocking.
+    lock.Unlock();
+    p->join();
+    lock.Lock();
+
+    if (p->sd >= 0)
+      ::close(p->sd);
+    ldout(cct,10) << "reaper reaped pipe " << p << " " << p->get_peer_addr() << dendl;
+    p->put();
+    ldout(cct,10) << "reaper deleted pipe " << p << dendl;
+  }
+  ldout(cct,10) << "reaper done" << dendl;
+}
+
+void SimpleMessenger::queue_reap(Pipe *pipe)
+{
+  ldout(cct,10) << "queue_reap " << pipe << dendl;
+  lock.Lock();
+  pipe_reap_queue.push_back(pipe);
+  reaper_cond.Signal();
+  lock.Unlock();
+}
+
+
+
+int SimpleMessenger::bind(const entity_addr_t &bind_addr)
+{
+  lock.Lock();
+  if (started) {
+    ldout(cct,10) << "rank.bind already started" << dendl;
+    lock.Unlock();
+    return -1;
+  }
+  ldout(cct,10) << "rank.bind " << bind_addr << dendl;
+  lock.Unlock();
+
+  // bind to a socket
+  set<int> avoid_ports;
+  int r = accepter.bind(bind_addr, avoid_ports);
+  if (r >= 0)
+    did_bind = true;
+  return r;
+}
+
+int SimpleMessenger::rebind(const set<int>& avoid_ports)
+{
+  ldout(cct,1) << "rebind avoid " << avoid_ports << dendl;
+  assert(did_bind);
+  accepter.stop();
+  mark_down_all();
+  return accepter.rebind(avoid_ports);
+}
+
+int SimpleMessenger::start()
+{
+  lock.Lock();
+  ldout(cct,1) << "messenger.start" << dendl;
+
+  // register at least one entity, first!
+  assert(my_inst.name.type() >= 0);
+
+  assert(!started);
+  started = true;
+
+  if (!did_bind) {
+    my_inst.addr.nonce = nonce;
+    init_local_connection();
+  }
+
+  lock.Unlock();
+
+  reaper_started = true;
+  reaper_thread.create();
+  return 0;
+}
+
+Pipe *SimpleMessenger::add_accept_pipe(int sd)
+{
+  lock.Lock();
+  Pipe *p = new Pipe(this, Pipe::STATE_ACCEPTING, NULL);
+  p->sd = sd;
+  p->pipe_lock.Lock();
+  p->start_reader();
+  p->pipe_lock.Unlock();
+  pipes.insert(p);
+  accepting_pipes.insert(p);
+  lock.Unlock();
+  return p;
+}
+
+/* connect_rank
+ * NOTE: assumes messenger.lock held.
+ */
+Pipe *SimpleMessenger::connect_rank(const entity_addr_t& addr,
+                                   int type,
+                                   PipeConnection *con,
+                                   Message *first)
+{
+  assert(lock.is_locked());
+  assert(addr != my_inst.addr);
+  
+  ldout(cct,10) << "connect_rank to " << addr << ", creating pipe and registering" << dendl;
+  
+  // create pipe
+  Pipe *pipe = new Pipe(this, Pipe::STATE_CONNECTING,
+                       static_cast<PipeConnection*>(con));
+  pipe->pipe_lock.Lock();
+  pipe->set_peer_type(type);
+  pipe->set_peer_addr(addr);
+  pipe->policy = get_policy(type);
+  pipe->start_writer();
+  if (first)
+    pipe->_send(first);
+  pipe->pipe_lock.Unlock();
+  pipe->register_pipe();
+  pipes.insert(pipe);
+
+  return pipe;
+}
+
+
+
+
+
+
+AuthAuthorizer *SimpleMessenger::get_authorizer(int peer_type, bool force_new)
+{
+  return ms_deliver_get_authorizer(peer_type, force_new);
+}
+
+bool SimpleMessenger::verify_authorizer(Connection *con, int peer_type,
+                                       int protocol, bufferlist& authorizer, bufferlist& authorizer_reply,
+                                       bool& isvalid,CryptoKey& session_key)
+{
+  return ms_deliver_verify_authorizer(con, peer_type, protocol, authorizer, authorizer_reply, isvalid,session_key);
+}
+
+ConnectionRef SimpleMessenger::get_connection(const entity_inst_t& dest)
+{
+  Mutex::Locker l(lock);
+  if (my_inst.addr == dest.addr) {
+    // local
+    return local_connection;
+  }
+
+  // remote
+  while (true) {
+    Pipe *pipe = _lookup_pipe(dest.addr);
+    if (pipe) {
+      ldout(cct, 10) << "get_connection " << dest << " existing " << pipe << dendl;
+    } else {
+      pipe = connect_rank(dest.addr, dest.name.type(), NULL, NULL);
+      ldout(cct, 10) << "get_connection " << dest << " new " << pipe << dendl;
+    }
+    Mutex::Locker l(pipe->pipe_lock);
+    if (pipe->connection_state)
+      return pipe->connection_state;
+    // we failed too quickly!  retry.  FIXME.
+  }
+}
+
+ConnectionRef SimpleMessenger::get_loopback_connection()
+{
+  return local_connection;
+}
+
+void SimpleMessenger::submit_message(Message *m, PipeConnection *con,
+                                    const entity_addr_t& dest_addr, int dest_type,
+                                    bool already_locked)
+{
+  if (cct->_conf->ms_dump_on_send) {
+    m->encode(-1, true);
+    ldout(cct, 0) << "submit_message " << *m << "\n";
+    m->get_payload().hexdump(*_dout);
+    if (m->get_data().length() > 0) {
+      *_dout << " data:\n";
+      m->get_data().hexdump(*_dout);
+    }
+    *_dout << dendl;
+    m->clear_payload();
+  }
+
+  // existing connection?
+  if (con) {
+    Pipe *pipe = NULL;
+    bool ok = static_cast<PipeConnection*>(con)->try_get_pipe(&pipe);
+    if (!ok) {
+      ldout(cct,0) << "submit_message " << *m << " remote, " << dest_addr
+                  << ", failed lossy con, dropping message " << m << dendl;
+      m->put();
+      return;
+    }
+    while (pipe && ok) {
+      // we loop in case of a racing reconnect, either from us or them
+      pipe->pipe_lock.Lock(); // can't use a Locker because of the Pipe ref
+      if (pipe->state != Pipe::STATE_CLOSED) {
+       ldout(cct,20) << "submit_message " << *m << " remote, " << dest_addr << ", have pipe." << dendl;
+       pipe->_send(m);
+       pipe->pipe_lock.Unlock();
+       pipe->put();
+       return;
+      }
+      Pipe *current_pipe;
+      ok = con->try_get_pipe(&current_pipe);
+      pipe->pipe_lock.Unlock();
+      if (current_pipe == pipe) {
+       ldout(cct,20) << "submit_message " << *m << " remote, " << dest_addr
+                     << ", had pipe " << pipe << ", but it closed." << dendl;
+       pipe->put();
+       current_pipe->put();
+       m->put();
+       return;
+      } else {
+       pipe->put();
+       pipe = current_pipe;
+      }
+    }
+  }
+
+  // local?
+  if (my_inst.addr == dest_addr) {
+    // local
+    ldout(cct,20) << "submit_message " << *m << " local" << dendl;
+    dispatch_queue.local_delivery(m, m->get_priority());
+    return;
+  }
+
+  // remote, no existing pipe.
+  const Policy& policy = get_policy(dest_type);
+  if (policy.server) {
+    ldout(cct,20) << "submit_message " << *m << " remote, " << dest_addr << ", lossy server for target type "
+                 << ceph_entity_type_name(dest_type) << ", no session, dropping." << dendl;
+    m->put();
+  } else {
+    ldout(cct,20) << "submit_message " << *m << " remote, " << dest_addr << ", new pipe." << dendl;
+    if (!already_locked) {
+      /** We couldn't handle the Message without reference to global data, so
+       *  grab the lock and do it again. If we got here, we know it's a non-lossy
+       *  Connection, so we can use our existing pointer without doing another lookup. */
+      Mutex::Locker l(lock);
+      submit_message(m, con, dest_addr, dest_type, true);
+    } else {
+      connect_rank(dest_addr, dest_type, static_cast<PipeConnection*>(con), m);
+    }
+  }
+}
+
+int SimpleMessenger::send_keepalive(Connection *con)
+{
+  int ret = 0;
+  Pipe *pipe = static_cast<Pipe *>(
+    static_cast<PipeConnection*>(con)->get_pipe());
+  if (pipe) {
+    ldout(cct,20) << "send_keepalive con " << con << ", have pipe." << dendl;
+    assert(pipe->msgr == this);
+    pipe->pipe_lock.Lock();
+    pipe->_send_keepalive();
+    pipe->pipe_lock.Unlock();
+    pipe->put();
+  } else {
+    ldout(cct,0) << "send_keepalive con " << con << ", no pipe." << dendl;
+    ret = -EPIPE;
+  }
+  return ret;
+}
+
+
+
+void SimpleMessenger::wait()
+{
+  lock.Lock();
+  if (!started) {
+    lock.Unlock();
+    return;
+  }
+  lock.Unlock();
+
+  if(dispatch_queue.is_started()) {
+    ldout(cct,10) << "wait: waiting for dispatch queue" << dendl;
+    dispatch_queue.wait();
+    ldout(cct,10) << "wait: dispatch queue is stopped" << dendl;
+  }
+
+  // done!  clean up.
+  if (did_bind) {
+    ldout(cct,20) << "wait: stopping accepter thread" << dendl;
+    accepter.stop();
+    did_bind = false;
+    ldout(cct,20) << "wait: stopped accepter thread" << dendl;
+  }
+
+  if (reaper_started) {
+    ldout(cct,20) << "wait: stopping reaper thread" << dendl;
+    lock.Lock();
+    reaper_cond.Signal();
+    reaper_stop = true;
+    lock.Unlock();
+    reaper_thread.join();
+    reaper_started = false;
+    ldout(cct,20) << "wait: stopped reaper thread" << dendl;
+  }
+
+  // close+reap all pipes
+  lock.Lock();
+  {
+    ldout(cct,10) << "wait: closing pipes" << dendl;
+
+    while (!rank_pipe.empty()) {
+      Pipe *p = rank_pipe.begin()->second;
+      p->unregister_pipe();
+      p->pipe_lock.Lock();
+      p->stop_and_wait();
+      p->pipe_lock.Unlock();
+    }
+
+    reaper();
+    ldout(cct,10) << "wait: waiting for pipes " << pipes << " to close" << dendl;
+    while (!pipes.empty()) {
+      reaper_cond.Wait(lock);
+      reaper();
+    }
+  }
+  lock.Unlock();
+
+  ldout(cct,10) << "wait: done." << dendl;
+  ldout(cct,1) << "shutdown complete." << dendl;
+  started = false;
+}
+
+
+void SimpleMessenger::mark_down_all()
+{
+  ldout(cct,1) << "mark_down_all" << dendl;
+  lock.Lock();
+  for (set<Pipe*>::iterator q = accepting_pipes.begin(); q != accepting_pipes.end(); ++q) {
+    Pipe *p = *q;
+    ldout(cct,5) << "mark_down_all accepting_pipe " << p << dendl;
+    p->pipe_lock.Lock();
+    p->stop_and_wait();
+    PipeConnectionRef con = p->connection_state;
+    if (con && con->clear_pipe(p))
+      dispatch_queue.queue_reset(con.get());
+    p->pipe_lock.Unlock();
+  }
+  accepting_pipes.clear();
+
+  while (!rank_pipe.empty()) {
+    ceph::unordered_map<entity_addr_t,Pipe*>::iterator it = rank_pipe.begin();
+    Pipe *p = it->second;
+    ldout(cct,5) << "mark_down_all " << it->first << " " << p << dendl;
+    rank_pipe.erase(it);
+    p->unregister_pipe();
+    p->pipe_lock.Lock();
+    p->stop_and_wait();
+    PipeConnectionRef con = p->connection_state;
+    if (con && con->clear_pipe(p))
+      dispatch_queue.queue_reset(con.get());
+    p->pipe_lock.Unlock();
+  }
+  lock.Unlock();
+}
+
+void SimpleMessenger::mark_down(const entity_addr_t& addr)
+{
+  lock.Lock();
+  Pipe *p = _lookup_pipe(addr);
+  if (p) {
+    ldout(cct,1) << "mark_down " << addr << " -- " << p << dendl;
+    p->unregister_pipe();
+    p->pipe_lock.Lock();
+    p->stop_and_wait();
+    if (p->connection_state) {
+      // generate a reset event for the caller in this case, even
+      // though they asked for it, since this is the addr-based (and
+      // not Connection* based) interface
+      PipeConnectionRef con = p->connection_state;
+      if (con && con->clear_pipe(p))
+       dispatch_queue.queue_reset(con.get());
+    }
+    p->pipe_lock.Unlock();
+  } else {
+    ldout(cct,1) << "mark_down " << addr << " -- pipe dne" << dendl;
+  }
+  lock.Unlock();
+}
+
+void SimpleMessenger::mark_down(Connection *con)
+{
+  if (con == NULL)
+    return;
+  lock.Lock();
+  Pipe *p = static_cast<Pipe *>(static_cast<PipeConnection*>(con)->get_pipe());
+  if (p) {
+    ldout(cct,1) << "mark_down " << con << " -- " << p << dendl;
+    assert(p->msgr == this);
+    p->unregister_pipe();
+    p->pipe_lock.Lock();
+    p->stop_and_wait();
+    if (p->connection_state) {
+      // do not generate a reset event for the caller in this case,
+      // since they asked for it.
+      p->connection_state->clear_pipe(p);
+    }
+    p->pipe_lock.Unlock();
+    p->put();
+  } else {
+    ldout(cct,1) << "mark_down " << con << " -- pipe dne" << dendl;
+  }
+  lock.Unlock();
+}
+
+void SimpleMessenger::mark_disposable(Connection *con)
+{
+  lock.Lock();
+  Pipe *p = static_cast<Pipe *>(static_cast<PipeConnection*>(con)->get_pipe());
+  if (p) {
+    ldout(cct,1) << "mark_disposable " << con << " -- " << p << dendl;
+    assert(p->msgr == this);
+    p->pipe_lock.Lock();
+    p->policy.lossy = true;
+    p->pipe_lock.Unlock();
+    p->put();
+  } else {
+    ldout(cct,1) << "mark_disposable " << con << " -- pipe dne" << dendl;
+  }
+  lock.Unlock();
+}
+
+void SimpleMessenger::learned_addr(const entity_addr_t &peer_addr_for_me)
+{
+  // be careful here: multiple threads may block here, and readers of
+  // my_inst.addr do NOT hold any lock.
+
+  // this always goes from true -> false under the protection of the
+  // mutex.  if it is already false, we need not retake the mutex at
+  // all.
+  if (!need_addr)
+    return;
+
+  lock.Lock();
+  if (need_addr) {
+    entity_addr_t t = peer_addr_for_me;
+    t.set_port(my_inst.addr.get_port());
+    my_inst.addr.addr = t.addr;
+    ldout(cct,1) << "learned my addr " << my_inst.addr << dendl;
+    need_addr = false;
+    init_local_connection();
+  }
+  lock.Unlock();
+}
+
+void SimpleMessenger::init_local_connection()
+{
+  local_connection->peer_addr = my_inst.addr;
+  local_connection->peer_type = my_inst.name.type();
+  ms_deliver_handle_fast_connect(local_connection.get());
+}
diff --git a/src/msg/simple/SimpleMessenger.h b/src/msg/simple/SimpleMessenger.h
new file mode 100644 (file)
index 0000000..bb63dbe
--- /dev/null
@@ -0,0 +1,417 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software 
+ * Foundation.  See file COPYING.
+ * 
+ */
+
+#ifndef CEPH_SIMPLEMESSENGER_H
+#define CEPH_SIMPLEMESSENGER_H
+
+#include "include/types.h"
+#include "include/xlist.h"
+
+#include <list>
+#include <map>
+using namespace std;
+#include "include/unordered_map.h"
+#include "include/unordered_set.h"
+
+#include "common/Mutex.h"
+#include "include/atomic.h"
+#include "include/Spinlock.h"
+#include "common/Cond.h"
+#include "common/Thread.h"
+#include "common/Throttle.h"
+
+#include "msg/SimplePolicyMessenger.h"
+#include "msg/Message.h"
+#include "include/assert.h"
+
+#include "DispatchQueue.h"
+#include "Pipe.h"
+#include "Accepter.h"
+
+/*
+ * This class handles transmission and reception of messages. Generally
+ * speaking, there are several major components:
+ *
+ * - Connection
+ *    Each logical session is associated with a Connection.
+ * - Pipe
+ *    Each network connection is handled through a pipe, which handles
+ *    the input and output of each message.  There is normally a 1:1
+ *    relationship between Pipe and Connection, but logical sessions may
+ *    get handed off between Pipes when sockets reconnect or during
+ *    connection races.
+ * - IncomingQueue
+ *    Incoming messages are associated with an IncomingQueue, and there
+ *    is one such queue associated with each Pipe.
+ * - DispatchQueue
+ *    IncomingQueues get queued in the DIspatchQueue, which is responsible
+ *    for doing a round-robin sweep and processing them via a worker thread.
+ * - SimpleMessenger
+ *    It's the exterior class passed to the external message handler and
+ *    most of the API details.
+ *
+ * Lock ordering:
+ *
+ *   SimpleMessenger::lock
+ *       Pipe::pipe_lock
+ *           DispatchQueue::lock
+ *               IncomingQueue::lock
+ */
+
+class SimpleMessenger : public SimplePolicyMessenger {
+  // First we have the public Messenger interface implementation...
+public:
+  /**
+   * Initialize the SimpleMessenger!
+   *
+   * @param cct The CephContext to use
+   * @param name The name to assign ourselves
+   * _nonce A unique ID to use for this SimpleMessenger. It should not
+   * be a value that will be repeated if the daemon restarts.
+   */
+  SimpleMessenger(CephContext *cct, entity_name_t name,
+                 string mname, uint64_t _nonce);
+
+  /**
+   * Destroy the SimpleMessenger. Pretty simple since all the work is done
+   * elsewhere.
+   */
+  virtual ~SimpleMessenger();
+
+  /** @defgroup Accessors
+   * @{
+   */
+  void set_addr_unknowns(entity_addr_t& addr);
+
+  int get_dispatch_queue_len() {
+    return dispatch_queue.get_queue_len();
+  }
+
+  double get_dispatch_queue_max_age(utime_t now) {
+    return dispatch_queue.get_max_age(now);
+  }
+  /** @} Accessors */
+
+  /**
+   * @defgroup Configuration functions
+   * @{
+   */
+  void set_cluster_protocol(int p) {
+    assert(!started && !did_bind);
+    cluster_protocol = p;
+  }
+
+  int bind(const entity_addr_t& bind_addr);
+  int rebind(const set<int>& avoid_ports);
+
+  /** @} Configuration functions */
+
+  /**
+   * @defgroup Startup/Shutdown
+   * @{
+   */
+  virtual int start();
+  virtual void wait();
+  virtual int shutdown();
+
+  /** @} // Startup/Shutdown */
+
+  /**
+   * @defgroup Messaging
+   * @{
+   */
+  virtual int send_message(Message *m, const entity_inst_t& dest) {
+    return _send_message(m, dest);
+  }
+
+  int send_message(Message *m, Connection *con) {
+    return _send_message(m, con);
+  }
+
+  /** @} // Messaging */
+
+  /**
+   * @defgroup Connection Management
+   * @{
+   */
+  virtual ConnectionRef get_connection(const entity_inst_t& dest);
+  virtual ConnectionRef get_loopback_connection();
+  int send_keepalive(Connection *con);
+  virtual void mark_down(const entity_addr_t& addr);
+  void mark_down(Connection *con);
+  void mark_disposable(Connection *con);
+  virtual void mark_down_all();
+  /** @} // Connection Management */
+protected:
+  /**
+   * @defgroup Messenger Interfaces
+   * @{
+   */
+  /**
+   * Start up the DispatchQueue thread once we have somebody to dispatch to.
+   */
+  virtual void ready();
+  /** @} // Messenger Interfaces */
+private:
+  /**
+   * @defgroup Inner classes
+   * @{
+   */
+
+public:
+  Accepter accepter;
+  DispatchQueue dispatch_queue;
+
+  friend class Accepter;
+
+  /**
+   * Register a new pipe for accept
+   *
+   * @param sd socket
+   */
+  Pipe *add_accept_pipe(int sd);
+
+  Connection *create_anon_connection() {
+    return new PipeConnection(cct, NULL);
+  }
+
+private:
+
+  /**
+   * A thread used to tear down Pipes when they're complete.
+   */
+  class ReaperThread : public Thread {
+    SimpleMessenger *msgr;
+  public:
+    ReaperThread(SimpleMessenger *m) : msgr(m) {}
+    void *entry() {
+      msgr->reaper_entry();
+      return 0;
+    }
+  } reaper_thread;
+
+  /**
+   * @} // Inner classes
+   */
+
+  /**
+   * @defgroup Utility functions
+   * @{
+   */
+
+  /**
+   * Create a Pipe associated with the given entity (of the given type).
+   * Initiate the connection. (This function returning does not guarantee
+   * connection success.)
+   *
+   * @param addr The address of the entity to connect to.
+   * @param type The peer type of the entity at the address.
+   * @param con An existing Connection to associate with the new Pipe. If
+   * NULL, it creates a new Connection.
+   * @param msg an initial message to queue on the new pipe
+   *
+   * @return a pointer to the newly-created Pipe. Caller does not own a
+   * reference; take one if you need it.
+   */
+  Pipe *connect_rank(const entity_addr_t& addr, int type, PipeConnection *con,
+                    Message *first);
+  /**
+   * Send a message, lazily or not.
+   * This just glues send_message together and passes
+   * the input on to submit_message.
+   */
+  int _send_message(Message *m, const entity_inst_t& dest);
+  /**
+   * Same as above, but for the Connection-based variants.
+   */
+  int _send_message(Message *m, Connection *con);
+  /**
+   * Queue up a Message for delivery to the entity specified
+   * by addr and dest_type.
+   * submit_message() is responsible for creating
+   * new Pipes (and closing old ones) as necessary.
+   *
+   * @param m The Message to queue up. This function eats a reference.
+   * @param con The existing Connection to use, or NULL if you don't know of one.
+   * @param addr The address to send the Message to.
+   * @param dest_type The peer type of the address we're sending to
+   * just drop silently under failure.
+   * @param already_locked If false, submit_message() will acquire the
+   * SimpleMessenger lock before accessing shared data structures; otherwise
+   * it will assume the lock is held. NOTE: if you are making a request
+   * without locking, you MUST have filled in the con with a valid pointer.
+   */
+  void submit_message(Message *m, PipeConnection *con,
+                     const entity_addr_t& addr, int dest_type,
+                     bool already_locked);
+  /**
+   * Look through the pipes in the pipe_reap_queue and tear them down.
+   */
+  void reaper();
+  /**
+   * @} // Utility functions
+   */
+
+  // SimpleMessenger stuff
+  /// approximately unique ID set by the Constructor for use in entity_addr_t
+  uint64_t nonce;
+  /// overall lock used for SimpleMessenger data structures
+  Mutex lock;
+  /// true, specifying we haven't learned our addr; set false when we find it.
+  // maybe this should be protected by the lock?
+  bool need_addr;
+
+public:
+  bool get_need_addr() const { return need_addr; }
+
+private:
+  /**
+   *  false; set to true if the SimpleMessenger bound to a specific address;
+   *  and set false again by Accepter::stop(). This isn't lock-protected
+   *  since you shouldn't be able to race the only writers.
+   */
+  bool did_bind;
+  /// counter for the global seq our connection protocol uses
+  __u32 global_seq;
+  /// lock to protect the global_seq
+  ceph_spinlock_t global_seq_lock;
+
+  /**
+   * hash map of addresses to Pipes
+   *
+   * NOTE: a Pipe* with state CLOSED may still be in the map but is considered
+   * invalid and can be replaced by anyone holding the msgr lock
+   */
+  ceph::unordered_map<entity_addr_t, Pipe*> rank_pipe;
+  /**
+   * list of pipes are in teh process of accepting
+   *
+   * These are not yet in the rank_pipe map.
+   */
+  set<Pipe*> accepting_pipes;
+  /// a set of all the Pipes we have which are somehow active
+  set<Pipe*>      pipes;
+  /// a list of Pipes we want to tear down
+  list<Pipe*>     pipe_reap_queue;
+
+  /// internal cluster protocol version, if any, for talking to entities of the same type.
+  int cluster_protocol;
+
+  /// Throttle preventing us from building up a big backlog waiting for dispatch
+  Throttle dispatch_throttler;
+
+  bool reaper_started, reaper_stop;
+  Cond reaper_cond;
+
+  /// This Cond is slept on by wait() and signaled by dispatch_entry()
+  Cond  wait_cond;
+
+  friend class Pipe;
+
+  Pipe *_lookup_pipe(const entity_addr_t& k) {
+    ceph::unordered_map<entity_addr_t, Pipe*>::iterator p = rank_pipe.find(k);
+    if (p == rank_pipe.end())
+      return NULL;
+    // see lock cribbing in Pipe::fault()
+    if (p->second->state_closed.read())
+      return NULL;
+    return p->second;
+  }
+
+public:
+
+  int timeout;
+
+  /// con used for sending messages to ourselves
+  ConnectionRef local_connection;
+
+  /**
+   * @defgroup SimpleMessenger internals
+   * @{
+   */
+
+  /**
+   * This wraps ms_deliver_get_authorizer. We use it for Pipe.
+   */
+  AuthAuthorizer *get_authorizer(int peer_type, bool force_new);
+  /**
+   * This wraps ms_deliver_verify_authorizer; we use it for Pipe.
+   */
+  bool verify_authorizer(Connection *con, int peer_type, int protocol, bufferlist& auth, bufferlist& auth_reply,
+                         bool& isvalid,CryptoKey& session_key);
+  /**
+   * Increment the global sequence for this SimpleMessenger and return it.
+   * This is for the connect protocol, although it doesn't hurt if somebody
+   * else calls it.
+   *
+   * @return a global sequence ID that nobody else has seen.
+   */
+  __u32 get_global_seq(__u32 old=0) {
+    ceph_spin_lock(&global_seq_lock);
+    if (old > global_seq)
+      global_seq = old;
+    __u32 ret = ++global_seq;
+    ceph_spin_unlock(&global_seq_lock);
+    return ret;
+  }
+  /**
+   * Get the protocol version we support for the given peer type: either
+   * a peer protocol (if it matches our own), the protocol version for the
+   * peer (if we're connecting), or our protocol version (if we're accepting).
+   */
+  int get_proto_version(int peer_type, bool connect);
+
+  /**
+   * Fill in the address and peer type for the local connection, which
+   * is used for delivering messages back to ourself.
+   */
+  void init_local_connection();
+  /**
+   * Tell the SimpleMessenger its full IP address.
+   *
+   * This is used by Pipes when connecting to other endpoints, and
+   * probably shouldn't be called by anybody else.
+   */
+  void learned_addr(const entity_addr_t& peer_addr_for_me);
+
+  /**
+   * Release memory accounting back to the dispatch throttler.
+   *
+   * @param msize The amount of memory to release.
+   */
+  void dispatch_throttle_release(uint64_t msize);
+
+  /**
+   * This function is used by the reaper thread. As long as nobody
+   * has set reaper_stop, it calls the reaper function, then
+   * waits to be signaled when it needs to reap again (or when it needs
+   * to stop).
+   */
+  void reaper_entry();
+  /**
+   * Add a pipe to the pipe_reap_queue, to be torn down on
+   * the next call to reaper().
+   * It should really only be the Pipe calling this, in our current
+   * implementation.
+   *
+   * @param pipe A Pipe which has stopped its threads and is
+   * ready to be torn down.
+   */
+  void queue_reap(Pipe *pipe);
+  /**
+   * @} // SimpleMessenger Internals
+   */
+} ;
+
+#endif /* CEPH_SIMPLEMESSENGER_H */