Commit a02ba06a authored by Dan Melnic's avatar Dan Melnic Committed by Facebook GitHub Bot

Add support for Linux TCP zerocopy RX

Summary:
Add support for Linux TCP zerocopy RX

(Note: this ignores all push blocking failures!)

Reviewed By: simpkins

Differential Revision: D32741177

fbshipit-source-id: 412981fdd4a27437d6f22a80dc7c5fc864335ddf
parent 886ed94b
......@@ -37,6 +37,7 @@
#include <folly/io/SocketOptionMap.h>
#include <folly/portability/Fcntl.h>
#include <folly/portability/Sockets.h>
#include <folly/portability/SysMman.h>
#include <folly/portability/SysUio.h>
#include <folly/portability/Unistd.h>
......@@ -46,6 +47,109 @@
#include <sys/ioctl.h>
#endif
using ZeroCopyMemStore = folly::AsyncReader::ReadCallback::ZeroCopyMemStore;
namespace {
class ZeroCopyMMapMemStoreFallback : public ZeroCopyMemStore {
public:
ZeroCopyMMapMemStoreFallback(size_t /*entries*/, size_t /*size*/) {}
~ZeroCopyMMapMemStoreFallback() override = default;
ZeroCopyMemStore::EntryPtr get() override { return nullptr; }
void put(ZeroCopyMemStore::Entry* /*entry*/) override {}
};
#if TCP_ZEROCOPY_RECEIVE
std::unique_ptr<folly::IOBuf> getRXZeroCopyIOBuf(
ZeroCopyMemStore::EntryPtr&& ptr) {
auto* entry = ptr.release();
return folly::IOBuf::takeOwnership(
entry->data,
entry->len,
entry->len,
[](void* /*buf*/, void* userData) {
reinterpret_cast<ZeroCopyMemStore::Entry*>(userData)->put();
},
entry);
}
class ZeroCopyMMapMemStoreReal : public ZeroCopyMemStore {
public:
ZeroCopyMMapMemStoreReal(size_t entries, size_t size) {
// we just need a socket so the kernel
// will set the vma->vm_ops = &tcp_vm_ops
int fd = ::socket(AF_INET, SOCK_STREAM, 0);
if (fd >= 0) {
void* addr =
::mmap(nullptr, entries * size, PROT_READ, MAP_SHARED, fd, 0);
::close(fd);
if (addr != MAP_FAILED) {
addr_ = addr;
numEntries_ = entries;
entrySize_ = size;
entries_.resize(numEntries_);
for (size_t i = 0; i < numEntries_; i++) {
entries_[i].data =
reinterpret_cast<uint8_t*>(addr_) + (i * entrySize_);
entries_[i].capacity = entrySize_;
entries_[i].store = this;
avail_.push_back(&entries_[i]);
}
}
}
}
~ZeroCopyMMapMemStoreReal() override {
CHECK_EQ(avail_.size(), numEntries_);
if (addr_) {
::munmap(addr_, numEntries_ * entrySize_);
}
}
ZeroCopyMemStore::EntryPtr get() override {
std::unique_lock<std::mutex> lk(availMutex_);
if (!avail_.empty()) {
auto* entry = avail_.front();
avail_.pop_front();
DCHECK(entry->len == 0);
DCHECK(entry->capacity == entrySize_);
ZeroCopyMemStore::EntryPtr ret(entry);
return ret;
}
return nullptr;
}
void put(ZeroCopyMemStore::Entry* entry) override {
if (entry) {
DCHECK(entry->store == this);
if (entry->len) {
auto ret = ::madvise(entry->data, entry->len, MADV_DONTNEED);
entry->len = 0;
DCHECK(!ret);
}
std::unique_lock<std::mutex> lk(availMutex_);
avail_.push_back(entry);
}
}
private:
std::vector<ZeroCopyMemStore::Entry> entries_;
std::mutex availMutex_;
std::deque<ZeroCopyMemStore::Entry*> avail_;
void* addr_{nullptr};
size_t numEntries_{0};
size_t entrySize_{0};
};
using ZeroCopyMMapMemStore = ZeroCopyMMapMemStoreReal;
#else
using ZeroCopyMMapMemStore = ZeroCopyMMapMemStoreFallback;
#endif
} // namespace
#if FOLLY_HAVE_VLA
#define FOLLY_HAVE_VLA_01 1
#else
......@@ -66,6 +170,11 @@ static constexpr bool msgErrQueueSupported =
false;
#endif // FOLLY_HAVE_MSG_ERRQUEUE
std::unique_ptr<ZeroCopyMemStore> AsyncSocket::createDefaultZeroCopyMemStore(
size_t entries, size_t size) {
return std::make_unique<ZeroCopyMMapMemStore>(entries, size);
}
static AsyncSocketException const& getSocketClosedLocallyEx() {
static auto& ex = *new AsyncSocketException(
AsyncSocketException::END_OF_FILE, "socket closed locally");
......@@ -2634,47 +2743,124 @@ void AsyncSocket::splitIovecArray(
CHECK_EQ(targetBytes, dstBytes);
}
void AsyncSocket::handleRead() noexcept {
VLOG(5) << "AsyncSocket::handleRead() this=" << this << ", fd=" << fd_
<< ", state=" << state_;
assert(state_ == StateEnum::ESTABLISHED);
assert((shutdownFlags_ & SHUT_READ) == 0);
assert(readCallback_ != nullptr);
assert(eventFlags_ & EventHandler::READ);
AsyncSocket::ReadCode AsyncSocket::processZeroCopyRead() {
#if TCP_ZEROCOPY_RECEIVE
if (!zerocopyReadSupported_) {
return ReadCode::READ_NOT_SUPPORTED;
}
// Loop until:
// - a read attempt would block
// - readCallback_ is uninstalled
// - the number of loop iterations exceeds the optional maximum
// - this AsyncSocket is moved to another EventBase
//
// When we invoke readDataAvailable() it may uninstall the readCallback_,
// which is why need to check for it here.
//
// The last bullet point is slightly subtle. readDataAvailable() may also
// detach this socket from this EventBase. However, before
// readDataAvailable() returns another thread may pick it up, attach it to
// a different EventBase, and install another readCallback_. We need to
// exit immediately after readDataAvailable() returns if the eventBase_ has
// changed. (The caller must perform some sort of locking to transfer the
// AsyncSocket between threads properly. This will be sufficient to ensure
// that this thread sees the updated eventBase_ variable after
// readDataAvailable() returns.)
size_t numReads = maxReadsPerEvent_ ? maxReadsPerEvent_ : size_t(-1);
EventBase* originalEventBase = eventBase_;
while (readCallback_ && eventBase_ == originalEventBase && numReads--) {
auto* memStore = readCallback_->readZeroCopyEnabled();
if (!memStore) {
// set zerocopyReadSupported_ to false to avoid further virtual calls
zerocopyReadSupported_ = false;
return ReadCode::READ_NOT_SUPPORTED;
}
if (preReceivedData_ && !preReceivedData_->empty()) {
VLOG(5) << "AsyncSocket::performReadInternal() this=" << this
<< ", reading pre-received data";
readCallback_->readZeroCopyDataAvailable(
std::move(preReceivedData_), 0 /*additionalBytes*/);
return ReadCode::READ_DONE;
}
auto ptr = memStore->get();
void* copybuf = nullptr;
size_t copybuf_len = 0;
readCallback_->getZeroCopyFallbackBuffer(&copybuf, &copybuf_len);
folly::netops::tcp_zerocopy_receive zc = {};
socklen_t zc_len = sizeof(zc);
zc.address = reinterpret_cast<uint64_t>(ptr->data);
zc.length = ptr->capacity;
auto zc_length = zc.length;
zc.copybuf_address = reinterpret_cast<__u64>(copybuf);
zc.copybuf_len = copybuf_len;
auto zc_copybuf_len = zc.copybuf_len;
auto ret =
::getsockopt(fd_.toFd(), IPPROTO_TCP, TCP_ZEROCOPY_RECEIVE, &zc, &zc_len);
if (!ret) {
// check for errors
if (zc.err) {
readErr_ = READ_ERROR;
AsyncSocketException ex(
AsyncSocketException::INTERNAL_ERROR,
withAddr(" TCP_ZEROCOPY_RECEIVE) failed"),
zc.err);
return failRead(__func__, ex);
}
std::unique_ptr<folly::IOBuf> buf;
if (zc.length) {
// adjust the len
ptr->len = zc.length;
auto tmp = getRXZeroCopyIOBuf(std::move(ptr));
buf = std::move(tmp);
}
auto len = zc.length + zc.copybuf_len;
if (len) {
readCallback_->readZeroCopyDataAvailable(std::move(buf), zc.copybuf_len);
// if there is no buffer data, we continue if we filled up the zerocopy
// buffer otherwise we're done
if (zc.copybuf_len == 0) {
return (zc.length == zc_length) ? ReadCode::READ_CONTINUE
: ReadCode::READ_DONE;
}
// we continue if we filled up the copy buffer
return (zc.copybuf_len == zc_copybuf_len) ? ReadCode::READ_CONTINUE
: ReadCode::READ_DONE;
} else {
// No more data to read right now.
return ReadCode::READ_DONE;
}
} else {
if (errno == EIO) {
// EOF
readErr_ = READ_EOF;
// EOF
shutdownFlags_ |= SHUT_READ;
if (!updateEventRegistration(0, EventHandler::READ)) {
// we've already been moved into STATE_ERROR
assert(state_ == StateEnum::ERROR);
assert(readCallback_ == nullptr);
return ReadCode::READ_DONE;
}
ReadCallback* callback = readCallback_;
readCallback_ = nullptr;
callback->readEOF();
return ReadCode::READ_DONE;
}
// treat any other error as not supported, fall back to regular read
zerocopyReadSupported_ = false;
}
#endif
return ReadCode::READ_NOT_SUPPORTED;
}
AsyncSocket::ReadCode AsyncSocket::processNormalRead() {
auto readMode = readCallback_->getReadMode();
// Get the buffer(s) to read into.
void* buf = nullptr;
size_t buflen = 0, offset = 0, num = 0;
IOBufIovecBuilder::IoVecVec iovs; // this can be an Asyncsocket member too
size_t offset = 0, num = 0;
size_t buflen = 0;
IOBufIovecBuilder::IoVecVec iovs; // this can be an AsyncSocket member too
try {
if (readMode == AsyncReader::ReadCallback::ReadMode::ReadVec) {
prepareReadBuffers(iovs);
num = iovs.size();
VLOG(5) << "prepareReadBuffers() bufs=" << iovs.data()
<< ", num=" << num;
VLOG(5) << "prepareReadBuffers() bufs=" << iovs.data() << ", num=" << num;
} else {
prepareReadBuffer(&buf, &buflen);
VLOG(5) << "prepareReadBuffer() buf=" << buf << ", buflen=" << buflen;
......@@ -2707,22 +2893,20 @@ void AsyncSocket::handleRead() noexcept {
auto readResult = (readMode == AsyncReader::ReadCallback::ReadMode::ReadVec)
? performReadv(iovs.data(), num)
: performRead(&buf, &buflen, &offset);
auto bytesRead = readResult.readReturn;
VLOG(4) << "this=" << this << ", AsyncSocket::handleRead() got "
<< bytesRead << " bytes";
VLOG(4) << "this=" << this << ", AsyncSocket::handleRead() got " << bytesRead
<< " bytes";
if (bytesRead > 0) {
readCallback_->readDataAvailable(size_t(bytesRead));
// Fall through and continue around the loop if the read
// completely filled the available buffer.
// Note that readCallback_ may have been uninstalled or changed inside
// readDataAvailable().
if (size_t(bytesRead) < buflen) {
return;
}
// Continue reading if we filled the available buffer
return (size_t(bytesRead) < buflen) ? ReadCode::READ_DONE
: ReadCode::READ_CONTINUE;
} else if (bytesRead == READ_BLOCKING) {
// No more data to read right now.
return;
return ReadCode::READ_DONE;
} else if (bytesRead == READ_ERROR) {
readErr_ = READ_ERROR;
if (readResult.exception) {
......@@ -2743,29 +2927,75 @@ void AsyncSocket::handleRead() noexcept {
// we've already been moved into STATE_ERROR
assert(state_ == StateEnum::ERROR);
assert(readCallback_ == nullptr);
return;
return ReadCode::READ_DONE;
}
ReadCallback* callback = readCallback_;
readCallback_ = nullptr;
callback->readEOF();
return;
return ReadCode::READ_DONE;
}
return ReadCode::READ_DONE; // redundant
}
void AsyncSocket::handleRead() noexcept {
VLOG(5) << "AsyncSocket::handleRead() this=" << this << ", fd=" << fd_
<< ", state=" << state_;
assert(state_ == StateEnum::ESTABLISHED);
assert((shutdownFlags_ & SHUT_READ) == 0);
assert(readCallback_ != nullptr);
assert(eventFlags_ & EventHandler::READ);
// Loop until:
// - a read attempt would block
// - readCallback_ is uninstalled
// - the number of loop iterations exceeds the optional maximum
// - this AsyncSocket is moved to another EventBase
//
// When we invoke readDataAvailable() it may uninstall the readCallback_,
// which is why need to check for it here.
//
// The last bullet point is slightly subtle. readDataAvailable() may also
// detach this socket from this EventBase. However, before
// readDataAvailable() returns another thread may pick it up, attach it to
// a different EventBase, and install another readCallback_. We need to
// exit immediately after readDataAvailable() returns if the eventBase_ has
// changed. (The caller must perform some sort of locking to transfer the
// AsyncSocket between threads properly. This will be sufficient to ensure
// that this thread sees the updated eventBase_ variable after
// readDataAvailable() returns.)
size_t numReads = maxReadsPerEvent_ ? maxReadsPerEvent_ : size_t(-1);
EventBase* originalEventBase = eventBase_;
while (readCallback_ && eventBase_ == originalEventBase && numReads--) {
auto ret = processZeroCopyRead();
if (ret == ReadCode::READ_NOT_SUPPORTED) {
ret = processNormalRead();
}
switch (ret) {
case ReadCode::READ_NOT_SUPPORTED:
CHECK(false);
case ReadCode::READ_CONTINUE:
break;
case ReadCode::READ_DONE:
return;
};
if (readCallback_ && eventBase_ == originalEventBase) {
// We might still have data in the socket.
// (e.g. see comment in AsyncSSLSocket::checkForImmediateRead)
scheduleImmediateRead();
}
}
}
/**
* This function attempts to write as much data as possible, until no more data
* can be written.
* This function attempts to write as much data as possible, until no more
* data can be written.
*
* - If it sends all available data, it unregisters for write events, and stops
* the writeTimeout_.
* - If it sends all available data, it unregisters for write events, and
* stops the writeTimeout_.
*
* - If not all of the data can be sent immediately, it reschedules
* writeTimeout_ (if a non-zero timeout is set), and ensures the handler is
......@@ -2811,9 +3041,9 @@ void AsyncSocket::handleWrite() noexcept {
writeReqTail_ = nullptr;
// This is the last write request.
// Unregister for write events and cancel the send timer
// before we invoke the callback. We have to update the state properly
// before calling the callback, since it may want to detach us from
// the EventBase.
// before we invoke the callback. We have to update the state
// properly before calling the callback, since it may want to detach
// us from the EventBase.
if (eventFlags_ & EventHandler::WRITE) {
if (!updateEventRegistration(0, EventHandler::WRITE)) {
assert(state_ == StateEnum::ERROR);
......@@ -2871,8 +3101,8 @@ void AsyncSocket::handleWrite() noexcept {
if (bufferCallback_) {
bufferCallback_->onEgressBuffered();
}
// Stop after a partial write; it's highly likely that a subsequent write
// attempt will just return EAGAIN.
// Stop after a partial write; it's highly likely that a subsequent
// write attempt will just return EAGAIN.
//
// Ensure that we are registered for write events.
if ((eventFlags_ & EventHandler::WRITE) == 0) {
......@@ -2911,8 +3141,8 @@ void AsyncSocket::checkForImmediateRead() noexcept {
//
// Checking if the socket is readable now also seems like it would probably
// be a pessimism. In most cases it probably wouldn't be readable, and we
// would just waste an extra system call. Even if it is readable, waiting to
// find out from libevent on the next event loop doesn't seem that bad.
// would just waste an extra system call. Even if it is readable, waiting
// to find out from libevent on the next event loop doesn't seem that bad.
//
// The exception to this is if we have pre-received data. In that case there
// is definitely data available immediately.
......@@ -3004,9 +3234,9 @@ void AsyncSocket::handleConnect() noexcept {
// If SHUT_WRITE_PENDING is set and we don't have any write requests to
// perform, immediately shutdown the write half of the socket.
if ((shutdownFlags_ & SHUT_WRITE_PENDING) && writeReqHead_ == nullptr) {
// SHUT_READ shouldn't be set. If close() is called on the socket while we
// are still connecting we just abort the connect rather than waiting for
// it to complete.
// SHUT_READ shouldn't be set. If close() is called on the socket while
// we are still connecting we just abort the connect rather than waiting
// for it to complete.
assert((shutdownFlags_ & SHUT_READ) == 0);
netops_->shutdown(fd_, SHUT_WR);
shutdownFlags_ |= SHUT_WRITE;
......@@ -3154,7 +3384,8 @@ AsyncSocket::WriteResult AsyncSocket::sendSocketMessage(
}
}
// if maybeOffsetToSplitWrite points to end of the vector, remove the split
// if maybeOffsetToSplitWrite points to end of the vector, remove the
// split
if (mergedRequest.maybeOffsetToSplitWrite.has_value() && // explicit
mergedRequest.maybeOffsetToSplitWrite == endOffset) {
mergedRequest.maybeOffsetToSplitWrite.reset(); // no split needed
......@@ -3402,8 +3633,8 @@ AsyncSocket::WriteResult AsyncSocket::performWrite(
/**
* Re-register the EventHandler after eventFlags_ has changed.
*
* If an error occurs, fail() is called to move the socket into the error state
* and call all currently installed callbacks. After an error, the
* If an error occurs, fail() is called to move the socket into the error
* state and call all currently installed callbacks. After an error, the
* AsyncSocket is completely unregistered.
*
* @return Returns true on success, or false on error.
......@@ -3520,7 +3751,8 @@ void AsyncSocket::failConnect(const char* fn, const AsyncSocketException& ex) {
finishFail(ex);
}
void AsyncSocket::failRead(const char* fn, const AsyncSocketException& ex) {
AsyncSocket::ReadCode AsyncSocket::failRead(
const char* fn, const AsyncSocketException& ex) {
VLOG(5) << "AsyncSocket(this=" << this << ", fd=" << fd_
<< ", state=" << state_ << " host=" << addr_.describe()
<< "): failed while reading in " << fn << "(): " << ex.what();
......@@ -3533,6 +3765,9 @@ void AsyncSocket::failRead(const char* fn, const AsyncSocketException& ex) {
}
finishFail(ex);
// done handling the error, we can exit the loop
return AsyncSocket::ReadCode::READ_DONE;
}
void AsyncSocket::failErrMessageRead(
......@@ -3558,8 +3793,8 @@ void AsyncSocket::failWrite(const char* fn, const AsyncSocketException& ex) {
startFail();
// Only invoke the first write callback, since the error occurred while
// writing this request. Let any other pending write callbacks be invoked in
// finishFail().
// writing this request. Let any other pending write callbacks be invoked
// in finishFail().
if (writeReqHead_ != nullptr) {
WriteRequest* req = writeReqHead_;
writeReqHead_ = req->getNext();
......@@ -3694,9 +3929,9 @@ void AsyncSocket::invokeConnectErr(const AsyncSocketException& ex) {
connectEndTime_ = std::chrono::steady_clock::now();
if ((state_ == StateEnum::CONNECTING) || (state_ == StateEnum::ERROR)) {
// invokeConnectErr() can be invoked when state is {FAST_OPEN, CLOSED,
// ESTABLISHED} (!?) and a bunch of other places that are not what this call
// back wants. This seems like a bug but work around here while we explore
// it independently
// ESTABLISHED} (!?) and a bunch of other places that are not what this
// call back wants. This seems like a bug but work around here while we
// explore it independently
for (const auto& cb : lifecycleObservers_) {
cb->connectError(this, ex);
}
......
......@@ -651,6 +651,31 @@ class AsyncSocket : public AsyncTransport {
}
}
/**
* Create a memory store to use for zero copy reads.
*
* The memory store contains a fixed number of entries, each with a fixed
* size. When data is read using zero-copy the kernel will place it in one
* of these entries, and it will be returned to the callback with
* readZeroCopyDataAvailable(). The callback must release the IOBuf
* reference to make the entry available again for future zero-copy reads.
* If all entries are exhausted the read code will fall back to non-zero-copy
* reads.
*
* Note: it is the caller's responsibility to ensure that they do not destroy
* the ZeroCopyMemStore while it still has any outstanding entries in use.
* The caller must ensure the ZeroCopyMemStore exists until all callers have
* finished using any data returned via zero-copy reads, and released the
* IOBuf objects containing that data.
*
* @param entries The number of entries to allocate in the memory store.
* @param size The size of each entry, in bytes. This should be a
* multiple of the kernel page size.
*/
static std::unique_ptr<AsyncReader::ReadCallback::ZeroCopyMemStore>
createDefaultZeroCopyMemStore(size_t entries, size_t size);
bool setZeroCopy(bool enable) override;
bool getZeroCopy() const override { return zeroCopyEnabled_; }
......@@ -1467,13 +1492,19 @@ class AsyncSocket : public AsyncTransport {
void doClose();
// error handling methods
enum class ReadCode {
READ_NOT_SUPPORTED = 0,
READ_CONTINUE = 1,
READ_DONE = 2,
};
void startFail();
void finishFail();
void finishFail(const AsyncSocketException& ex);
void invokeAllErrors(const AsyncSocketException& ex);
void fail(const char* fn, const AsyncSocketException& ex);
void failConnect(const char* fn, const AsyncSocketException& ex);
void failRead(const char* fn, const AsyncSocketException& ex);
ReadCode failRead(const char* fn, const AsyncSocketException& ex);
void failErrMessageRead(const char* fn, const AsyncSocketException& ex);
void failWrite(
const char* fn,
......@@ -1517,6 +1548,8 @@ class AsyncSocket : public AsyncTransport {
void releaseIOBuf(
std::unique_ptr<folly::IOBuf> buf, ReleaseIOBufCallback* callback);
ReadCode processZeroCopyRead();
ReadCode processNormalRead();
/**
* Attempt to enable Observer ByteEvents for this socket.
*
......@@ -1630,6 +1663,9 @@ class AsyncSocket : public AsyncTransport {
size_t zeroCopyReenableThreshold_{0};
size_t zeroCopyReenableCounter_{0};
// zerocopy read
bool zerocopyReadSupported_{true};
// subclasses may cache these on first call to get
mutable std::unique_ptr<const AsyncTransportCertificate> peerCertData_{
nullptr};
......
......@@ -238,6 +238,79 @@ class AsyncReader {
virtual void readDataAvailable(size_t len) noexcept = 0;
class ZeroCopyMemStore {
public:
struct Entry {
void* data{nullptr};
size_t len{0}; // in use
size_t capacity{0}; // capacity
ZeroCopyMemStore* store{nullptr};
void put() {
DCHECK(store);
store->put(this);
}
};
struct EntryDeleter {
void operator()(Entry* entry) { entry->put(); }
};
using EntryPtr = std::unique_ptr<Entry, EntryDeleter>;
virtual ~ZeroCopyMemStore() = default;
virtual EntryPtr get() = 0;
virtual void put(Entry*) = 0;
};
/* the next 4 methods can be used if the callback wants to support zerocopy
* RX on Linux as described in https://lwn.net/Articles/754681/ If the
* current kernel version does not support zerocopy RX, the callback will
* revert to regular recv processing
* In case we support zerocopy RX, the callback might be notified of buffer
* chains composed of mmap memory and also memory allocated via the
* getZeroCopyReadBuffer method
*/
/**
* Return a ZeroCopyMemStore to use if the callback would like to enable
* zero-copy reads. Return nullptr to disable zero-copy reads.
*
* The caller must ensure that the ZeroCopyMemStore remains valid for as
* long as this callback is installed and reading data, and until put()
* has been called for every outstanding Entry allocated with get().
*/
virtual ZeroCopyMemStore* readZeroCopyEnabled() noexcept { return nullptr; }
/**
* Get a buffer to read data into when using zero-copy reads if some data
* cannot be read using a zero-copy page.
*
* When data is available, some data may be returned in zero-copy pages,
* followed by some amount of data in this fallback buffer.
*/
virtual void getZeroCopyFallbackBuffer(
void** /*bufReturn*/, size_t* /*lenReturn*/) noexcept {
CHECK(false);
}
/**
* readZeroCopyDataAvailable() will be called when data is available from a
* zero-copy read.
*
* The data returned may be in two separate parts: data that was actually
* read using zero copy pages will be in zeroCopyData. Additionally, some
* number of bytes may have been placed in the fallback buffer returned by
* getZeroCopyFallbackBuffer(). additionalBytes indicates the number of
* bytes placed in getZeroCopyFallbackBuffer().
*/
virtual void readZeroCopyDataAvailable(
std::unique_ptr<IOBuf>&& /*zeroCopyData*/,
size_t /*additionalBytes*/) noexcept {
CHECK(false);
}
/**
* When data becomes available, isBufferMovable() will be invoked to figure
* out which API will be used, readBufferAvailable() or
......@@ -411,6 +484,17 @@ class AsyncWriter {
virtual bool getZeroCopy() const { return false; }
struct RXZerocopyParams {
bool enable{false};
size_t mapSize{0};
};
FOLLY_NODISCARD virtual bool setRXZeroCopy(RXZerocopyParams /*params*/) {
return false;
}
FOLLY_NODISCARD virtual bool getRXZeroCopy() const { return false; }
using ZeroCopyEnableFunc =
std::function<bool(const std::unique_ptr<folly::IOBuf>& buf)>;
......
......@@ -298,6 +298,147 @@ class BufferCallback : public folly::AsyncTransport::BufferCallback {
bool bufferCleared_{false};
};
class ZeroCopyReadCallback : public folly::AsyncTransport::ReadCallback {
public:
explicit ZeroCopyReadCallback(
folly::AsyncTransport::ReadCallback::ZeroCopyMemStore* memStore,
size_t _maxBufferSz = 4096)
: memStore_(memStore),
state(STATE_WAITING),
exception(folly::AsyncSocketException::UNKNOWN, "none"),
maxBufferSz(_maxBufferSz) {}
~ZeroCopyReadCallback() override { currentBuffer.free(); }
// zerocopy
folly::AsyncTransport::ReadCallback::ZeroCopyMemStore*
readZeroCopyEnabled() noexcept override {
return memStore_;
}
void getZeroCopyFallbackBuffer(
void** bufReturn, size_t* lenReturn) noexcept override {
if (!currentZeroCopyBuffer.buffer) {
currentZeroCopyBuffer.allocate(maxBufferSz);
}
*bufReturn = currentZeroCopyBuffer.buffer;
*lenReturn = currentZeroCopyBuffer.length;
}
void readZeroCopyDataAvailable(
std::unique_ptr<folly::IOBuf>&& zeroCopyData,
size_t additionalBytes) noexcept override {
auto ioBuf = std::move(zeroCopyData);
if (additionalBytes) {
auto tmp = folly::IOBuf::takeOwnership(
currentZeroCopyBuffer.buffer,
currentZeroCopyBuffer.length,
0,
additionalBytes);
currentZeroCopyBuffer.reset();
if (ioBuf) {
ioBuf->prependChain(std::move(tmp));
} else {
ioBuf = std::move(tmp);
}
}
if (!data_) {
data_ = std::move(ioBuf);
} else {
data_->prependChain(std::move(ioBuf));
}
}
void getReadBuffer(void** bufReturn, size_t* lenReturn) override {
if (!currentBuffer.buffer) {
currentBuffer.allocate(maxBufferSz);
}
*bufReturn = currentBuffer.buffer;
*lenReturn = currentBuffer.length;
}
void readDataAvailable(size_t len) noexcept override {
auto ioBuf = folly::IOBuf::takeOwnership(
currentBuffer.buffer, currentBuffer.length, 0, len);
currentBuffer.reset();
if (!data_) {
data_ = std::move(ioBuf);
} else {
data_->prependChain(std::move(ioBuf));
}
}
void readEOF() noexcept override { state = STATE_SUCCEEDED; }
void readErr(const folly::AsyncSocketException& ex) noexcept override {
state = STATE_FAILED;
exception = ex;
}
void verifyData(const std::string& expected) const {
verifyData((const unsigned char*)expected.data(), expected.size());
}
void verifyData(const unsigned char* expected, size_t expectedLen) const {
CHECK(!!data_);
auto len = data_->computeChainDataLength();
CHECK_EQ(len, expectedLen);
auto* buf = data_.get();
auto* current = buf;
size_t offset = 0;
do {
size_t cmpLen = std::min(current->length(), expectedLen - offset);
CHECK_EQ(cmpLen, current->length());
CHECK_EQ(memcmp(current->data(), expected + offset, cmpLen), 0);
offset += cmpLen;
current = current->next();
} while (current != buf);
std::ignore = expected;
CHECK_EQ(offset, expectedLen);
}
class Buffer {
public:
Buffer() = default;
Buffer(char* buf, size_t len) : buffer(buf), length(len) {}
~Buffer() {
if (buffer) {
::free(buffer);
}
}
void reset() {
buffer = nullptr;
length = 0;
}
void allocate(size_t len) {
CHECK(buffer == nullptr);
buffer = static_cast<char*>(malloc(len));
length = len;
}
void free() {
::free(buffer);
reset();
}
char* buffer{nullptr};
size_t length{0};
};
folly::AsyncTransport::ReadCallback::ZeroCopyMemStore* memStore_;
StateEnum state;
folly::AsyncSocketException exception;
Buffer currentBuffer, currentZeroCopyBuffer;
VoidCallback dataAvailableCallback;
const size_t maxBufferSz;
std::unique_ptr<folly::IOBuf> data_;
};
class ReadVerifier {};
class TestSendMsgParamsCallback
......
......@@ -631,6 +631,59 @@ TEST_P(AsyncSocketConnectTest, ConnectAndReadv) {
ASSERT_FALSE(socket->isClosedByPeer());
}
TEST_P(AsyncSocketConnectTest, ConnectAndZeroCopyRead) {
TestServer server;
// connect()
EventBase evb;
std::shared_ptr<AsyncSocket> socket = AsyncSocket::newSocket(&evb);
if (GetParam() == TFOState::ENABLED) {
socket->enableTFO();
}
ConnCallback ccb;
socket->connect(&ccb, server.getAddress(), 30);
static constexpr size_t kBuffSize = 4096;
static constexpr size_t kDataSize = 128 * 1024;
static constexpr size_t kNumEntries = 1024;
static constexpr size_t kEntrySize = 128 * 1024;
auto memStore =
AsyncSocket::createDefaultZeroCopyMemStore(kNumEntries, kEntrySize);
ZeroCopyReadCallback rcb(memStore.get(), kBuffSize);
socket->setReadCB(&rcb);
if (GetParam() == TFOState::ENABLED) {
// Trigger a connection
socket->writeChain(nullptr, IOBuf::copyBuffer("hey"));
}
// Even though we haven't looped yet, we should be able to accept
// the connection and send data to it.
std::shared_ptr<BlockingSocket> acceptedSocket = server.accept();
std::string data(kDataSize, ' ');
// generate random data
std::mt19937 rng(folly::randomNumberSeed());
for (size_t i = 0; i < data.size(); ++i) {
data[i] = static_cast<char>(rng());
}
acceptedSocket->write(
reinterpret_cast<unsigned char*>(data.data()), data.size());
acceptedSocket->flush();
acceptedSocket->close();
// Loop
evb.loop();
ASSERT_EQ(ccb.state, STATE_SUCCEEDED);
rcb.verifyData(data);
ASSERT_FALSE(socket->isClosedBySelf());
ASSERT_FALSE(socket->isClosedByPeer());
}
/**
* Test installing a read callback and then closing immediately before the
* connect attempt finishes.
......
......@@ -40,6 +40,7 @@ using sa_family_t = ADDRESS_FAMILY;
#define SOL_UDP 0x0
#define UDP_SEGMENT 0x0
#define IP_BIND_ADDRESS_NO_PORT 0
#define TCP_ZEROCOPY_RECEIVE 0
// We don't actually support either of these flags
// currently.
......@@ -139,6 +140,11 @@ struct mmsghdr {
#define FOLLY_HAVE_MSG_ERRQUEUE 1
#ifndef FOLLY_HAVE_SO_TIMESTAMPING
#define FOLLY_HAVE_SO_TIMESTAMPING 1
#ifndef TCP_ZEROCOPY_RECEIVE
#define TCP_ZEROCOPY_RECEIVE 35
#endif
#else
#define TCP_ZEROCOPY_RECEIVE 0
#endif
/* for struct sock_extended_err*/
#include <linux/errqueue.h>
......@@ -207,6 +213,24 @@ struct sock_txtime {
__kernel_clockid_t clockid; /* reference clockid */
__u32 flags; /* as defined by enum txtime_flags */
};
/* Copied from uapi/linux/tcp.h */
/* setsockopt(fd, IPPROTO_TCP, TCP_ZEROCOPY_RECEIVE, ...) */
struct tcp_zerocopy_receive {
__u64 address; /* in: address of mapping */
__u32 length; /* in/out: number of bytes to map/mapped */
__u32 recv_skip_hint; /* out: amount of bytes to skip */
__u32 inq; /* out: amount of bytes in read queue */
__s32 err; /* out: socket error */
__u64 copybuf_address; /* in: copybuf address (small reads) */
__s32 copybuf_len; /* in/out: copybuf bytes avail/used or error */
__u32 flags; /* in: flags */
__u64 msg_control; /* ancillary data */
__u64 msg_controllen;
__u32 msg_flags;
__u32 reserved; /* set to 0 for now */
};
} // namespace netops
} // namespace folly
#endif
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment