Commit 0bf12e84 authored by Zeyi (Rice) Fan's avatar Zeyi (Rice) Fan Committed by Facebook Github Bot

Split asynchronous writing logic out from AsyncFileWriter

Summary:
This diff splits `AsyncFileWriter` into two parts: `AsyncLogWriter` and `AsyncFileWriter`, so we could re-use the asynchronous IO operating logic with other types of log writer. See the following diffs for example use case.

`AsyncLogWriter` is a virtual class that provides the asynchronous logic and manages a queue of incoming log messages. The users can write subclasses with this class to provided their own IO implementation by overriding the virtual method `performIO`.

`AsyncFileWriter` is one implementation of the `AsyncLogWriter` that writes incoming log messages to a file.

Most of this diff is simply moving things around. Substantial changes are placed in next diff.

Reviewed By: chadaustin

Differential Revision: D15228054

fbshipit-source-id: 86c3405755099c59aa2df8a310250fded648e413
parent 6fe1af4a
......@@ -17,170 +17,23 @@
#include <folly/Exception.h>
#include <folly/FileUtil.h>
#include <folly/detail/AtFork.h>
#include <folly/logging/LoggerDB.h>
#include <folly/system/ThreadName.h>
using folly::File;
using folly::StringPiece;
namespace folly {
constexpr size_t AsyncFileWriter::kDefaultMaxBufferSize;
AsyncFileWriter::AsyncFileWriter(StringPiece path)
: AsyncFileWriter{File{path.str(), O_WRONLY | O_APPEND | O_CREAT}} {}
AsyncFileWriter::AsyncFileWriter(folly::File&& file) : file_{std::move(file)} {
folly::detail::AtFork::registerHandler(
this,
[this] { return preFork(); },
[this] { postForkParent(); },
[this] { postForkChild(); });
// Start the I/O thread after registering the atfork handler.
// preFork() may be invoked in another thread as soon as registerHandler()
// returns. It will check FLAG_IO_THREAD_STARTED to see if the I/O thread is
// running yet.
{
auto data = data_.lock();
data->flags |= FLAG_IO_THREAD_STARTED;
data->ioThread = std::thread([this] { ioThread(); });
}
}
AsyncFileWriter::AsyncFileWriter(folly::File&& file) : file_{std::move(file)} {}
AsyncFileWriter::~AsyncFileWriter() {
std::vector<std::string>* ioQueue;
size_t numDiscarded;
{
// Stop the I/O thread
auto data = data_.lock();
stopIoThread(data, FLAG_DESTROYING);
// stopIoThread() causes the I/O thread to stop as soon as possible,
// without waiting for all pending messages to be written. Extract any
// remaining messages to write them below.
ioQueue = data->getCurrentQueue();
numDiscarded = data->numDiscarded;
}
// Unregister the atfork handler after stopping the I/O thread.
// preFork(), postForkParent(), and postForkChild() calls can run
// concurrently with the destructor until unregisterHandler() returns.
folly::detail::AtFork::unregisterHandler(this);
// If there are still any pending messages, flush them now.
if (!ioQueue->empty()) {
try {
performIO(ioQueue, numDiscarded);
} catch (const std::exception& ex) {
onIoError(ex);
}
}
cleanup();
}
bool AsyncFileWriter::ttyOutput() const {
return isatty(file_.fd());
}
void AsyncFileWriter::writeMessage(StringPiece buffer, uint32_t flags) {
return writeMessage(buffer.str(), flags);
}
void AsyncFileWriter::writeMessage(std::string&& buffer, uint32_t flags) {
auto data = data_.lock();
if ((data->currentBufferSize >= data->maxBufferBytes) &&
!(flags & NEVER_DISCARD)) {
++data->numDiscarded;
return;
}
data->currentBufferSize += buffer.size();
auto* queue = data->getCurrentQueue();
queue->emplace_back(std::move(buffer));
messageReady_.notify_one();
}
void AsyncFileWriter::flush() {
auto data = data_.lock();
auto start = data->ioThreadCounter;
// Wait until ioThreadCounter increments by at least two.
// Waiting for a single increment is not sufficient, as this happens after
// the I/O thread has swapped the queues, which is before it has actually
// done the I/O.
while (data->ioThreadCounter < start + 2) {
// Enqueue an empty string and wake the I/O thread.
// The empty string ensures that the I/O thread will break out of its wait
// loop and increment the ioThreadCounter, even if there is no other work
// to do.
data->getCurrentQueue()->emplace_back();
messageReady_.notify_one();
// Wait for notification from the I/O thread that it has done work.
ioCV_.wait(data.getUniqueLock());
}
}
void AsyncFileWriter::setMaxBufferSize(size_t size) {
auto data = data_.lock();
data->maxBufferBytes = size;
}
size_t AsyncFileWriter::getMaxBufferSize() const {
auto data = data_.lock();
return data->maxBufferBytes;
}
void AsyncFileWriter::ioThread() {
folly::setThreadName("log_writer");
while (true) {
// With the lock held, grab a pointer to the current queue, then increment
// the ioThreadCounter index so that other threads will write into the
// other queue as we process this one.
std::vector<std::string>* ioQueue;
size_t numDiscarded;
{
auto data = data_.lock();
ioQueue = data->getCurrentQueue();
while (ioQueue->empty() && !(data->flags & FLAG_STOP)) {
// Wait for a message or one of the above flags to be set.
messageReady_.wait(data.getUniqueLock());
}
if (data->flags & FLAG_STOP) {
// We have been asked to stop. We exit immediately in this case
// without writing out any pending messages. If we are stopping due
// to a fork() the I/O thread will be restarted after the fork (as
// long as we are not also being destroyed). If we are stopping due
// to the destructor, any remaining messages will be written out
// inside the destructor.
data->flags |= FLAG_IO_THREAD_STOPPED;
data.unlock();
ioCV_.notify_all();
return;
}
++data->ioThreadCounter;
numDiscarded = data->numDiscarded;
data->numDiscarded = 0;
data->currentBufferSize = 0;
}
ioCV_.notify_all();
// Write the log messages now that we have released the lock
try {
performIO(ioQueue, numDiscarded);
} catch (const std::exception& ex) {
onIoError(ex);
}
// clear() empties the vector, but the allocated capacity remains so we can
// just reuse it without having to re-allocate in most cases.
ioQueue->clear();
}
}
void AsyncFileWriter::performIO(
std::vector<std::string>* ioQueue,
size_t numDiscarded) {
......@@ -234,81 +87,4 @@ std::string AsyncFileWriter::getNumDiscardedMsg(size_t numDiscarded) {
" log messages discarded: logging faster than we can write\n");
}
bool AsyncFileWriter::preFork() {
// Stop the I/O thread.
//
// It would perhaps be better to not stop the I/O thread in the parent
// process. However, this leaves us in a slightly tricky situation in the
// child process where data_->ioThread has been initialized and does not
// really point to a valid thread. While we could store it in a union and
// replace it without ever calling its destructor, in practice this still has
// some tricky corner cases to deal with.
// Grab the data lock to ensure no other thread is holding it
// while we fork.
lockedData_ = data_.lock();
// If the I/O thread has been started, stop it now
if (lockedData_->flags & FLAG_IO_THREAD_STARTED) {
stopIoThread(lockedData_, 0);
}
return true;
}
void AsyncFileWriter::postForkParent() {
// Restart the I/O thread
restartThread();
}
void AsyncFileWriter::postForkChild() {
// Clear any messages in the queue. We only want them to be written once,
// and we let the parent process handle writing them.
lockedData_->queues[0].clear();
lockedData_->queues[1].clear();
// Restart the I/O thread
restartThread();
}
void AsyncFileWriter::stopIoThread(
folly::Synchronized<Data, std::mutex>::LockedPtr& data,
uint32_t extraFlags) {
data->flags |= (FLAG_STOP | extraFlags);
messageReady_.notify_one();
ioCV_.wait(data.getUniqueLock(), [&] {
return bool(data->flags & FLAG_IO_THREAD_STOPPED);
});
// Check FLAG_IO_THREAD_JOINED before calling join().
// preFork() and the destructor may both run concurrently in separate
// threads, and only one should try to join the thread.
if ((data->flags & FLAG_IO_THREAD_JOINED) == 0) {
data->ioThread.join();
data->flags |= FLAG_IO_THREAD_JOINED;
}
}
void AsyncFileWriter::restartThread() {
// Move lockedData_ into a local member variable so it will be released
// when we return.
folly::Synchronized<Data, std::mutex>::LockedPtr data =
std::move(lockedData_);
if (!(data->flags & FLAG_IO_THREAD_STARTED)) {
// Do not start the I/O thread if the constructor has not finished yet
return;
}
if (data->flags & FLAG_DESTROYING) {
// Do not restart the I/O thread if we were being destroyed.
// If there are more pending messages that need to be flushed the
// destructor's stopIoThread() call will handle flushing the messages in
// this case.
return;
}
data->flags &= ~(FLAG_STOP | FLAG_IO_THREAD_JOINED | FLAG_IO_THREAD_STOPPED);
data->ioThread = std::thread([this] { ioThread(); });
}
} // namespace folly
......@@ -15,41 +15,19 @@
*/
#pragma once
#include <condition_variable>
#include <mutex>
#include <thread>
#include <folly/logging/AsyncLogWriter.h>
#include <folly/File.h>
#include <folly/Range.h>
#include <folly/Synchronized.h>
#include <folly/logging/LogWriter.h>
#include <vector>
namespace folly {
/**
* A LogWriter implementation that asynchronously writes to a file descriptor.
*
* This class performs the log I/O in a separarate thread.
* An implementation of `folly::AsyncLogWriter` that writes log messages into a
* file.
*
* The advantage of this class over ImmediateFileWriter is that logging I/O can
* never slow down or block your normal program operation. If log messages are
* generated faster than they can be written, messages will be dropped (and an
* indication of how many messages were dropped will be written to the log file
* when we are able to catch up a bit.)
*
* However, one downside is that if your program crashes, not all log messages
* may have been written, so you may lose messages generated immediately before
* the crash.
* See `folly::AsyncLogWriter` for details on asynchronous IO.
*/
class AsyncFileWriter : public LogWriter {
class AsyncFileWriter : public AsyncLogWriter {
public:
/**
* The default maximum buffer size.
*
* The comments for setMaxBufferSize() explain how this parameter is used.
*/
static constexpr size_t kDefaultMaxBufferSize = 1024 * 1024;
/**
* Construct an AsyncFileWriter that appends to the file at the specified
* path.
......@@ -63,35 +41,11 @@ class AsyncFileWriter : public LogWriter {
~AsyncFileWriter();
void writeMessage(folly::StringPiece buffer, uint32_t flags = 0) override;
void writeMessage(std::string&& buffer, uint32_t flags = 0) override;
/**
* Block until the I/O thread has finished writing all messages that
* were already enqueued when flush() was called.
*/
void flush() override;
/**
* Returns true if the output steam is a tty.
*/
bool ttyOutput() const override;
/**
* Set the maximum buffer size for this AsyncFileWriter, in bytes.
*
* This controls the upper bound on how much unwritten data will be buffered
* in memory. If messages are being logged faster than they can be written
* to output file, new messages will be discarded if they would cause the
* amount of buffered data to exceed this limit.
*/
void setMaxBufferSize(size_t size);
/**
* Get the maximum buffer size for this AsyncFileWriter, in bytes.
*/
size_t getMaxBufferSize() const;
/**
* Get the output file.
*/
......@@ -100,79 +54,12 @@ class AsyncFileWriter : public LogWriter {
}
private:
enum Flags : uint32_t {
// FLAG_IO_THREAD_STARTED indicates that the constructor has started the
// I/O thread.
FLAG_IO_THREAD_STARTED = 0x01,
// FLAG_DESTROYING indicates that the destructor is running and destroying
// the I/O thread.
FLAG_DESTROYING = 0x02,
// FLAG_STOP indicates that the I/O thread has been asked to stop.
// This is set either by the destructor or by preFork()
FLAG_STOP = 0x04,
// FLAG_IO_THREAD_STOPPED indicates that the I/O thread is about to return
// and can now be joined. ioCV_ will be signalled when this flag is set.
FLAG_IO_THREAD_STOPPED = 0x08,
// FLAG_IO_THREAD_JOINED indicates that the I/O thread has been joined.
FLAG_IO_THREAD_JOINED = 0x10,
};
/*
* A simple implementation using two queues.
* All writer threads enqueue into one queue while the I/O thread is
* processing the other.
*
* We could potentially also provide an implementation using folly::MPMCQueue
* in the future, which may improve contention under very high write loads.
*/
struct Data {
std::array<std::vector<std::string>, 2> queues;
uint32_t flags{0};
uint64_t ioThreadCounter{0};
size_t maxBufferBytes{kDefaultMaxBufferSize};
size_t currentBufferSize{0};
size_t numDiscarded{0};
std::thread ioThread;
std::vector<std::string>* getCurrentQueue() {
return &queues[ioThreadCounter & 0x1];
}
};
void performIO(std::vector<std::string>* ioQueue, size_t numDiscarded)
override;
void ioThread();
void performIO(std::vector<std::string>* ioQueue, size_t numDiscarded);
void onIoError(const std::exception& ex);
void onIoError(const std::exception& ex) override;
std::string getNumDiscardedMsg(size_t numDiscarded);
bool preFork();
void postForkParent();
void postForkChild();
void stopIoThread(
folly::Synchronized<Data, std::mutex>::LockedPtr& data,
uint32_t extraFlags);
void restartThread();
folly::File file_;
folly::Synchronized<Data, std::mutex> data_;
/**
* messageReady_ is signaled by writer threads whenever they add a new
* message to the current queue.
*/
std::condition_variable messageReady_;
/**
* ioCV_ is signaled by the I/O thread each time it increments
* the ioThreadCounter (once each time around its loop).
*/
std::condition_variable ioCV_;
/**
* lockedData_ exists only to help pass the lock between preFork() and
* postForkParent()/postForkChild(). We potentially could add some new
* low-level methods to Synchronized to allow manually locking and unlocking
* to avoid having to store this object as a member variable.
*/
folly::Synchronized<Data, std::mutex>::LockedPtr lockedData_;
};
} // namespace folly
/*
* Copyright 2017-present Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <folly/logging/AsyncLogWriter.h>
#include <folly/Exception.h>
#include <folly/FileUtil.h>
#include <folly/detail/AtFork.h>
#include <folly/system/ThreadName.h>
using folly::File;
using folly::StringPiece;
namespace folly {
constexpr size_t AsyncLogWriter::kDefaultMaxBufferSize;
AsyncLogWriter::AsyncLogWriter() {
folly::detail::AtFork::registerHandler(
this,
[this] { return preFork(); },
[this] { postForkParent(); },
[this] { postForkChild(); });
// Start the I/O thread after registering the atfork handler.
// preFork() may be invoked in another thread as soon as registerHandler()
// returns. It will check FLAG_IO_THREAD_STARTED to see if the I/O thread is
// running yet.
{
auto data = data_.lock();
data->flags |= FLAG_IO_THREAD_STARTED;
data->ioThread = std::thread([this] { ioThread(); });
}
}
void AsyncLogWriter::cleanup() {
std::vector<std::string>* ioQueue;
size_t numDiscarded;
{
// Stop the I/O thread
auto data = data_.lock();
stopIoThread(data, FLAG_DESTROYING);
// stopIoThread() causes the I/O thread to stop as soon as possible,
// without waiting for all pending messages to be written. Extract any
// remaining messages to write them below.
ioQueue = data->getCurrentQueue();
numDiscarded = data->numDiscarded;
}
// Unregister the atfork handler after stopping the I/O thread.
// preFork(), postForkParent(), and postForkChild() calls can run
// concurrently with the destructor until unregisterHandler() returns.
folly::detail::AtFork::unregisterHandler(this);
// If there are still any pending messages, flush them now.
if (!ioQueue->empty()) {
try {
performIO(ioQueue, numDiscarded);
} catch (const std::exception& ex) {
onIoError(ex);
}
}
}
void AsyncLogWriter::writeMessage(StringPiece buffer, uint32_t flags) {
return writeMessage(buffer.str(), flags);
}
void AsyncLogWriter::writeMessage(std::string&& buffer, uint32_t flags) {
auto data = data_.lock();
if ((data->currentBufferSize >= data->maxBufferBytes) &&
!(flags & NEVER_DISCARD)) {
++data->numDiscarded;
return;
}
data->currentBufferSize += buffer.size();
auto* queue = data->getCurrentQueue();
queue->emplace_back(std::move(buffer));
messageReady_.notify_one();
}
void AsyncLogWriter::flush() {
auto data = data_.lock();
auto start = data->ioThreadCounter;
// Wait until ioThreadCounter increments by at least two.
// Waiting for a single increment is not sufficient, as this happens after
// the I/O thread has swapped the queues, which is before it has actually
// done the I/O.
while (data->ioThreadCounter < start + 2) {
// Enqueue an empty string and wake the I/O thread.
// The empty string ensures that the I/O thread will break out of its wait
// loop and increment the ioThreadCounter, even if there is no other work
// to do.
data->getCurrentQueue()->emplace_back();
messageReady_.notify_one();
// Wait for notification from the I/O thread that it has done work.
ioCV_.wait(data.getUniqueLock());
}
}
void AsyncLogWriter::setMaxBufferSize(size_t size) {
auto data = data_.lock();
data->maxBufferBytes = size;
}
size_t AsyncLogWriter::getMaxBufferSize() const {
auto data = data_.lock();
return data->maxBufferBytes;
}
void AsyncLogWriter::ioThread() {
folly::setThreadName("log_writer");
while (true) {
// With the lock held, grab a pointer to the current queue, then increment
// the ioThreadCounter index so that other threads will write into the
// other queue as we process this one.
std::vector<std::string>* ioQueue;
size_t numDiscarded;
{
auto data = data_.lock();
ioQueue = data->getCurrentQueue();
while (ioQueue->empty() && !(data->flags & FLAG_STOP)) {
// Wait for a message or one of the above flags to be set.
messageReady_.wait(data.getUniqueLock());
}
if (data->flags & FLAG_STOP) {
// We have been asked to stop. We exit immediately in this case
// without writing out any pending messages. If we are stopping due
// to a fork() the I/O thread will be restarted after the fork (as
// long as we are not also being destroyed). If we are stopping due
// to the destructor, any remaining messages will be written out
// inside the destructor.
data->flags |= FLAG_IO_THREAD_STOPPED;
data.unlock();
ioCV_.notify_all();
return;
}
++data->ioThreadCounter;
numDiscarded = data->numDiscarded;
data->numDiscarded = 0;
data->currentBufferSize = 0;
}
ioCV_.notify_all();
// Write the log messages now that we have released the lock
try {
performIO(ioQueue, numDiscarded);
} catch (const std::exception& ex) {
onIoError(ex);
}
// clear() empties the vector, but the allocated capacity remains so we can
// just reuse it without having to re-allocate in most cases.
ioQueue->clear();
}
}
bool AsyncLogWriter::preFork() {
// Stop the I/O thread.
//
// It would perhaps be better to not stop the I/O thread in the parent
// process. However, this leaves us in a slightly tricky situation in the
// child process where data_->ioThread has been initialized and does not
// really point to a valid thread. While we could store it in a union and
// replace it without ever calling its destructor, in practice this still has
// some tricky corner cases to deal with.
// Grab the data lock to ensure no other thread is holding it
// while we fork.
lockedData_ = data_.lock();
// If the I/O thread has been started, stop it now
if (lockedData_->flags & FLAG_IO_THREAD_STARTED) {
stopIoThread(lockedData_, 0);
}
return true;
}
void AsyncLogWriter::postForkParent() {
// Restart the I/O thread
restartThread();
}
void AsyncLogWriter::postForkChild() {
// Clear any messages in the queue. We only want them to be written once,
// and we let the parent process handle writing them.
lockedData_->queues[0].clear();
lockedData_->queues[1].clear();
// Restart the I/O thread
restartThread();
}
void AsyncLogWriter::stopIoThread(
folly::Synchronized<Data, std::mutex>::LockedPtr& data,
uint32_t extraFlags) {
data->flags |= (FLAG_STOP | extraFlags);
messageReady_.notify_one();
ioCV_.wait(data.getUniqueLock(), [&] {
return bool(data->flags & FLAG_IO_THREAD_STOPPED);
});
// Check FLAG_IO_THREAD_JOINED before calling join().
// preFork() and the destructor may both run concurrently in separate
// threads, and only one should try to join the thread.
if ((data->flags & FLAG_IO_THREAD_JOINED) == 0) {
data->ioThread.join();
data->flags |= FLAG_IO_THREAD_JOINED;
}
}
void AsyncLogWriter::restartThread() {
// Move lockedData_ into a local member variable so it will be released
// when we return.
folly::Synchronized<Data, std::mutex>::LockedPtr data =
std::move(lockedData_);
if (!(data->flags & FLAG_IO_THREAD_STARTED)) {
// Do not start the I/O thread if the constructor has not finished yet
return;
}
if (data->flags & FLAG_DESTROYING) {
// Do not restart the I/O thread if we were being destroyed.
// If there are more pending messages that need to be flushed the
// destructor's stopIoThread() call will handle flushing the messages in
// this case.
return;
}
data->flags &= ~(FLAG_STOP | FLAG_IO_THREAD_JOINED | FLAG_IO_THREAD_STOPPED);
data->ioThread = std::thread([this] { ioThread(); });
}
} // namespace folly
/*
* Copyright 2017-present Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include <condition_variable>
#include <mutex>
#include <thread>
#include <folly/File.h>
#include <folly/Range.h>
#include <folly/Synchronized.h>
#include <folly/logging/LogWriter.h>
namespace folly {
/**
* An abstract LogWriter implementation that provides functionaility for
* asynchronous IO operations. Users can subclass this class and provide their
* own IO operation implementation by overriding `performIO` method. This class
* will automatically manage incoming log messages and call the method in
* appropriate time.
*
* This class performs the log I/O in a separarate thread.
*
* The advantage of this class over ImmediateFileWriter is that logging I/O can
* never slow down or block your normal program operation. If log messages are
* generated faster than they can be written, messages will be dropped (and an
* indication of how many messages were dropped will be written to the log file
* when we are able to catch up a bit.)
*
* However, one downside is that if your program crashes, not all log messages
* may have been written, so you may lose messages generated immediately before
* the crash.
*/
class AsyncLogWriter : public LogWriter {
public:
/**
* The default maximum buffer size.
*
* The comments for setMaxBufferSize() explain how this parameter is used.
*/
static constexpr size_t kDefaultMaxBufferSize = 1024 * 1024;
explicit AsyncLogWriter();
void writeMessage(folly::StringPiece buffer, uint32_t flags = 0) override;
void writeMessage(std::string&& buffer, uint32_t flags = 0) override;
/**
* Block until the I/O thread has finished writing all messages that
* were already enqueued when flush() was called.
*/
void flush() override;
/**
* Set the maximum buffer size for this AsyncLogWriter, in bytes.
*
* This controls the upper bound on how much unwritten data will be buffered
* in memory. If messages are being logged faster than they can be written
* to output file, new messages will be discarded if they would cause the
* amount of buffered data to exceed this limit.
*/
void setMaxBufferSize(size_t size);
/**
* Get the maximum buffer size for this AsyncLogWriter, in bytes.
*/
size_t getMaxBufferSize() const;
protected:
/**
* Drain up the log message queue. Subclasses must call this method in their
* destructors to avoid losing log messages at shutdown.
*/
void cleanup();
private:
enum Flags : uint32_t {
// FLAG_IO_THREAD_STARTED indicates that the constructor has started the
// I/O thread.
FLAG_IO_THREAD_STARTED = 0x01,
// FLAG_DESTROYING indicates that the destructor is running and destroying
// the I/O thread.
FLAG_DESTROYING = 0x02,
// FLAG_STOP indicates that the I/O thread has been asked to stop.
// This is set either by the destructor or by preFork()
FLAG_STOP = 0x04,
// FLAG_IO_THREAD_STOPPED indicates that the I/O thread is about to return
// and can now be joined. ioCV_ will be signalled when this flag is set.
FLAG_IO_THREAD_STOPPED = 0x08,
// FLAG_IO_THREAD_JOINED indicates that the I/O thread has been joined.
FLAG_IO_THREAD_JOINED = 0x10,
};
/*
* A simple implementation using two queues.
* All writer threads enqueue into one queue while the I/O thread is
* processing the other.
*
* We could potentially also provide an implementation using folly::MPMCQueue
* in the future, which may improve contention under very high write loads.
*/
struct Data {
std::array<std::vector<std::string>, 2> queues;
uint32_t flags{0};
uint64_t ioThreadCounter{0};
size_t maxBufferBytes{kDefaultMaxBufferSize};
size_t currentBufferSize{0};
size_t numDiscarded{0};
std::thread ioThread;
std::vector<std::string>* getCurrentQueue() {
return &queues[ioThreadCounter & 0x1];
}
};
/**
* Subclasses should override this method to provide IO operations on log
* messages. This method will be called in a separate IO thread.
*/
virtual void performIO(
std::vector<std::string>* logs,
size_t numDiscarded) = 0;
virtual void onIoError(const std::exception& ex) = 0;
void ioThread();
bool preFork();
void postForkParent();
void postForkChild();
void stopIoThread(
folly::Synchronized<Data, std::mutex>::LockedPtr& data,
uint32_t extraFlags);
void restartThread();
folly::Synchronized<Data, std::mutex> data_;
/**
* messageReady_ is signaled by writer threads whenever they add a new
* message to the current queue.
*/
std::condition_variable messageReady_;
/**
* ioCV_ is signaled by the I/O thread each time it increments
* the ioThreadCounter (once each time around its loop).
*/
std::condition_variable ioCV_;
/**
* lockedData_ exists only to help pass the lock between preFork() and
* postForkParent()/postForkChild(). We potentially could add some new
* low-level methods to Synchronized to allow manually locking and unlocking
* to avoid having to store this object as a member variable.
*/
folly::Synchronized<Data, std::mutex>::LockedPtr lockedData_;
}; // namespace folly
} // namespace folly
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