Memory-based transport rewrite.

The old implementations of the memory-based transports (TBufferedTransport,
TFramedTransport, and TMemoryBuffer) shared very little code and all worked
a bit differently.  This change unifies them all as subclasses of a single
base (TBufferBase) which handles the fast-path operations (when requests
can be satisfied by the buffer) with inline methods (that will eventually
be made nonvirtual in the template branch) and calls out to pure-virutal
methods to handle full/empty buffers.  All of the buffer-management is now
done in terms of "base and bound" pointers rather than "pos" integers.

These classes were moved to TBufferTransports.{h,cpp}.  The .h is included
in TTransportUtils for backwards compatibility.

Also added a "TShortReadTransport" to assist testing transports.


git-svn-id: https://svn.apache.org/repos/asf/incubator/thrift/trunk@665676 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/lib/cpp/Makefile.am b/lib/cpp/Makefile.am
index ac5a776..df64353 100644
--- a/lib/cpp/Makefile.am
+++ b/lib/cpp/Makefile.am
@@ -41,6 +41,7 @@
                        src/transport/TSocketPool.cpp \
                        src/transport/TServerSocket.cpp \
                        src/transport/TTransportUtils.cpp \
+                       src/transport/TBufferTransports.cpp \
                        src/server/TServer.cpp \
                        src/server/TSimpleServer.cpp \
                        src/server/TThreadPoolServer.cpp \
@@ -107,6 +108,7 @@
                          src/transport/TTransport.h \
                          src/transport/TTransportException.h \
                          src/transport/TTransportUtils.h \
+                         src/transport/TBufferTransports.h \
                          src/transport/TZlibTransport.h
 
 include_serverdir = $(include_thriftdir)/server
diff --git a/lib/cpp/src/processor/PeekProcessor.h b/lib/cpp/src/processor/PeekProcessor.h
index 49e7cd9..697be27 100644
--- a/lib/cpp/src/processor/PeekProcessor.h
+++ b/lib/cpp/src/processor/PeekProcessor.h
@@ -11,6 +11,7 @@
 #include <TProcessor.h>
 #include <transport/TTransport.h>
 #include <transport/TTransportUtils.h>
+#include <transport/TBufferTransports.h>
 #include <boost/shared_ptr.hpp>
 
 namespace facebook { namespace thrift { namespace processor {
diff --git a/lib/cpp/src/protocol/TDebugProtocol.h b/lib/cpp/src/protocol/TDebugProtocol.h
index bac2098..c08720b 100644
--- a/lib/cpp/src/protocol/TDebugProtocol.h
+++ b/lib/cpp/src/protocol/TDebugProtocol.h
@@ -12,8 +12,6 @@
 
 #include <boost/shared_ptr.hpp>
 
-#include <transport/TTransportUtils.h>
-
 namespace facebook { namespace thrift { namespace protocol {
 
 /*
@@ -158,6 +156,9 @@
 }}} // facebook::thrift::protocol
 
 
+// TODO(dreiss): Move (part of) ThriftDebugString into a .cpp file and remove this.
+#include <transport/TBufferTransports.h>
+
 namespace facebook { namespace thrift {
 
 template<typename ThriftStruct>
diff --git a/lib/cpp/src/protocol/TJSONProtocol.h b/lib/cpp/src/protocol/TJSONProtocol.h
index efb8e63..cece97e 100644
--- a/lib/cpp/src/protocol/TJSONProtocol.h
+++ b/lib/cpp/src/protocol/TJSONProtocol.h
@@ -8,7 +8,6 @@
 #define _THRIFT_PROTOCOL_TJSONPROTOCOL_H_ 1
 
 #include "TProtocol.h"
-#include <transport/TTransportUtils.h>
 
 #include <stack>
 
@@ -303,6 +302,9 @@
 }}} // facebook::thrift::protocol
 
 
+// TODO(dreiss): Move part of ThriftJSONString into a .cpp file and remove this.
+#include <transport/TBufferTransports.h>
+
 namespace facebook { namespace thrift {
 
 template<typename ThriftStruct>
diff --git a/lib/cpp/src/server/TNonblockingServer.h b/lib/cpp/src/server/TNonblockingServer.h
index de2e43b..3de5b94 100644
--- a/lib/cpp/src/server/TNonblockingServer.h
+++ b/lib/cpp/src/server/TNonblockingServer.h
@@ -9,7 +9,7 @@
 
 #include <Thrift.h>
 #include <server/TServer.h>
-#include <transport/TTransportUtils.h>
+#include <transport/TBufferTransports.h>
 #include <concurrency/ThreadManager.h>
 #include <stack>
 #include <string>
diff --git a/lib/cpp/src/transport/TBufferTransports.cpp b/lib/cpp/src/transport/TBufferTransports.cpp
new file mode 100644
index 0000000..9164a1f
--- /dev/null
+++ b/lib/cpp/src/transport/TBufferTransports.cpp
@@ -0,0 +1,353 @@
+// Copyright (c) 2006- Facebook
+// Distributed under the Thrift Software License
+//
+// See accompanying file LICENSE or visit the Thrift site at:
+// http://developers.facebook.com/thrift/
+
+#include <cassert>
+#include <algorithm>
+
+#include <transport/TBufferTransports.h>
+
+using std::string;
+
+namespace facebook { namespace thrift { namespace transport {
+
+
+uint32_t TBufferedTransport::readSlow(uint8_t* buf, uint32_t len) {
+  uint32_t want = len;
+  uint32_t have = rBound_ - rBase_;
+
+  // We should only take the slow path if we can't satisfy the read
+  // with the data already in the buffer.
+  assert(have < want);
+
+  // Copy out whatever we have.
+  if (have > 0) {
+    memcpy(buf, rBase_, have);
+    want -= have;
+    buf += have;
+  }
+  // Get more from underlying transport up to buffer size.
+  // Note that this makes a lot of sense if len < rBufSize_
+  // and almost no sense otherwise.  TODO(dreiss): Fix that
+  // case (possibly including some readv hotness).
+  setReadBuffer(rBuf_.get(), transport_->read(rBuf_.get(), rBufSize_));
+
+  // Hand over whatever we have.
+  uint32_t give = std::min(want, static_cast<uint32_t>(rBound_ - rBase_));
+  memcpy(buf, rBase_, give);
+  rBase_ += give;
+  want -= give;
+
+  return (len - want);
+}
+
+void TBufferedTransport::writeSlow(const uint8_t* buf, uint32_t len) {
+  uint32_t have_bytes = wBase_ - wBuf_.get();
+  uint32_t space = wBound_ - wBase_;
+  // We should only take the slow path if we can't accomodate the write
+  // with the free space already in the buffer.
+  assert(wBound_ - wBase_ < static_cast<ptrdiff_t>(len));
+
+  // Now here's the tricky question: should we copy data from buf into our
+  // internal buffer and write it from there, or should we just write out
+  // the current internal buffer in one syscall and write out buf in another.
+  // If our currently buffered data plus buf is at least double our buffer
+  // size, we will have to do two syscalls no matter what (except in the
+  // degenerate case when our buffer is empty), so there is no use copying.
+  // Otherwise, there is sort of a sliding scale.  If we have N-1 bytes
+  // buffered and need to write 2, it would be crazy to do two syscalls.
+  // On the other hand, if we have 2 bytes buffered and are writing 2N-3,
+  // we can save a syscall in the short term by loading up our buffer, writing
+  // it out, and copying the rest of the bytes into our buffer.  Of course,
+  // if we get another 2-byte write, we haven't saved any syscalls at all,
+  // and have just copied nearly 2N bytes for nothing.  Finding a perfect
+  // policy would require predicting the size of future writes, so we're just
+  // going to always eschew syscalls if we have less than 2N bytes to write.
+
+  // The case where we have to do two syscalls.
+  // This case also covers the case where the buffer is empty,
+  // but it is clearer (I think) to think of it as two separate cases.
+  if ((have_bytes + len >= 2*wBufSize_) || (have_bytes == 0)) {
+    // TODO(dreiss): writev
+    if (have_bytes > 0) {
+      transport_->write(wBuf_.get(), have_bytes);
+    }
+    transport_->write(buf, len);
+    wBase_ = wBuf_.get();
+    return;
+  }
+
+  // Fill up our internal buffer for a write.
+  memcpy(wBase_, buf, space);
+  buf += space;
+  len -= space;
+  transport_->write(wBuf_.get(), wBufSize_);
+
+  // Copy the rest into our buffer.
+  assert(len < wBufSize_);
+  memcpy(wBuf_.get(), buf, len);
+  wBase_ = wBuf_.get() + len;
+  return;
+}
+
+const uint8_t* TBufferedTransport::borrowSlow(uint8_t* buf, uint32_t* len) {
+  // If the request is bigger than our buffer, we are hosed.
+  if (*len > rBufSize_) {
+    return NULL;
+  }
+
+  // The number of bytes of data we have already.
+  uint32_t have = rBound_ - rBase_;
+  // The number of additional bytes we need from the underlying transport.
+  uint32_t need = *len - have;
+  // The space from the start of the buffer to the end of our data.
+  uint32_t offset = rBound_ - rBuf_.get();
+  assert(need > 0);
+
+  // If we have less than half our buffer space available, shift the data
+  // we have down to the start.  If the borrow is big compared to our buffer,
+  // this could be kind of a waste, but if the borrow is small, it frees up
+  // space at the end of our buffer to do a bigger single read from the
+  // underlying transport.  Also, if our needs extend past the end of the
+  // buffer, we have to do a copy no matter what.
+  if ((offset > rBufSize_/2) || (offset + need > rBufSize_)) {
+    memmove(rBuf_.get(), rBase_, have);
+    setReadBuffer(rBuf_.get(), have);
+  }
+
+  // First try to fill up the buffer.
+  uint32_t got = transport_->read(rBound_, rBufSize_ - have);
+  rBound_ += got;
+  need -= got;
+
+  // If that fails, readAll until we get what we need.
+  if (need > 0) {
+    rBound_ += transport_->readAll(rBound_, need);
+  }
+
+  *len = rBound_ - rBase_;
+  return rBase_;
+}
+
+void TBufferedTransport::flush()  {
+  // Write out any data waiting in the write buffer.
+  uint32_t have_bytes = wBase_ - wBuf_.get();
+  if (have_bytes > 0) {
+    transport_->write(wBuf_.get(), have_bytes);
+    wBase_ = wBuf_.get();
+  }
+
+  // Flush the underlying transport.
+  transport_->flush();
+}
+
+
+uint32_t TFramedTransport::readSlow(uint8_t* buf, uint32_t len) {
+  uint32_t want = len;
+  uint32_t have = rBound_ - rBase_;
+
+  // We should only take the slow path if we can't satisfy the read
+  // with the data already in the buffer.
+  assert(have < want);
+
+  // Copy out whatever we have.
+  if (have > 0) {
+    memcpy(buf, rBase_, have);
+    want -= have;
+    buf += have;
+  }
+
+  // Read another frame.
+  readFrame();
+
+  // TODO(dreiss): Should we warn when reads cross frames?
+
+  // Hand over whatever we have.
+  uint32_t give = std::min(want, static_cast<uint32_t>(rBound_ - rBase_));
+  memcpy(buf, rBase_, give);
+  rBase_ += give;
+  want -= give;
+
+  return (len - want);
+}
+
+void TFramedTransport::readFrame() {
+  // TODO(dreiss): Think about using readv here, even though it would
+  // result in (gasp) read-ahead.
+
+  // Read the size of the next frame.
+  int32_t sz;
+  transport_->readAll((uint8_t*)&sz, sizeof(sz));
+  sz = ntohl(sz);
+
+  if (sz < 0) {
+    throw TTransportException("Frame size has negative value");
+  }
+
+  // Read the frame payload, and reset markers.
+  if (sz > static_cast<int32_t>(rBufSize_)) {
+    rBuf_.reset(new uint8_t[sz]);
+    rBufSize_ = sz;
+  }
+  transport_->readAll(rBuf_.get(), sz);
+  setReadBuffer(rBuf_.get(), sz);
+}
+
+void TFramedTransport::writeSlow(const uint8_t* buf, uint32_t len) {
+  // Double buffer size until sufficient.
+  uint32_t have = wBase_ - wBuf_.get();
+  while (wBufSize_ < len + have) {
+    wBufSize_ *= 2;
+  }
+
+  // TODO(dreiss): Consider modifying this class to use malloc/free
+  // so we can use realloc here.
+
+  // Allocate new buffer.
+  uint8_t* new_buf = new uint8_t[wBufSize_];
+
+  // Copy the old buffer to the new one.
+  memcpy(new_buf, wBuf_.get(), have);
+
+  // Now point buf to the new one.
+  wBuf_.reset(new_buf);
+  wBase_ = wBuf_.get() + have;
+  wBound_ = wBuf_.get() + wBufSize_;
+
+  // Copy the data into the new buffer.
+  memcpy(wBase_, buf, len);
+  wBase_ += len;
+}
+
+void TFramedTransport::flush()  {
+  int32_t sz_hbo, sz_nbo;
+  assert(wBufSize_ > sizeof(sz_nbo));
+
+  // Slip the frame size into the start of the buffer.
+  sz_hbo = wBase_ - (wBuf_.get() + sizeof(sz_nbo));
+  sz_nbo = (int32_t)htonl((uint32_t)(sz_hbo));
+  memcpy(wBuf_.get(), (uint8_t*)&sz_nbo, sizeof(sz_nbo));
+
+  // Write size and frame body.
+  transport_->write(wBuf_.get(), sizeof(sz_nbo)+sz_hbo);
+
+  // Reset our pointers.
+  wBase_ = wBuf_.get();
+
+  // Pad the buffer so we can insert the size later.
+  uint32_t pad = 0;
+  this->write((uint8_t*)&pad, sizeof(pad));
+
+  // Flush the underlying transport.
+  transport_->flush();
+}
+
+const uint8_t* TFramedTransport::borrowSlow(uint8_t* buf, uint32_t* len) {
+  // Don't try to be clever with shifting buffers.
+  // If the fast path failed let the protocol use its slow path.
+  // Besides, who is going to try to borrow across messages?
+  return NULL;
+}
+
+
+void TMemoryBuffer::computeRead(uint32_t len, uint8_t** out_start, uint32_t* out_give) {
+  // Correct rBound_ so we can use the fast path in the future.
+  rBound_ = wBase_;
+
+  // Decide how much to give.
+  uint32_t give = std::min(len, available_read());
+
+  *out_start = rBase_;
+  *out_give = give;
+
+  // Preincrement rBase_ so the caller doesn't have to.
+  rBase_ += give;
+}
+
+uint32_t TMemoryBuffer::readSlow(uint8_t* buf, uint32_t len) {
+  uint8_t* start;
+  uint32_t give;
+  computeRead(len, &start, &give);
+
+  // Copy into the provided buffer.
+  memcpy(buf, start, give);
+
+  return give;
+}
+
+uint32_t TMemoryBuffer::readAppendToString(std::string& str, uint32_t len) {
+  // Don't get some stupid assertion failure.
+  if (buffer_ == NULL) {
+    return 0;
+  }
+
+  uint8_t* start;
+  uint32_t give;
+  computeRead(len, &start, &give);
+
+  // Append to the provided string.
+  str.append((char*)start, give);
+
+  return give;
+}
+
+void TMemoryBuffer::ensureCanWrite(uint32_t len) {
+  // Check available space
+  uint32_t avail = available_write();
+  if (len <= avail) {
+    return;
+  }
+
+  if (!owner_) {
+    throw TTransportException("Insufficient space in external MemoryBuffer");
+  }
+
+  // Grow the buffer as necessary.
+  while (len > avail) {
+    bufferSize_ *= 2;
+    wBound_ = buffer_ + bufferSize_;
+    avail = available_write();
+  }
+
+  // Allocate into a new pointer so we don't bork ours if it fails.
+  void* new_buffer = std::realloc(buffer_, bufferSize_);
+  if (new_buffer == NULL) {
+    throw TTransportException("Out of memory.");
+  }
+
+  ptrdiff_t offset = (uint8_t*)new_buffer - buffer_;
+  buffer_ += offset;
+  rBase_ += offset;
+  rBound_ += offset;
+  wBase_ += offset;
+  wBound_ += offset;
+}
+
+void TMemoryBuffer::writeSlow(const uint8_t* buf, uint32_t len) {
+  ensureCanWrite(len);
+
+  // Copy into the buffer and increment wBase_.
+  memcpy(wBase_, buf, len);
+  wBase_ += len;
+}
+
+void TMemoryBuffer::wroteBytes(uint32_t len) {
+  uint32_t avail = available_write();
+  if (len > avail) {
+    throw TTransportException("Client wrote more bytes than size of buffer.");
+  }
+  wBase_ += len;
+}
+
+const uint8_t* TMemoryBuffer::borrowSlow(uint8_t* buf, uint32_t* len) {
+  rBound_ = wBase_;
+  if (available_read() >= *len) {
+    *len = available_read();
+    return rBase_;
+  }
+  return NULL;
+}
+
+}}} // facebook::thrift::transport
diff --git a/lib/cpp/src/transport/TBufferTransports.h b/lib/cpp/src/transport/TBufferTransports.h
new file mode 100644
index 0000000..4277d40
--- /dev/null
+++ b/lib/cpp/src/transport/TBufferTransports.h
@@ -0,0 +1,664 @@
+// Copyright (c) 2006- Facebook
+// Distributed under the Thrift Software License
+//
+// See accompanying file LICENSE or visit the Thrift site at:
+// http://developers.facebook.com/thrift/
+
+#ifndef _THRIFT_TRANSPORT_TDOUBLEBUFFERS_H_
+#define _THRIFT_TRANSPORT_TDOUBLEBUFFERS_H_ 1
+
+#include "boost/scoped_array.hpp"
+
+#include <transport/TTransport.h>
+
+#ifdef __GNUC__
+#define TDB_LIKELY(val) (__builtin_expect((val), 1))
+#define TDB_UNLIKELY(val) (__builtin_expect((val), 0))
+#else
+#define TDB_LIKELY(val) (val)
+#define TDB_UNLIKELY(val) (val)
+#endif
+
+namespace facebook { namespace thrift { namespace transport {
+
+
+/**
+ * Base class for all transports that use read/write buffers for performance.
+ *
+ * TBufferBase is designed to implement the fast-path "memcpy" style
+ * operations that work in the common case.  It does so with small and
+ * (eventually) nonvirtual, inlinable methods.  TBufferBase is an abstract
+ * class.  Subclasses are expected to define the "slow path" operations
+ * that have to be done when the buffers are full or empty.
+ *
+ * @author David Reiss <dreiss@facebook.com>
+ */
+class TBufferBase : public TTransport {
+
+ public:
+
+  /**
+   * Fast-path read.
+   *
+   * When we have enough data buffered to fulfill the read, we can satisfy it
+   * with a single memcpy, then adjust our internal pointers.  If the buffer
+   * is empty, we call out to our slow path, implemented by a subclass.
+   * This method is meant to eventually be nonvirtual and inlinable.
+   */
+  uint32_t read(uint8_t* buf, uint32_t len) {
+    uint8_t* new_rBase = rBase_ + len;
+    if (TDB_LIKELY(new_rBase <= rBound_)) {
+      memcpy(buf, rBase_, len);
+      rBase_ = new_rBase;
+      return len;
+    }
+    return readSlow(buf, len);
+  }
+
+  /**
+   * Fast-path write.
+   *
+   * When we have enough empty space in our buffer to accomodate the write, we
+   * can satisfy it with a single memcpy, then adjust our internal pointers.
+   * If the buffer is full, we call out to our slow path, implemented by a
+   * subclass.  This method is meant to eventually be nonvirtual and
+   * inlinable.
+   */
+  void write(const uint8_t* buf, uint32_t len) {
+    uint8_t* new_wBase = wBase_ + len;
+    if (TDB_LIKELY(new_wBase <= wBound_)) {
+      memcpy(wBase_, buf, len);
+      wBase_ = new_wBase;
+      return;
+    }
+    writeSlow(buf, len);
+  }
+
+  /**
+   * Fast-path borrow.  A lot like the fast-path read.
+   */
+  const uint8_t* borrow(uint8_t* buf, uint32_t* len) {
+    if (TDB_LIKELY(static_cast<ptrdiff_t>(*len) <= rBound_ - rBase_)) {
+      // With strict aliasing, writing to len shouldn't force us to
+      // refetch rBase_ from memory.  TODO(dreiss): Verify this.
+      *len = rBound_ - rBase_;
+      return rBase_;
+    }
+    return borrowSlow(buf, len);
+  }
+
+  /**
+   * Consume doesn't require a slow path.
+   */
+  void consume(uint32_t len) {
+    if (TDB_LIKELY(static_cast<ptrdiff_t>(len) <= rBound_ - rBase_)) {
+      rBase_ += len;
+    } else {
+      throw TTransportException(TTransportException::BAD_ARGS,
+                                "consume did not follow a borrow.");
+    }
+  }
+
+
+ protected:
+
+  /// Slow path read.
+  virtual uint32_t readSlow(uint8_t* buf, uint32_t len) = 0;
+
+  /// Slow path read.
+  virtual void writeSlow(const uint8_t* buf, uint32_t len) = 0;
+
+  /**
+   * Slow path borrow.
+   *
+   * POSTCONDITION: return == NULL || rBound_ - rBase_ >= *len
+   */
+  virtual const uint8_t* borrowSlow(uint8_t* buf, uint32_t* len) = 0;
+
+  /**
+   * Trivial constructor.
+   *
+   * Initialize pointers safely.  Constructing is not a very
+   * performance-sensitive operation, so it is okay to just leave it to
+   * the concrete class to set up pointers correctly.
+   */
+  TBufferBase()
+    : rBase_(NULL)
+    , rBound_(NULL)
+    , wBase_(NULL)
+    , wBound_(NULL)
+  {}
+
+  /// Convenience mutator for setting the read buffer.
+  void setReadBuffer(uint8_t* buf, uint32_t len) {
+    rBase_ = buf;
+    rBound_ = buf+len;
+  }
+
+  /// Convenience mutator for setting the read buffer.
+  void setWriteBuffer(uint8_t* buf, uint32_t len) {
+    wBase_ = buf;
+    wBound_ = buf+len;
+  }
+
+  virtual ~TBufferBase() {}
+
+  /// Reads begin here.
+  uint8_t* rBase_;
+  /// Reads may extend to just before here.
+  uint8_t* rBound_;
+
+  /// Writes begin here.
+  uint8_t* wBase_;
+  /// Writes may extend to just before here.
+  uint8_t* wBound_;
+};
+
+
+/**
+ * Buffered transport. For reads it will read more data than is requested
+ * and will serve future data out of a local buffer. For writes, data is
+ * stored to an in memory buffer before being written out.
+ *
+ * @author Mark Slee <mcslee@facebook.com>
+ * @author David Reiss <dreiss@facebook.com>
+ */
+class TBufferedTransport : public TBufferBase {
+ public:
+
+  static const int DEFAULT_BUFFER_SIZE = 512;
+
+  /// Use default buffer sizes.
+  TBufferedTransport(boost::shared_ptr<TTransport> transport)
+    : transport_(transport)
+    , rBufSize_(DEFAULT_BUFFER_SIZE)
+    , wBufSize_(DEFAULT_BUFFER_SIZE)
+    , rBuf_(new uint8_t[rBufSize_])
+    , wBuf_(new uint8_t[wBufSize_])
+  {
+    initPointers();
+  }
+
+  /// Use specified buffer sizes.
+  TBufferedTransport(boost::shared_ptr<TTransport> transport, uint32_t sz)
+    : transport_(transport)
+    , rBufSize_(sz)
+    , wBufSize_(sz)
+    , rBuf_(new uint8_t[rBufSize_])
+    , wBuf_(new uint8_t[wBufSize_])
+  {
+    initPointers();
+  }
+
+  /// Use specified read and write buffer sizes.
+  TBufferedTransport(boost::shared_ptr<TTransport> transport, uint32_t rsz, uint32_t wsz)
+    : transport_(transport)
+    , rBufSize_(rsz)
+    , wBufSize_(rsz)
+    , rBuf_(new uint8_t[rBufSize_])
+    , wBuf_(new uint8_t[wBufSize_])
+  {
+    initPointers();
+  }
+
+  void open() {
+    transport_->open();
+  }
+
+  bool isOpen() {
+    return transport_->isOpen();
+  }
+
+  bool peek() {
+    if (rBase_ == rBound_) {
+      setReadBuffer(rBuf_.get(), transport_->read(rBuf_.get(), rBufSize_));
+    }
+    return (rBound_ > rBase_);
+  }
+
+  void close() {
+    flush();
+    transport_->close();
+  }
+
+  virtual uint32_t readSlow(uint8_t* buf, uint32_t len);
+
+  virtual void writeSlow(const uint8_t* buf, uint32_t len);
+
+  void flush();
+
+
+  /**
+   * The following behavior is currently implemented by TBufferedTransport,
+   * but that may change in a future version:
+   * 1/ If len is at most rBufSize_, borrow will never return NULL.
+   *    Depending on the underlying transport, it could throw an exception
+   *    or hang forever.
+   * 2/ Some borrow requests may copy bytes internally.  However,
+   *    if len is at most rBufSize_/2, none of the copied bytes
+   *    will ever have to be copied again.  For optimial performance,
+   *    stay under this limit.
+   */
+  virtual const uint8_t* borrowSlow(uint8_t* buf, uint32_t* len);
+
+  boost::shared_ptr<TTransport> getUnderlyingTransport() {
+    return transport_;
+  }
+
+ protected:
+  void initPointers() {
+    setReadBuffer(rBuf_.get(), 0);
+    setWriteBuffer(wBuf_.get(), wBufSize_);
+    // Write size never changes.
+  }
+
+  boost::shared_ptr<TTransport> transport_;
+
+  uint32_t rBufSize_;
+  uint32_t wBufSize_;
+  boost::scoped_array<uint8_t> rBuf_;
+  boost::scoped_array<uint8_t> wBuf_;
+};
+
+
+/**
+ * Wraps a transport into a buffered one.
+ *
+ * @author Mark Slee <mcslee@facebook.com>
+ */
+class TBufferedTransportFactory : public TTransportFactory {
+ public:
+  TBufferedTransportFactory() {}
+
+  virtual ~TBufferedTransportFactory() {}
+
+  /**
+   * Wraps the transport into a buffered one.
+   */
+  virtual boost::shared_ptr<TTransport> getTransport(boost::shared_ptr<TTransport> trans) {
+    return boost::shared_ptr<TTransport>(new TBufferedTransport(trans));
+  }
+
+};
+
+
+/**
+ * Framed transport. All writes go into an in-memory buffer until flush is
+ * called, at which point the transport writes the length of the entire
+ * binary chunk followed by the data payload. This allows the receiver on the
+ * other end to always do fixed-length reads.
+ *
+ * @author Mark Slee <mcslee@facebook.com>
+ * @author David Reiss <dreiss@facebook.com>
+ */
+class TFramedTransport : public TBufferBase {
+ public:
+
+  static const int DEFAULT_BUFFER_SIZE = 512;
+
+  /// Use default buffer sizes.
+  TFramedTransport(boost::shared_ptr<TTransport> transport)
+    : transport_(transport)
+    , rBufSize_(0)
+    , wBufSize_(DEFAULT_BUFFER_SIZE)
+    , rBuf_()
+    , wBuf_(new uint8_t[wBufSize_])
+  {
+    initPointers();
+  }
+
+  TFramedTransport(boost::shared_ptr<TTransport> transport, uint32_t sz)
+    : transport_(transport)
+    , rBufSize_(0)
+    , wBufSize_(sz)
+    , rBuf_()
+    , wBuf_(new uint8_t[wBufSize_])
+  {
+    initPointers();
+  }
+
+  void open() {
+    transport_->open();
+  }
+
+  bool isOpen() {
+    return transport_->isOpen();
+  }
+
+  bool peek() {
+    return (rBase_ < rBound_) || transport_->peek();
+  }
+
+  void close() {
+    flush();
+    transport_->close();
+  }
+
+  virtual uint32_t readSlow(uint8_t* buf, uint32_t len);
+
+  virtual void writeSlow(const uint8_t* buf, uint32_t len);
+
+  virtual void flush();
+
+  const uint8_t* borrowSlow(uint8_t* buf, uint32_t* len);
+
+  boost::shared_ptr<TTransport> getUnderlyingTransport() {
+    return transport_;
+  }
+
+ protected:
+  /**
+   * Reads a frame of input from the underlying stream.
+   */
+  void readFrame();
+
+  void initPointers() {
+    setReadBuffer(NULL, 0);
+    setWriteBuffer(wBuf_.get(), wBufSize_);
+
+    // Pad the buffer so we can insert the size later.
+    int32_t pad = 0;
+    this->write((uint8_t*)&pad, sizeof(pad));
+  }
+
+  boost::shared_ptr<TTransport> transport_;
+
+  uint32_t rBufSize_;
+  uint32_t wBufSize_;
+  boost::scoped_array<uint8_t> rBuf_;
+  boost::scoped_array<uint8_t> wBuf_;
+};
+
+/**
+ * Wraps a transport into a framed one.
+ *
+ * @author Dave Simpson <dave@powerset.com>
+ */
+class TFramedTransportFactory : public TTransportFactory {
+ public:
+  TFramedTransportFactory() {}
+
+  virtual ~TFramedTransportFactory() {}
+
+  /**
+   * Wraps the transport into a framed one.
+   */
+  virtual boost::shared_ptr<TTransport> getTransport(boost::shared_ptr<TTransport> trans) {
+    return boost::shared_ptr<TTransport>(new TFramedTransport(trans));
+  }
+
+};
+
+
+/**
+ * A memory buffer is a tranpsort that simply reads from and writes to an
+ * in memory buffer. Anytime you call write on it, the data is simply placed
+ * into a buffer, and anytime you call read, data is read from that buffer.
+ *
+ * The buffers are allocated using C constructs malloc,realloc, and the size
+ * doubles as necessary.  We've considered using scoped
+ *
+ * @author Mark Slee <mcslee@facebook.com>
+ * @author David Reiss <dreiss@facebook.com>
+ */
+class TMemoryBuffer : public TBufferBase {
+ private:
+
+  // Common initialization done by all constructors.
+  void initCommon(uint8_t* buf, uint32_t size, bool owner, uint32_t wPos) {
+    if (buf == NULL && size != 0) {
+      assert(owner);
+      buf = (uint8_t*)std::malloc(size);
+      if (buf == NULL) {
+        throw TTransportException("Out of memory");
+      }
+    }
+
+    buffer_ = buf;
+    bufferSize_ = size;
+
+    rBase_ = buffer_;
+    rBound_ = buffer_ + wPos;
+    // TODO(dreiss): Investigate NULL-ing this if !owner.
+    wBase_ = buffer_ + wPos;
+    wBound_ = buffer_ + bufferSize_;
+
+    owner_ = owner;
+
+    // rBound_ is really an artifact.  In principle, it should always be
+    // equal to wBase_.  We update it in a few places (computeRead, etc.).
+  }
+
+ public:
+  static const uint32_t defaultSize = 1024;
+
+  /**
+   * This enum specifies how a TMemoryBuffer should treat
+   * memory passed to it via constructors or resetBuffer.
+   *
+   * OBSERVE:
+   *   TMemoryBuffer will simply store a pointer to the memory.
+   *   It is the callers responsibility to ensure that the pointer
+   *   remains valid for the lifetime of the TMemoryBuffer,
+   *   and that it is properly cleaned up.
+   *   Note that no data can be written to observed buffers.
+   *
+   * COPY:
+   *   TMemoryBuffer will make an internal copy of the buffer.
+   *   The caller has no responsibilities.
+   *
+   * TAKE_OWNERSHIP:
+   *   TMemoryBuffer will become the "owner" of the buffer,
+   *   and will be responsible for freeing it.
+   *   The membory must have been allocated with malloc.
+   */
+  enum MemoryPolicy {
+    OBSERVE = 1,
+    COPY = 2,
+    TAKE_OWNERSHIP = 3,
+  };
+
+  /**
+   * Construct a TMemoryBuffer with a default-sized buffer,
+   * owned by the TMemoryBuffer object.
+   */
+  TMemoryBuffer() {
+    initCommon(NULL, defaultSize, true, 0);
+  }
+
+  /**
+   * Construct a TMemoryBuffer with a buffer of a specified size,
+   * owned by the TMemoryBuffer object.
+   *
+   * @param sz  The initial size of the buffer.
+   */
+  TMemoryBuffer(uint32_t sz) {
+    initCommon(NULL, sz, true, 0);
+  }
+
+  /**
+   * Construct a TMemoryBuffer with buf as its initial contents.
+   *
+   * @param buf    The initial contents of the buffer.
+   *               Note that, while buf is a non-const pointer,
+   *               TMemoryBuffer will not write to it if policy == OBSERVE,
+   *               so it is safe to const_cast<uint8_t*>(whatever).
+   * @param sz     The size of @c buf.
+   * @param policy See @link MemoryPolicy @endlink .
+   */
+  TMemoryBuffer(uint8_t* buf, uint32_t sz, MemoryPolicy policy = OBSERVE) {
+    if (buf == NULL && sz != 0) {
+      throw TTransportException(TTransportException::BAD_ARGS,
+                                "TMemoryBuffer given null buffer with non-zero size.");
+    }
+
+    switch (policy) {
+      case OBSERVE:
+      case TAKE_OWNERSHIP:
+        initCommon(buf, sz, policy == TAKE_OWNERSHIP, sz);
+        break;
+      case COPY:
+        initCommon(NULL, sz, true, 0);
+        this->write(buf, sz);
+        break;
+      default:
+        throw TTransportException(TTransportException::BAD_ARGS,
+                                  "Invalid MemoryPolicy for TMemoryBuffer");
+    }
+  }
+
+  ~TMemoryBuffer() {
+    if (owner_) {
+      std::free(buffer_);
+    }
+  }
+
+  bool isOpen() {
+    return true;
+  }
+
+  bool peek() {
+    return (rBase_ < wBase_);
+  }
+
+  void open() {}
+
+  void close() {}
+
+  // TODO(dreiss): Make bufPtr const.
+  void getBuffer(uint8_t** bufPtr, uint32_t* sz) {
+    *bufPtr = rBase_;
+    *sz = wBase_ - rBase_;
+  }
+
+  std::string getBufferAsString() {
+    if (buffer_ == NULL) {
+      return "";
+    }
+    uint8_t* buf;
+    uint32_t sz;
+    getBuffer(&buf, &sz);
+    return std::string((char*)buf, (std::string::size_type)sz);
+  }
+
+  void appendBufferToString(std::string& str) {
+    if (buffer_ == NULL) {
+      return;
+    }
+    uint8_t* buf;
+    uint32_t sz;
+    getBuffer(&buf, &sz);
+    str.append((char*)buf, sz);
+  }
+
+  void resetBuffer() {
+    rBase_ = buffer_;
+    rBound_ = buffer_;
+    wBase_ = buffer_;
+    // It isn't safe to write into a buffer we don't own.
+    if (!owner_) {
+      wBound_ = wBase_;
+      bufferSize_ = 0;
+    }
+  }
+
+  /// See constructor documentation.
+  void resetBuffer(uint8_t* buf, uint32_t sz, MemoryPolicy policy = OBSERVE) {
+    // Use a variant of the copy-and-swap trick for assignment operators.
+    // This is sub-optimal in terms of performance for two reasons:
+    //   1/ The constructing and swapping of the (small) values
+    //      in the temporary object takes some time, and is not necessary.
+    //   2/ If policy == COPY, we allocate the new buffer before
+    //      freeing the old one, precluding the possibility of
+    //      reusing that memory.
+    // I doubt that either of these problems could be optimized away,
+    // but the second is probably no a common case, and the first is minor.
+    // I don't expect resetBuffer to be a common operation, so I'm willing to
+    // bite the performance bullet to make the method this simple.
+
+    // Construct the new buffer.
+    TMemoryBuffer new_buffer(buf, sz, policy);
+    // Move it into ourself.
+    this->swap(new_buffer);
+    // Our old self gets destroyed.
+  }
+
+  std::string readAsString(uint32_t len) {
+    std::string str;
+    (void)readAppendToString(str, len);
+    return str;
+  }
+
+  uint32_t readAppendToString(std::string& str, uint32_t len);
+
+  void readEnd() {
+    if (rBase_ == wBase_) {
+      resetBuffer();
+    }
+  }
+
+  uint32_t available_read() const {
+    // Remember, wBase_ is the real rBound_.
+    return wBase_ - rBase_;
+  }
+
+  uint32_t available_write() const {
+    return wBound_ - wBase_;
+  }
+
+  // Returns a pointer to where the client can write data to append to
+  // the TMemoryBuffer, and ensures the buffer is big enough to accomodate a
+  // write of the provided length.  The returned pointer is very convenient for
+  // passing to read(), recv(), or similar. You must call wroteBytes() as soon
+  // as data is written or the buffer will not be aware that data has changed.
+  uint8_t* getWritePtr(uint32_t len) {
+    ensureCanWrite(len);
+    return wBase_;
+  }
+
+  // Informs the buffer that the client has written 'len' bytes into storage
+  // that had been provided by getWritePtr().
+  void wroteBytes(uint32_t len);
+
+ protected:
+  void swap(TMemoryBuffer& that) {
+    using std::swap;
+    swap(buffer_,     that.buffer_);
+    swap(bufferSize_, that.bufferSize_);
+
+    swap(rBase_,      that.rBase_);
+    swap(rBound_,     that.rBound_);
+    swap(wBase_,      that.wBase_);
+    swap(wBound_,     that.wBound_);
+
+    swap(owner_,      that.owner_);
+  }
+
+  // Make sure there's at least 'len' bytes available for writing.
+  void ensureCanWrite(uint32_t len);
+
+  // Compute the position and available data for reading.
+  void computeRead(uint32_t len, uint8_t** out_start, uint32_t* out_give);
+
+  uint32_t readSlow(uint8_t* buf, uint32_t len);
+
+  void writeSlow(const uint8_t* buf, uint32_t len);
+
+  const uint8_t* borrowSlow(uint8_t* buf, uint32_t* len);
+
+  // Data buffer
+  uint8_t* buffer_;
+
+  // Allocated buffer size
+  uint32_t bufferSize_;
+
+  // Is this object the owner of the buffer?
+  bool owner_;
+
+  // Don't forget to update constrctors, initCommon, and swap if
+  // you add new members.
+};
+
+}}} // facebook::thrift::transport
+
+#endif // #ifndef _THRIFT_TRANSPORT_TDOUBLEBUFFERS_H_
diff --git a/lib/cpp/src/transport/THttpClient.cpp b/lib/cpp/src/transport/THttpClient.cpp
index a294093..8c82d7f 100644
--- a/lib/cpp/src/transport/THttpClient.cpp
+++ b/lib/cpp/src/transport/THttpClient.cpp
@@ -71,7 +71,7 @@
 }
 
 uint32_t THttpClient::read(uint8_t* buf, uint32_t len) {
-  if (readBuffer_.available() == 0) {
+  if (readBuffer_.available_read() == 0) {
     readBuffer_.resetBuffer();
     uint32_t got = readMoreData();
     if (got == 0) {
diff --git a/lib/cpp/src/transport/THttpClient.h b/lib/cpp/src/transport/THttpClient.h
index 1ae3575..dbffe99 100644
--- a/lib/cpp/src/transport/THttpClient.h
+++ b/lib/cpp/src/transport/THttpClient.h
@@ -7,7 +7,7 @@
 #ifndef _THRIFT_TRANSPORT_THTTPCLIENT_H_
 #define _THRIFT_TRANSPORT_THTTPCLIENT_H_ 1
 
-#include <transport/TTransportUtils.h>
+#include <transport/TBufferTransports.h>
 
 namespace facebook { namespace thrift { namespace transport {
 
diff --git a/lib/cpp/src/transport/TShortReadTransport.h b/lib/cpp/src/transport/TShortReadTransport.h
new file mode 100644
index 0000000..428fa9e
--- /dev/null
+++ b/lib/cpp/src/transport/TShortReadTransport.h
@@ -0,0 +1,84 @@
+// Copyright (c) 2006- Facebook
+// Distributed under the Thrift Software License
+//
+// See accompanying file LICENSE or visit the Thrift site at:
+// http://developers.facebook.com/thrift/
+
+#ifndef _THRIFT_TRANSPORT_TSHORTREADTRANSPORT_H_
+#define _THRIFT_TRANSPORT_TSHORTREADTRANSPORT_H_ 1
+
+#include <cstdlib>
+
+#include <transport/TTransport.h>
+
+namespace facebook { namespace thrift { namespace transport { namespace test {
+
+/**
+ * This class is only meant for testing.  It wraps another transport.
+ * Calls to read are passed through with some probability.  Otherwise,
+ * the read amount is randomly reduced before being passed through.
+ *
+ * @author David Reiss <dreiss@facebook.com>
+ */
+class TShortReadTransport : public TTransport {
+ public:
+  TShortReadTransport(boost::shared_ptr<TTransport> transport, double full_prob)
+    : transport_(transport)
+    , fullProb_(full_prob)
+  {}
+
+  bool isOpen() {
+    return transport_->isOpen();
+  }
+
+  bool peek() {
+    return transport_->peek();
+  }
+
+  void open() {
+    transport_->open();
+  }
+
+  void close() {
+    transport_->close();
+  }
+
+  uint32_t read(uint8_t* buf, uint32_t len) {
+    if (len == 0) {
+      return 0;
+    }
+
+    if (rand()/(double)RAND_MAX >= fullProb_) {
+      len = 1 + rand()%len;
+    }
+    return transport_->read(buf, len);
+  }
+
+  void write(const uint8_t* buf, uint32_t len) {
+    transport_->write(buf, len);
+  }
+
+  void flush() {
+    transport_->flush();
+  }
+
+  const uint8_t* borrow(uint8_t* buf, uint32_t* len) {
+    return transport_->borrow(buf, len);
+  }
+
+  void consume(uint32_t len) {
+    return transport_->consume(len);
+  }
+
+  boost::shared_ptr<TTransport> getUnderlyingTransport() {
+    return transport_;
+  }
+
+ protected:
+  boost::shared_ptr<TTransport> transport_;
+  double fullProb_;
+};
+
+}}}} // facebook::thrift::transport::test
+
+#endif // #ifndef _THRIFT_TRANSPORT_TSHORTREADTRANSPORT_H_
diff --git a/lib/cpp/src/transport/TTransportUtils.cpp b/lib/cpp/src/transport/TTransportUtils.cpp
index d00025e..d365a3c 100644
--- a/lib/cpp/src/transport/TTransportUtils.cpp
+++ b/lib/cpp/src/transport/TTransportUtils.cpp
@@ -10,348 +10,6 @@
 
 namespace facebook { namespace thrift { namespace transport {
 
-uint32_t TBufferedTransport::read(uint8_t* buf, uint32_t len) {
-  uint32_t need = len;
-
-  // We don't have enough data yet
-  if (rLen_-rPos_ < need) {
-    // Copy out whatever we have
-    if (rLen_-rPos_ > 0) {
-      memcpy(buf, rBuf_+rPos_, rLen_-rPos_);
-      need -= rLen_-rPos_;
-      buf += rLen_-rPos_;
-    }
-    // Get more from underlying transport up to buffer size
-    rLen_ = transport_->read(rBuf_, rBufSize_);
-    rPos_ = 0;
-  }
-
-  // Hand over whatever we have
-  uint32_t give = need;
-  if (rLen_-rPos_ < give) {
-    give = rLen_-rPos_;
-  }
-  memcpy(buf, rBuf_+rPos_, give);
-  rPos_ += give;
-  need -= give;
-  return (len - need);
-}
-
-void TBufferedTransport::write(const uint8_t* buf, uint32_t len) {
-  if (len == 0) {
-    return;
-  }
-
-  uint32_t pos = 0;
-
-  while ((len-pos) + wLen_ >= wBufSize_) {
-    uint32_t copy = wBufSize_ - wLen_;
-    memcpy(wBuf_ + wLen_, buf + pos, copy);
-
-    transport_->write(wBuf_, wBufSize_);
-    pos += copy;
-    wLen_ = 0;
-  }
-
-  if ((len - pos) > 0) {
-    memcpy(wBuf_ + wLen_, buf + pos, len - pos);
-    wLen_ += len - pos;
-  }
-}
-
-const uint8_t* TBufferedTransport::borrow(uint8_t* buf, uint32_t* len) {
-  // The number of additional bytes we need from the underlying transport.
-  // Could be zero or negative.
-  uint32_t need = *len - (rLen_-rPos_);
-
-  // If we have enough data, just hand over a pointer.
-  if (need <= 0) {
-    *len = rLen_-rPos_;
-    return rBuf_+rPos_;
-  }
-
-  // If the request is bigger than our buffer, we are hosed.
-  if (*len > rBufSize_) {
-    return NULL;
-  }
-
-  // If we have less than half our buffer available,
-  // or we need more space than is in the buffer,
-  // shift the data we have down to the start.
-  if ((rLen_ > rBufSize_/2) || (rLen_+need > rBufSize_)) {
-    memmove(rBuf_, rBuf_+rPos_, rLen_-rPos_);
-    rLen_ -= rPos_;
-    rPos_ = 0;
-  }
-
-  // First try to fill up the buffer.
-  uint32_t got = transport_->read(rBuf_+rLen_, rBufSize_-rLen_);
-  rLen_ += got;
-  need -= got;
-
-  // If that fails, readAll until we get what we need.
-  if (need > 0) {
-    rLen_ += transport_->readAll(rBuf_+rLen_, need);
-  }
-
-  *len = rLen_-rPos_;
-  return rBuf_+rPos_;
-}
-
-void TBufferedTransport::consume(uint32_t len) {
-  if (rLen_-rPos_ >= len) {
-    rPos_ += len;
-  } else {
-    throw TTransportException(TTransportException::BAD_ARGS,
-                              "consume did not follow a borrow.");
-  }
-}
-
-void TBufferedTransport::flush()  {
-  // Write out any data waiting in the write buffer
-  if (wLen_ > 0) {
-    transport_->write(wBuf_, wLen_);
-    wLen_ = 0;
-  }
-
-  // Flush the underlying transport
-  transport_->flush();
-}
-
-uint32_t TFramedTransport::read(uint8_t* buf, uint32_t len) {
-  if (!read_) {
-    return transport_->read(buf, len);
-  }
-
-  uint32_t need = len;
-
-  // We don't have enough data yet
-  if (rLen_-rPos_ < need) {
-    // Copy out whatever we have
-    if (rLen_-rPos_ > 0) {
-      memcpy(buf, rBuf_+rPos_, rLen_-rPos_);
-      need -= rLen_-rPos_;
-      buf += rLen_-rPos_;
-    }
-
-    // Read another chunk
-    readFrame();
-  }
-
-  // Hand over whatever we have
-  uint32_t give = need;
-  if (rLen_-rPos_ < give) {
-    give = rLen_-rPos_;
-  }
-  memcpy(buf, rBuf_+rPos_, give);
-  rPos_ += give;
-  need -= give;
-  return (len - need);
-}
-
-void TFramedTransport::readFrame() {
-  // Get rid of the old frame
-  if (rBuf_ != NULL) {
-    delete [] rBuf_;
-    rBuf_ = NULL;
-  }
-
-  // Read in the next chunk size
-  int32_t sz;
-  transport_->readAll((uint8_t*)&sz, 4);
-  sz = (int32_t)ntohl(sz);
-
-  if (sz < 0) {
-    throw TTransportException("Frame size has negative value");
-  }
-
-  // Read the frame payload, reset markers
-  rBuf_ = new uint8_t[sz];
-  transport_->readAll(rBuf_, sz);
-  rPos_ = 0;
-  rLen_ = sz;
-}
-
-void TFramedTransport::write(const uint8_t* buf, uint32_t len) {
-  if (len == 0) {
-    return;
-  }
-
-  // Shortcut out if not write mode
-  if (!write_) {
-    transport_->write(buf, len);
-    return;
-  }
-
-  // Need to grow the buffer
-  if (len + wLen_ >= wBufSize_) {
-
-    // Double buffer size until sufficient
-    while (wBufSize_ < len + wLen_) {
-      wBufSize_ *= 2;
-    }
-
-    // Allocate new buffer
-    uint8_t* wBuf2 = new uint8_t[wBufSize_];
-
-    // Copy the old buffer to the new one
-    memcpy(wBuf2, wBuf_, wLen_);
-
-    // Now point buf to the new one
-    delete [] wBuf_;
-    wBuf_ = wBuf2;
-  }
-
-  // Copy data into buffer
-  memcpy(wBuf_ + wLen_, buf, len);
-  wLen_ += len;
-}
-
-void TFramedTransport::flush()  {
-  if (!write_) {
-    transport_->flush();
-    return;
-  }
-
-  // Write frame size
-  int32_t sz = wLen_;
-  sz = (int32_t)htonl(sz);
-
-  transport_->write((const uint8_t*)&sz, 4);
-
-  // Write frame body
-  if (wLen_ > 0) {
-    transport_->write(wBuf_, wLen_);
-  }
-
-  // All done
-  wLen_ = 0;
-
-  // Flush the underlying
-  transport_->flush();
-}
-
-const uint8_t* TFramedTransport::borrow(uint8_t* buf, uint32_t* len) {
-  // Don't try to be clever with shifting buffers.
-  // If we have enough data, give a pointer to it,
-  // otherwise let the protcol use its slow path.
-  if (read_ && (rLen_-rPos_ >= *len)) {
-    *len = rLen_-rPos_;
-    return rBuf_+rPos_;
-  }
-  return NULL;
-}
-
-void TFramedTransport::consume(uint32_t len) {
-  if (rLen_-rPos_ >= len) {
-    rPos_ += len;
-  } else {
-    throw TTransportException(TTransportException::BAD_ARGS,
-                              "consume did not follow a borrow.");
-  }
-}
-
-uint32_t TMemoryBuffer::read(uint8_t* buf, uint32_t len) {
-  // Check avaible data for reading
-  uint32_t avail = wPos_ - rPos_;
-  if (avail == 0) {
-    return 0;
-  }
-
-  // Decide how much to give
-  uint32_t give = len;
-  if (avail < len) {
-    give = avail;
-  }
-
-  // Copy into buffer and increment rPos_
-  memcpy(buf, buffer_ + rPos_, give);
-  rPos_ += give;
-
-  return give;
-}
-
-uint32_t TMemoryBuffer::readAppendToString(std::string& str, uint32_t len) {
-  // Don't get some stupid assertion failure.
-  if (buffer_ == NULL) {
-    return 0;
-  }
-
-  // Check avaible data for reading
-  uint32_t avail = wPos_ - rPos_;
-  if (avail == 0) {
-    return 0;
-  }
-
-  // Device how much to give
-  uint32_t give = len;
-  if (avail < len) {
-    give = avail;
-  }
-
-  // Reserve memory, copy into string, and increment rPos_
-  str.reserve(str.length()+give);
-  str.append((char*)buffer_ + rPos_, give);
-  rPos_ += give;
-
-  return give;
-}
-
-void TMemoryBuffer::ensureCanWrite(uint32_t len) {
-  // Check available space
-  uint32_t avail = bufferSize_ - wPos_;
-  if (len <= avail) {
-    return;
-  }
-
-  if (!owner_) {
-    throw TTransportException("Insufficient space in external MemoryBuffer");
-  }
-
-  // Grow the buffer as necessary
-  while (len > avail) {
-    bufferSize_ *= 2;
-    avail = bufferSize_ - wPos_;
-  }
-  buffer_ = (uint8_t*)std::realloc(buffer_, bufferSize_);
-  if (buffer_ == NULL) {
-    throw TTransportException("Out of memory.");
-  }
-}
-
-void TMemoryBuffer::write(const uint8_t* buf, uint32_t len) {
-  ensureCanWrite(len);
-
-  // Copy into the buffer and increment wPos_
-  memcpy(buffer_ + wPos_, buf, len);
-  wPos_ += len;
-}
-
-void TMemoryBuffer::wroteBytes(uint32_t len) {
-  uint32_t avail = bufferSize_ - wPos_;
-  if (len > avail) {
-    throw TTransportException("Client wrote more bytes than size of buffer.");
-  }
-  wPos_ += len;
-}
-
-const uint8_t* TMemoryBuffer::borrow(uint8_t* buf, uint32_t* len) {
-  if (wPos_-rPos_ >= *len) {
-    *len = wPos_-rPos_;
-    return buffer_ + rPos_;
-  }
-  return NULL;
-}
-
-void TMemoryBuffer::consume(uint32_t len) {
-  if (wPos_-rPos_ >= len) {
-    rPos_ += len;
-  } else {
-    throw TTransportException(TTransportException::BAD_ARGS,
-                              "consume did not follow a borrow.");
-  }
-}
-
 uint32_t TPipedTransport::read(uint8_t* buf, uint32_t len) {
   uint32_t need = len;
 
diff --git a/lib/cpp/src/transport/TTransportUtils.h b/lib/cpp/src/transport/TTransportUtils.h
index 6c7e4db..61f6822 100644
--- a/lib/cpp/src/transport/TTransportUtils.h
+++ b/lib/cpp/src/transport/TTransportUtils.h
@@ -12,6 +12,8 @@
 #include <string>
 #include <algorithm>
 #include <transport/TTransport.h>
+// Include the buffered transports that used to be defined here.
+#include <transport/TBufferTransports.h>
 #include <transport/TFileTransport.h>
 
 namespace facebook { namespace thrift { namespace transport {
@@ -44,497 +46,6 @@
 
 
 /**
- * Buffered transport. For reads it will read more data than is requested
- * and will serve future data out of a local buffer. For writes, data is
- * stored to an in memory buffer before being written out.
- *
- * @author Mark Slee <mcslee@facebook.com>
- */
-class TBufferedTransport : public TTransport {
- public:
-  TBufferedTransport(boost::shared_ptr<TTransport> transport) :
-    transport_(transport),
-    rBufSize_(512), rPos_(0), rLen_(0),
-    wBufSize_(512), wLen_(0) {
-    rBuf_ = new uint8_t[rBufSize_];
-    wBuf_ = new uint8_t[wBufSize_];
-  }
-
-  TBufferedTransport(boost::shared_ptr<TTransport> transport, uint32_t sz) :
-    transport_(transport),
-    rBufSize_(sz), rPos_(0), rLen_(0),
-    wBufSize_(sz), wLen_(0) {
-    rBuf_ = new uint8_t[rBufSize_];
-    wBuf_ = new uint8_t[wBufSize_];
-  }
-
-  TBufferedTransport(boost::shared_ptr<TTransport> transport, uint32_t rsz, uint32_t wsz) :
-    transport_(transport),
-    rBufSize_(rsz), rPos_(0), rLen_(0),
-    wBufSize_(wsz), wLen_(0) {
-    rBuf_ = new uint8_t[rBufSize_];
-    wBuf_ = new uint8_t[wBufSize_];
-  }
-
-  ~TBufferedTransport() {
-    delete [] rBuf_;
-    delete [] wBuf_;
-  }
-
-  bool isOpen() {
-    return transport_->isOpen();
-  }
-
-  bool peek() {
-    if (rPos_ >= rLen_) {
-      rLen_ = transport_->read(rBuf_, rBufSize_);
-      rPos_ = 0;
-    }
-    return (rLen_ > rPos_);
-  }
-
-  void open() {
-    transport_->open();
-  }
-
-  void close() {
-    flush();
-    transport_->close();
-  }
-
-  uint32_t read(uint8_t* buf, uint32_t len);
-
-  void write(const uint8_t* buf, uint32_t len);
-
-  void flush();
-
-  /**
-   * The following behavior is currently implemented by TBufferedTransport,
-   * but that may change in a future version:
-   * 1/ If len is at most rBufSize_, borrow will never return NULL.
-   *    Depending on the underlying transport, it could throw an exception
-   *    or hang forever.
-   * 2/ Some borrow requests may copy bytes internally.  However,
-   *    if len is at most rBufSize_/2, none of the copied bytes
-   *    will ever have to be copied again.  For optimial performance,
-   *    stay under this limit.
-   */
-  const uint8_t* borrow(uint8_t* buf, uint32_t* len);
-
-  void consume(uint32_t len);
-
-  boost::shared_ptr<TTransport> getUnderlyingTransport() {
-    return transport_;
-  }
-
- protected:
-  boost::shared_ptr<TTransport> transport_;
-  uint8_t* rBuf_;
-  uint32_t rBufSize_;
-  uint32_t rPos_;
-  uint32_t rLen_;
-
-  uint8_t* wBuf_;
-  uint32_t wBufSize_;
-  uint32_t wLen_;
-};
-
-/**
- * Wraps a transport into a buffered one.
- *
- * @author Mark Slee <mcslee@facebook.com>
- */
-class TBufferedTransportFactory : public TTransportFactory {
- public:
-  TBufferedTransportFactory() {}
-
-  virtual ~TBufferedTransportFactory() {}
-
-  /**
-   * Wraps the transport into a buffered one.
-   */
-  virtual boost::shared_ptr<TTransport> getTransport(boost::shared_ptr<TTransport> trans) {
-    return boost::shared_ptr<TTransport>(new TBufferedTransport(trans));
-  }
-
-};
-
-/**
- * Framed transport. All writes go into an in-memory buffer until flush is
- * called, at which point the transport writes the length of the entire
- * binary chunk followed by the data payload. This allows the receiver on the
- * other end to always do fixed-length reads.
- *
- * @author Mark Slee <mcslee@facebook.com>
- */
-class TFramedTransport : public TTransport {
- public:
-  TFramedTransport(boost::shared_ptr<TTransport> transport) :
-    transport_(transport),
-    rPos_(0),
-    rLen_(0),
-    read_(true),
-    wBufSize_(512),
-    wLen_(0),
-    write_(true) {
-    rBuf_ = NULL;
-    wBuf_ = new uint8_t[wBufSize_];
-  }
-
-  TFramedTransport(boost::shared_ptr<TTransport> transport, uint32_t sz) :
-    transport_(transport),
-    rPos_(0),
-    rLen_(0),
-    read_(true),
-    wBufSize_(sz),
-    wLen_(0),
-    write_(true) {
-    rBuf_ = NULL;
-    wBuf_ = new uint8_t[wBufSize_];
-  }
-
-  ~TFramedTransport() {
-    if (rBuf_ != NULL) {
-      delete [] rBuf_;
-    }
-    if (wBuf_ != NULL) {
-      delete [] wBuf_;
-    }
-  }
-
-  void setRead(bool read) {
-    read_ = read;
-  }
-
-  void setWrite(bool write) {
-    write_ = write;
-  }
-
-  void open() {
-    transport_->open();
-  }
-
-  bool isOpen() {
-    return transport_->isOpen();
-  }
-
-  bool peek() {
-    if (rPos_ < rLen_) {
-      return true;
-    }
-    return transport_->peek();
-  }
-
-  void close() {
-    if (wLen_ > 0) {
-      flush();
-    }
-    transport_->close();
-  }
-
-  uint32_t read(uint8_t* buf, uint32_t len);
-
-  void write(const uint8_t* buf, uint32_t len);
-
-  void flush();
-
-  const uint8_t* borrow(uint8_t* buf, uint32_t* len);
-
-  void consume(uint32_t len);
-
-  boost::shared_ptr<TTransport> getUnderlyingTransport() {
-    return transport_;
-  }
-
- protected:
-  boost::shared_ptr<TTransport> transport_;
-  uint8_t* rBuf_;
-  uint32_t rPos_;
-  uint32_t rLen_;
-  bool read_;
-
-  uint8_t* wBuf_;
-  uint32_t wBufSize_;
-  uint32_t wLen_;
-  bool write_;
-
-  /**
-   * Reads a frame of input from the underlying stream.
-   */
-  void readFrame();
-};
-
-/**
- * Wraps a transport into a framed one.
- *
- * @author Dave Simpson <dave@powerset.com>
- */
-class TFramedTransportFactory : public TTransportFactory {
- public:
-  TFramedTransportFactory() {}
-
-  virtual ~TFramedTransportFactory() {}
-
-  /**
-   * Wraps the transport into a framed one.
-   */
-  virtual boost::shared_ptr<TTransport> getTransport(boost::shared_ptr<TTransport> trans) {
-    return boost::shared_ptr<TTransport>(new TFramedTransport(trans));
-  }
-
-};
-
-
-/**
- * A memory buffer is a tranpsort that simply reads from and writes to an
- * in memory buffer. Anytime you call write on it, the data is simply placed
- * into a buffer, and anytime you call read, data is read from that buffer.
- *
- * The buffers are allocated using C constructs malloc,realloc, and the size
- * doubles as necessary.
- *
- * @author Mark Slee <mcslee@facebook.com>
- * @author David Reiss <dreiss@facebook.com>
- */
-class TMemoryBuffer : public TTransport {
- private:
-
-  // Common initialization done by all constructors.
-  void initCommon(uint8_t* buf, uint32_t size, bool owner, uint32_t wPos) {
-    if (buf == NULL && size != 0) {
-      assert(owner);
-      buf = (uint8_t*)std::malloc(size);
-      if (buf == NULL) {
-        throw TTransportException("Out of memory");
-      }
-    }
-
-    buffer_ = buf;
-    bufferSize_ = size;
-    owner_ = owner;
-    wPos_ = wPos;
-    rPos_ = 0;
-  }
-
-  // make sure there's at least 'len' bytes available for writing
-  void ensureCanWrite(uint32_t len);
-
- public:
-  static const uint32_t defaultSize = 1024;
-
-  /**
-   * This enum specifies how a TMemoryBuffer should treat
-   * memory passed to it via constructors or resetBuffer.
-   *
-   * OBSERVE:
-   *   TMemoryBuffer will simply store a pointer to the memory.
-   *   It is the callers responsibility to ensure that the pointer
-   *   remains valid for the lifetime of the TMemoryBuffer,
-   *   and that it is properly cleaned up.
-   *   Note that no data can be written to observed buffers.
-   *
-   * COPY:
-   *   TMemoryBuffer will make an internal copy of the buffer.
-   *   The caller has no responsibilities.
-   *
-   * TAKE_OWNERSHIP:
-   *   TMemoryBuffer will become the "owner" of the buffer,
-   *   and will be responsible for freeing it.
-   *   The membory must have been allocated with malloc.
-   */
-  enum MemoryPolicy {
-    OBSERVE = 1,
-    COPY = 2,
-    TAKE_OWNERSHIP = 3,
-  };
-
-  /**
-   * Construct a TMemoryBuffer with a default-sized buffer,
-   * owned by the TMemoryBuffer object.
-   */
-  TMemoryBuffer() {
-    initCommon(NULL, defaultSize, true, 0);
-  }
-
-  /**
-   * Construct a TMemoryBuffer with a buffer of a specified size,
-   * owned by the TMemoryBuffer object.
-   *
-   * @param sz  The initial size of the buffer.
-   */
-  TMemoryBuffer(uint32_t sz) {
-    initCommon(NULL, sz, true, 0);
-  }
-
-  /**
-   * Construct a TMemoryBuffer with buf as its initial contents.
-   *
-   * @param buf    The initial contents of the buffer.
-   *               Note that, while buf is a non-const pointer,
-   *               TMemoryBuffer will not write to it if policy == OBSERVE,
-   *               so it is safe to const_cast<uint8_t*>(whatever).
-   * @param sz     The size of @c buf.
-   * @param policy See @link MemoryPolicy @endlink .
-   */
-  TMemoryBuffer(uint8_t* buf, uint32_t sz, MemoryPolicy policy = OBSERVE) {
-    if (buf == NULL && sz != 0) {
-      throw TTransportException(TTransportException::BAD_ARGS,
-                                "TMemoryBuffer given null buffer with non-zero size.");
-    }
-
-    switch (policy) {
-      case OBSERVE:
-      case TAKE_OWNERSHIP:
-        initCommon(buf, sz, policy == TAKE_OWNERSHIP, sz);
-        break;
-      case COPY:
-        initCommon(NULL, sz, true, 0);
-        this->write(buf, sz);
-        break;
-      default:
-        throw TTransportException(TTransportException::BAD_ARGS,
-                                  "Invalid MemoryPolicy for TMemoryBuffer");
-    }
-  }
-
-  ~TMemoryBuffer() {
-    if (owner_) {
-      std::free(buffer_);
-      buffer_ = NULL;
-    }
-  }
-
-  bool isOpen() {
-    return true;
-  }
-
-  bool peek() {
-    return (rPos_ < wPos_);
-  }
-
-  void open() {}
-
-  void close() {}
-
-  // TODO(dreiss): Make bufPtr const.
-  void getBuffer(uint8_t** bufPtr, uint32_t* sz) {
-    *bufPtr = buffer_;
-    *sz = wPos_;
-  }
-
-  std::string getBufferAsString() {
-    if (buffer_ == NULL) {
-      return "";
-    }
-    return std::string((char*)buffer_, (std::string::size_type)wPos_);
-  }
-
-  void appendBufferToString(std::string& str) {
-    if (buffer_ == NULL) {
-      return;
-    }
-    str.append((char*)buffer_, wPos_);
-  }
-
-  void resetBuffer() {
-    wPos_ = 0;
-    rPos_ = 0;
-    // It isn't safe to write into a buffer we don't own.
-    if (!owner_) {
-      bufferSize_ = 0;
-    }
-  }
-
-  /// See constructor documentation.
-  void resetBuffer(uint8_t* buf, uint32_t sz, MemoryPolicy policy = OBSERVE) {
-    // Use a variant of the copy-and-swap trick for assignment operators.
-    // This is sub-optimal in terms of performance for two reasons:
-    //   1/ The constructing and swapping of the (small) values
-    //      in the temporary object takes some time, and is not necessary.
-    //   2/ If policy == COPY, we allocate the new buffer before
-    //      freeing the old one, precluding the possibility of
-    //      reusing that memory.
-    // I doubt that either of these problems could be optimized away,
-    // but the second is probably no a common case, and the first is minor.
-    // I don't expect resetBuffer to be a common operation, so I'm willing to
-    // bite the performance bullet to make the method this simple.
-
-    // Construct the new buffer.
-    TMemoryBuffer new_buffer(buf, sz, policy);
-    // Move it into ourself.
-    this->swap(new_buffer);
-    // Our old self gets destroyed.
-  }
-
-  uint32_t read(uint8_t* buf, uint32_t len);
-
-  std::string readAsString(uint32_t len) {
-    std::string str;
-    (void)readAppendToString(str, len);
-    return str;
-  }
-
-  uint32_t readAppendToString(std::string& str, uint32_t len);
-
-  void readEnd() {
-    if (rPos_ == wPos_) {
-      resetBuffer();
-    }
-  }
-
-  void write(const uint8_t* buf, uint32_t len);
-
-  uint32_t available() const {
-    return wPos_ - rPos_;
-  }
-
-  const uint8_t* borrow(uint8_t* buf, uint32_t* len);
-
-  void consume(uint32_t len);
-
-  void swap(TMemoryBuffer& that) {
-    using std::swap;
-    swap(buffer_,     that.buffer_);
-    swap(bufferSize_, that.bufferSize_);
-    swap(wPos_,       that.wPos_);
-    swap(owner_,      that.owner_);
-  }
-
-  // Returns a pointer to where the client can write data to append to
-  // the TMemoryBuffer, and ensures the buffer is big enough to accomodate a
-  // write of the provided length.  The returned pointer is very convenient for
-  // passing to read(), recv(), or similar. You must call wroteBytes() as soon
-  // as data is written or the buffer will not be aware that data has changed.
-  uint8_t* getWritePtr(uint32_t len) {
-    ensureCanWrite(len);
-    return buffer_ + wPos_;
-  }
-
-  // Informs the buffer that the client has written 'len' bytes into storage
-  // that had been provided by getWritePtr().
-  void wroteBytes(uint32_t len);
-
- private:
-  // Data buffer
-  uint8_t* buffer_;
-
-  // Allocated buffer size
-  uint32_t bufferSize_;
-
-  // Where the write is at
-  uint32_t wPos_;
-
-  // Where the reader is at
-  uint32_t rPos_;
-
-  // Is this object the owner of the buffer?
-  bool owner_;
-
-  // Don't forget to update constrctors, initCommon, and swap if
-  // you add new members.
-};
-
-/**
  * TPipedTransport. This transport allows piping of a request from one
  * transport to another either when readEnd() or writeEnd(). The typical
  * use case for this is to log a request or a reply to disk.