Commit 843b72c0 authored by Dave Watson's avatar Dave Watson

Move thrift/lib/cpp/async to folly.

Summary:
Move the minimum amount of stuff and still have everything compile.  Would like to move TAsyncSocket/ServerSocket/SSL/UDP eventually, but not this round.

thrift async is used very widely now: thrift, proxygen, newer mysql async interface, even trying it out in memcache.  A common complaint is that it doesn't get wide enough notice under thrift/, so let's move it to folly/.  After moving TAsyncSocket, both HHVM and proxygen could avoid a dep on thrift as well.

Changes:
* mv files to folly/io/async
* remove 'T' prefix on classes/filenames
* change namespace to 'folly'
* remove any thrift references.

Tried this before in D470080, this time doesn't attempt to abstract libevent

@override-unit-failures

Test Plan:
fbconfig -r thrift; fbmake dev.
Will iterate on any other contbuild failures

Reviewed By: pgriess@fb.com

FB internal diff: D1195393
parent e22f0a67
...@@ -79,6 +79,14 @@ nobase_follyinclude_HEADERS = \ ...@@ -79,6 +79,14 @@ nobase_follyinclude_HEADERS = \
io/RecordIO.h \ io/RecordIO.h \
io/RecordIO-inl.h \ io/RecordIO-inl.h \
io/TypedIOBuf.h \ io/TypedIOBuf.h \
io/async/AsyncTimeout.h \
io/async/EventBase.h \
io/async/EventFDWrapper.h \
io/async/EventHandler.h \
io/async/EventUtil.h \
io/async/NotificationQueue.h \
io/async/Request.h \
io/async/TimeoutManager.h \
json.h \ json.h \
Lazy.h \ Lazy.h \
Likely.h \ Likely.h \
...@@ -175,6 +183,10 @@ libfolly_la_SOURCES = \ ...@@ -175,6 +183,10 @@ libfolly_la_SOURCES = \
io/IOBuf.cpp \ io/IOBuf.cpp \
io/IOBufQueue.cpp \ io/IOBufQueue.cpp \
io/RecordIO.cpp \ io/RecordIO.cpp \
io/async/AsyncTimeout.cpp \
io/async/EventBase.cpp \
io/async/EventHandler.cpp \
io/async/Request.cpp \
json.cpp \ json.cpp \
detail/MemoryIdler.cpp \ detail/MemoryIdler.cpp \
MemoryMapping.cpp \ MemoryMapping.cpp \
......
...@@ -60,6 +60,8 @@ AC_CHECK_HEADER(double-conversion/double-conversion.h, [], [AC_MSG_ERROR( ...@@ -60,6 +60,8 @@ AC_CHECK_HEADER(double-conversion/double-conversion.h, [], [AC_MSG_ERROR(
AC_CHECK_LIB([double-conversion],[ceil],[],[AC_MSG_ERROR( AC_CHECK_LIB([double-conversion],[ceil],[],[AC_MSG_ERROR(
[Please install double-conversion library])]) [Please install double-conversion library])])
AC_CHECK_LIB([event], [event_set], [], [AC_MSG_ERROR([Unable to find libevent])])
# Checks for typedefs, structures, and compiler characteristics. # Checks for typedefs, structures, and compiler characteristics.
AC_HEADER_STDBOOL AC_HEADER_STDBOOL
AC_C_CONST AC_C_CONST
......
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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/io/async/AsyncTimeout.h"
#include "folly/io/async/EventBase.h"
#include "folly/io/async/EventUtil.h"
#include "folly/io/async/Request.h"
#include <assert.h>
#include <glog/logging.h>
namespace folly {
AsyncTimeout::AsyncTimeout(TimeoutManager* timeoutManager)
: timeoutManager_(timeoutManager) {
event_set(&event_, -1, EV_TIMEOUT, &AsyncTimeout::libeventCallback, this);
event_.ev_base = nullptr;
timeoutManager_->attachTimeoutManager(
this,
TimeoutManager::InternalEnum::NORMAL);
RequestContext::getStaticContext();
}
AsyncTimeout::AsyncTimeout(EventBase* eventBase)
: timeoutManager_(eventBase) {
event_set(&event_, -1, EV_TIMEOUT, &AsyncTimeout::libeventCallback, this);
event_.ev_base = nullptr;
timeoutManager_->attachTimeoutManager(
this,
TimeoutManager::InternalEnum::NORMAL);
RequestContext::getStaticContext();
}
AsyncTimeout::AsyncTimeout(TimeoutManager* timeoutManager,
InternalEnum internal)
: timeoutManager_(timeoutManager) {
event_set(&event_, -1, EV_TIMEOUT, &AsyncTimeout::libeventCallback, this);
event_.ev_base = nullptr;
timeoutManager_->attachTimeoutManager(this, internal);
RequestContext::getStaticContext();
}
AsyncTimeout::AsyncTimeout(EventBase* eventBase, InternalEnum internal)
: timeoutManager_(eventBase) {
event_set(&event_, -1, EV_TIMEOUT, &AsyncTimeout::libeventCallback, this);
event_.ev_base = nullptr;
timeoutManager_->attachTimeoutManager(this, internal);
RequestContext::getStaticContext();
}
AsyncTimeout::AsyncTimeout(): timeoutManager_(nullptr) {
event_set(&event_, -1, EV_TIMEOUT, &AsyncTimeout::libeventCallback, this);
event_.ev_base = nullptr;
RequestContext::getStaticContext();
}
AsyncTimeout::~AsyncTimeout() {
cancelTimeout();
}
bool AsyncTimeout::scheduleTimeout(std::chrono::milliseconds timeout) {
assert(timeoutManager_ != nullptr);
context_ = RequestContext::saveContext();
return timeoutManager_->scheduleTimeout(this, timeout);
}
bool AsyncTimeout::scheduleTimeout(uint32_t milliseconds) {
return scheduleTimeout(std::chrono::milliseconds(milliseconds));
}
void AsyncTimeout::cancelTimeout() {
if (isScheduled()) {
timeoutManager_->cancelTimeout(this);
}
}
bool AsyncTimeout::isScheduled() const {
return EventUtil::isEventRegistered(&event_);
}
void AsyncTimeout::attachTimeoutManager(
TimeoutManager* timeoutManager,
InternalEnum internal) {
// This also implies no timeout is scheduled.
assert(timeoutManager_ == nullptr);
assert(timeoutManager->isInTimeoutManagerThread());
timeoutManager_ = timeoutManager;
timeoutManager_->attachTimeoutManager(this, internal);
}
void AsyncTimeout::attachEventBase(
EventBase* eventBase,
InternalEnum internal) {
attachTimeoutManager(eventBase, internal);
}
void AsyncTimeout::detachTimeoutManager() {
// Only allow the event base to be changed if the timeout is not
// currently installed.
if (isScheduled()) {
// Programmer bug. Abort the program.
LOG(ERROR) << "detachEventBase() called on scheduled timeout; aborting";
abort();
return;
}
if (timeoutManager_) {
timeoutManager_->detachTimeoutManager(this);
timeoutManager_ = nullptr;
}
}
void AsyncTimeout::detachEventBase() {
detachTimeoutManager();
}
void AsyncTimeout::libeventCallback(int fd, short events, void* arg) {
AsyncTimeout* timeout = reinterpret_cast<AsyncTimeout*>(arg);
assert(fd == -1);
assert(events == EV_TIMEOUT);
// double check that ev_flags gets reset when the timeout is not running
assert((timeout->event_.ev_flags & ~EVLIST_INTERNAL) == EVLIST_INIT);
// this can't possibly fire if timeout->eventBase_ is nullptr
(void) timeout->timeoutManager_->bumpHandlingTime();
auto old_ctx =
RequestContext::setContext(timeout->context_);
timeout->timeoutExpired();
RequestContext::setContext(old_ctx);
}
} // folly
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 "folly/io/async/TimeoutManager.h"
#include <boost/noncopyable.hpp>
#include <event.h>
#include <memory>
namespace folly {
class EventBase;
class RequestContext;
class TimeoutManager;
/**
* AsyncTimeout is used to asynchronously wait for a timeout to occur.
*/
class AsyncTimeout : private boost::noncopyable {
public:
typedef TimeoutManager::InternalEnum InternalEnum;
/**
* Create a new AsyncTimeout object, driven by the specified TimeoutManager.
*/
explicit AsyncTimeout(TimeoutManager* timeoutManager);
explicit AsyncTimeout(EventBase* eventBase);
/**
* Create a new internal AsyncTimeout object.
*
* Internal timeouts are like regular timeouts, but will not stop the
* TimeoutManager loop from exiting if the only remaining events are internal
* timeouts.
*
* This is useful for implementing fallback timeouts to abort the
* TimeoutManager loop if the other events have not been processed within a
* specified time period: if the event loop takes too long the timeout will
* fire and can stop the event loop. However, if all other events complete,
* the event loop will exit even though the internal timeout is still
* installed.
*/
AsyncTimeout(TimeoutManager* timeoutManager, InternalEnum internal);
AsyncTimeout(EventBase* eventBase, InternalEnum internal);
/**
* Create a new AsyncTimeout object, not yet assigned to a TimeoutManager.
*
* attachEventBase() must be called prior to scheduling the timeout.
*/
AsyncTimeout();
/**
* AsyncTimeout destructor.
*
* The timeout will be automatically cancelled if it is running.
*/
virtual ~AsyncTimeout();
/**
* timeoutExpired() is invoked when the timeout period has expired.
*/
virtual void timeoutExpired() noexcept = 0;
/**
* Schedule the timeout to fire in the specified number of milliseconds.
*
* After the specified number of milliseconds has elapsed, timeoutExpired()
* will be invoked by the TimeoutManager's main loop.
*
* If the timeout is already running, it will be rescheduled with the
* new timeout value.
*
* @param milliseconds The timeout duration, in milliseconds.
*
* @return Returns true if the timeout was successfully scheduled,
* and false if an error occurred. After an error, the timeout is
* always unscheduled, even if scheduleTimeout() was just
* rescheduling an existing timeout.
*/
bool scheduleTimeout(uint32_t milliseconds);
bool scheduleTimeout(std::chrono::milliseconds timeout);
/**
* Cancel the timeout, if it is running.
*/
void cancelTimeout();
/**
* Returns true if the timeout is currently scheduled.
*/
bool isScheduled() const;
/**
* Attach the timeout to a TimeoutManager.
*
* This may only be called if the timeout is not currently attached to a
* TimeoutManager (either by using the default constructor, or by calling
* detachTimeoutManager()).
*
* This method must be invoked in the TimeoutManager's thread.
*
* The internal parameter specifies if this timeout should be treated as an
* internal event. TimeoutManager::loop() will return when there are no more
* non-internal events remaining.
*/
void attachTimeoutManager(TimeoutManager* timeoutManager,
InternalEnum internal = InternalEnum::NORMAL);
void attachEventBase(EventBase* eventBase,
InternalEnum internal = InternalEnum::NORMAL);
/**
* Detach the timeout from its TimeoutManager.
*
* This may only be called when the timeout is not running.
* Once detached, the timeout may not be scheduled again until it is
* re-attached to a EventBase by calling attachEventBase().
*
* This method must be called from the current TimeoutManager's thread.
*/
void detachTimeoutManager();
void detachEventBase();
/**
* Returns the internal handle to the event
*/
struct event* getEvent() {
return &event_;
}
private:
static void libeventCallback(int fd, short events, void* arg);
struct event event_;
/*
* Store a pointer to the TimeoutManager. We only use this
* for some assert() statements, to make sure that AsyncTimeout is always
* used from the correct thread.
*/
TimeoutManager* timeoutManager_;
// Save the request context for when the timeout fires.
std::shared_ptr<RequestContext> context_;
};
} // folly
This diff is collapsed.
This diff is collapsed.
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.
*/
/**
* Work around the lack of <sys/eventfd.h> on glibc 2.5.1 which we still
* need to support, sigh.
*/
#pragma once
#include <features.h>
// <sys/eventfd.h> doesn't exist on older glibc versions
#if (defined(__GLIBC__) && __GLIBC_PREREQ(2, 9))
#include <sys/eventfd.h>
#else /* !(defined(__GLIBC__) && __GLIBC_PREREQ(2, 9)) */
#include <sys/syscall.h>
#include <unistd.h>
#include <fcntl.h>
// Use existing __NR_eventfd2 if already defined
// Values from the Linux kernel source:
// arch/x86/include/asm/unistd_{32,64}.h
#ifndef __NR_eventfd2
#if defined(__x86_64__)
#define __NR_eventfd2 290
#elif defined(__i386__)
#define __NR_eventfd2 328
#else
#error "Can't define __NR_eventfd2 for your architecture."
#endif
#endif
enum
{
EFD_SEMAPHORE = 1,
#define EFD_SEMAPHORE EFD_SEMAPHORE
EFD_CLOEXEC = 02000000,
#define EFD_CLOEXEC EFD_CLOEXEC
EFD_NONBLOCK = 04000
#define EFD_NONBLOCK EFD_NONBLOCK
};
// http://www.kernel.org/doc/man-pages/online/pages/man2/eventfd.2.html
// Use the eventfd2 system call, as in glibc 2.9+
// (requires kernel 2.6.30+)
#define eventfd(initval, flags) syscall(__NR_eventfd2,(initval),(flags))
#endif /* !(defined(__GLIBC__) && __GLIBC_PREREQ(2, 9)) */
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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/io/async/EventHandler.h"
#include "folly/io/async/EventBase.h"
#include <assert.h>
namespace folly {
EventHandler::EventHandler(EventBase* eventBase, int fd) {
event_set(&event_, fd, 0, &EventHandler::libeventCallback, this);
if (eventBase != nullptr) {
setEventBase(eventBase);
} else {
// Callers must set the EventBase and fd before using this timeout.
// Set event_->ev_base to nullptr to ensure that this happens.
// (otherwise libevent will initialize it to the "default" event_base)
event_.ev_base = nullptr;
eventBase_ = nullptr;
}
}
EventHandler::~EventHandler() {
unregisterHandler();
}
bool EventHandler::registerImpl(uint16_t events, bool internal) {
assert(event_.ev_base != nullptr);
// We have to unregister the event before we can change the event flags
if (isHandlerRegistered()) {
// If the new events are the same are the same as the already registered
// flags, we don't have to do anything. Just return.
if (events == event_.ev_events &&
static_cast<bool>(event_.ev_flags & EVLIST_INTERNAL) == internal) {
return true;
}
event_del(&event_);
}
// Update the event flags
// Unfortunately, event_set() resets the event_base, so we have to remember
// it before hand, then pass it back into event_base_set() afterwards
struct event_base* evb = event_.ev_base;
event_set(&event_, event_.ev_fd, events,
&EventHandler::libeventCallback, this);
event_base_set(evb, &event_);
// Set EVLIST_INTERNAL if this is an internal event
if (internal) {
event_.ev_flags |= EVLIST_INTERNAL;
}
// Add the event.
//
// Although libevent allows events to wait on both I/O and a timeout,
// we intentionally don't allow an EventHandler to also use a timeout.
// Callers must maintain a separate AsyncTimeout object if they want a
// timeout.
//
// Otherwise, it is difficult to handle persistent events properly. (The I/O
// event and timeout may both fire together the same time around the event
// loop. Normally we would want to inform the caller of the I/O event first,
// then the timeout. However, it is difficult to do this properly since the
// I/O callback could delete the EventHandler.) Additionally, if a caller
// uses the same struct event for both I/O and timeout, and they just want to
// reschedule the timeout, libevent currently makes an epoll_ctl() call even
// if the I/O event flags haven't changed. Using a separate event struct is
// therefore slightly more efficient in this case (although it does take up
// more space).
if (event_add(&event_, nullptr) < 0) {
LOG(ERROR) << "EventBase: failed to register event handler for fd "
<< event_.ev_fd << ": " << strerror(errno);
// Call event_del() to make sure the event is completely uninstalled
event_del(&event_);
return false;
}
return true;
}
void EventHandler::unregisterHandler() {
if (isHandlerRegistered()) {
event_del(&event_);
}
}
void EventHandler::attachEventBase(EventBase* eventBase) {
// attachEventBase() may only be called on detached handlers
assert(event_.ev_base == nullptr);
assert(!isHandlerRegistered());
// This must be invoked from the EventBase's thread
assert(eventBase->isInEventBaseThread());
setEventBase(eventBase);
}
void EventHandler::detachEventBase() {
ensureNotRegistered(__func__);
event_.ev_base = nullptr;
}
void EventHandler::changeHandlerFD(int fd) {
ensureNotRegistered(__func__);
// event_set() resets event_base.ev_base, so manually restore it afterwards
struct event_base* evb = event_.ev_base;
event_set(&event_, fd, 0, &EventHandler::libeventCallback, this);
event_.ev_base = evb; // don't use event_base_set(), since evb may be nullptr
}
void EventHandler::initHandler(EventBase* eventBase, int fd) {
ensureNotRegistered(__func__);
event_set(&event_, fd, 0, &EventHandler::libeventCallback, this);
setEventBase(eventBase);
}
void EventHandler::ensureNotRegistered(const char* fn) {
// Neither the EventBase nor file descriptor may be changed while the
// handler is registered. Treat it as a programmer bug and abort the program
// if this requirement is violated.
if (isHandlerRegistered()) {
LOG(ERROR) << fn << " called on registered handler; aborting";
abort();
}
}
void EventHandler::libeventCallback(int fd, short events, void* arg) {
EventHandler* handler = reinterpret_cast<EventHandler*>(arg);
assert(fd == handler->event_.ev_fd);
// this can't possibly fire if handler->eventBase_ is nullptr
(void) handler->eventBase_->bumpHandlingTime();
handler->handlerReady(events);
}
void EventHandler::setEventBase(EventBase* eventBase) {
event_base_set(eventBase->getLibeventBase(), &event_);
eventBase_ = eventBase;
}
bool EventHandler::isPending() {
if (event_.ev_flags & EVLIST_ACTIVE) {
if (event_.ev_res & EV_READ) {
return true;
}
}
return false;
}
} // folly
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 <glog/logging.h>
#include "folly/io/async/EventUtil.h"
#include <boost/noncopyable.hpp>
#include <stddef.h>
namespace folly {
class EventBase;
/**
* The EventHandler class is used to asynchronously wait for events on a file
* descriptor.
*
* Users that wish to wait on I/O events should derive from EventHandler and
* implement the handlerReady() method.
*/
class EventHandler : private boost::noncopyable {
public:
enum EventFlags {
NONE = 0,
READ = EV_READ,
WRITE = EV_WRITE,
READ_WRITE = (READ | WRITE),
PERSIST = EV_PERSIST
};
/**
* Create a new EventHandler object.
*
* @param eventBase The EventBase to use to drive this event handler.
* This may be nullptr, in which case the EventBase must be
* set separately using initHandler() or attachEventBase()
* before the handler can be registered.
* @param fd The file descriptor that this EventHandler will
* monitor. This may be -1, in which case the file
* descriptor must be set separately using initHandler() or
* changeHandlerFD() before the handler can be registered.
*/
explicit EventHandler(EventBase* eventBase = nullptr, int fd = -1);
/**
* EventHandler destructor.
*
* The event will be automatically unregistered if it is still registered.
*/
virtual ~EventHandler();
/**
* handlerReady() is invoked when the handler is ready.
*
* @param events A bitset indicating the events that are ready.
*/
virtual void handlerReady(uint16_t events) noexcept = 0;
/**
* Register the handler.
*
* If the handler is already registered, the registration will be updated
* to wait on the new set of events.
*
* @param events A bitset specifying the events to monitor.
* If the PERSIST bit is set, the handler will remain
* registered even after handlerReady() is called.
*
* @return Returns true if the handler was successfully registered,
* or false if an error occurred. After an error, the handler is
* always unregistered, even if it was already registered prior to
* this call to registerHandler().
*/
bool registerHandler(uint16_t events) {
return registerImpl(events, false);
}
/**
* Unregister the handler, if it is registered.
*/
void unregisterHandler();
/**
* Returns true if the handler is currently registered.
*/
bool isHandlerRegistered() const {
return EventUtil::isEventRegistered(&event_);
}
/**
* Attach the handler to a EventBase.
*
* This may only be called if the handler is not currently attached to a
* EventBase (either by using the default constructor, or by calling
* detachEventBase()).
*
* This method must be invoked in the EventBase's thread.
*/
void attachEventBase(EventBase* eventBase);
/**
* Detach the handler from its EventBase.
*
* This may only be called when the handler is not currently registered.
* Once detached, the handler may not be registered again until it is
* re-attached to a EventBase by calling attachEventBase().
*
* This method must be called from the current EventBase's thread.
*/
void detachEventBase();
/**
* Change the file descriptor that this handler is associated with.
*
* This may only be called when the handler is not currently registered.
*/
void changeHandlerFD(int fd);
/**
* Attach the handler to a EventBase, and change the file descriptor.
*
* This method may only be called if the handler is not currently attached to
* a EventBase. This is primarily intended to be used to initialize
* EventHandler objects created using the default constructor.
*/
void initHandler(EventBase* eventBase, int fd);
/**
* Return the set of events that we're currently registered for.
*/
uint16_t getRegisteredEvents() const {
return (isHandlerRegistered()) ?
event_.ev_events : 0;
}
/**
* Register the handler as an internal event.
*
* This event will not count as an active event for determining if the
* EventBase loop has more events to process. The EventBase loop runs
* only as long as there are active EventHandlers, however "internal" event
* handlers are not counted. Therefore this event handler will not prevent
* EventBase loop from exiting with no more work to do if there are no other
* non-internal event handlers registered.
*
* This is intended to be used only in very rare cases by the internal
* EventBase code. This API is not guaranteed to remain stable or portable
* in the future.
*/
bool registerInternalHandler(uint16_t events) {
return registerImpl(events, true);
}
bool isPending();
private:
bool registerImpl(uint16_t events, bool internal);
void ensureNotRegistered(const char* fn);
void setEventBase(EventBase* eventBase);
static void libeventCallback(int fd, short events, void* arg);
struct event event_;
EventBase* eventBase_;
};
} // folly
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 <event.h> // libevent
namespace folly {
/**
* low-level libevent utility functions
*/
class EventUtil {
public:
static bool isEventRegistered(const struct event* ev) {
// If any of these flags are set, the event is registered.
enum {
EVLIST_REGISTERED = (EVLIST_INSERTED | EVLIST_ACTIVE |
EVLIST_TIMEOUT | EVLIST_SIGNAL)
};
return (ev->ev_flags & EVLIST_REGISTERED);
}
};
} // folly
This diff is collapsed.
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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/io/async/Request.h"
#ifndef NO_LIB_GFLAGS
DEFINE_bool(enable_request_context, true,
"Enable collection of per-request queueing stats for thrift");
#endif
namespace folly {
#ifdef NO_LIB_GFLAGS
bool FLAGS_enable_thrift_request_context = true;
#endif
RequestContext* defaultContext;
}
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 <map>
#include <memory>
#include <glog/logging.h>
#include "folly/ThreadLocal.h"
#include "folly/RWSpinLock.h"
/**
* In many cases this header is included as a
* dependency to libraries which do not need
* command line flags. GFLAGS is a large binary
* and thus we do this so that a library which
* is size sensitive doesn't have to pull in
* GFLAGS if it doesn't want to.
*/
#ifndef NO_LIB_GFLAGS
#include <gflags/gflags.h>
DECLARE_bool(enable_request_context);
#endif
namespace folly {
#ifdef NO_LIB_GFLAGS
extern bool FLAGS_enable_request_context;
#endif
// Some request context that follows an async request through a process
// Everything in the context must be thread safe
class RequestData {
public:
virtual ~RequestData() {}
};
class RequestContext;
// If you do not call create() to create a unique request context,
// this default request context will always be returned, and is never
// copied between threads.
extern RequestContext* defaultContext;
class RequestContext {
public:
// Create a unique requext context for this request.
// It will be passed between queues / threads (where implemented),
// so it should be valid for the lifetime of the request.
static bool create() {
if(!FLAGS_enable_request_context) {
return false;
}
bool prev = getStaticContext().get() != nullptr;
getStaticContext().reset(new std::shared_ptr<RequestContext>(
std::make_shared<RequestContext>()));
return prev;
}
// Get the current context.
static RequestContext* get() {
if (!FLAGS_enable_request_context ||
getStaticContext().get() == nullptr) {
if (defaultContext == nullptr) {
defaultContext = new RequestContext;
}
return defaultContext;
}
return getStaticContext().get()->get();
}
// The following API may be used to set per-request data in a thread-safe way.
// This access is still performance sensitive, so please ask if you need help
// profiling any use of these functions.
void setContextData(
const std::string& val, std::unique_ptr<RequestData> data) {
if (!FLAGS_enable_request_context) {
return;
}
folly::RWSpinLock::WriteHolder guard(lock);
if (data_.find(val) != data_.end()) {
LOG_FIRST_N(WARNING, 1) <<
"Called RequestContext::setContextData with data already set";
data_[val] = nullptr;
} else {
data_[val] = std::move(data);
}
}
bool hasContextData(const std::string& val) {
folly::RWSpinLock::ReadHolder guard(lock);
return data_.find(val) != data_.end();
}
RequestData* getContextData(const std::string& val) {
folly::RWSpinLock::ReadHolder guard(lock);
auto r = data_.find(val);
if (r == data_.end()) {
return nullptr;
} else {
return r->second.get();
}
}
void clearContextData(const std::string& val) {
folly::RWSpinLock::WriteHolder guard(lock);
data_.erase(val);
}
// The following API is used to pass the context through queues / threads.
// saveContext is called to geta shared_ptr to the context, and
// setContext is used to reset it on the other side of the queue.
//
// A shared_ptr is used, because many request may fan out across
// multiple threads, or do post-send processing, etc.
static std::shared_ptr<RequestContext>
setContext(std::shared_ptr<RequestContext> ctx) {
if (FLAGS_enable_request_context) {
std::shared_ptr<RequestContext> old_ctx;
if (getStaticContext().get()) {
old_ctx = *getStaticContext().get();
}
if (ctx == nullptr) {
getStaticContext().reset(nullptr);
} else {
getStaticContext().reset(new std::shared_ptr<RequestContext>(ctx));
}
return old_ctx;
}
return std::shared_ptr<RequestContext>();
}
static std::shared_ptr<RequestContext> saveContext() {
if (!FLAGS_enable_request_context) {
return std::shared_ptr<RequestContext>();
}
if (getStaticContext().get() == nullptr) {
return std::shared_ptr<RequestContext>();
} else {
return *getStaticContext().get();
}
}
// Used to solve static destruction ordering issue. Any static object
// that uses RequestContext must call this function in its constructor.
//
// See below link for more details.
// http://stackoverflow.com/questions/335369/
// finding-c-static-initialization-order-problems#335746
static folly::ThreadLocalPtr<std::shared_ptr<RequestContext>>&
getStaticContext() {
static folly::ThreadLocalPtr<std::shared_ptr<RequestContext> > context;
return context;
}
private:
folly::RWSpinLock lock;
std::map<std::string, std::unique_ptr<RequestData>> data_;
};
/**
* Set the request context for a specific scope. For example,
* if you ran a part of a request in another thread you could
* use RequestContextGuard to copy apply the request context
* inside the other therad.
*/
class RequestContextGuard {
public:
explicit RequestContextGuard(std::shared_ptr<RequestContext> ctx) {
oldctx_ = RequestContext::setContext(std::move(ctx));
}
~RequestContextGuard() {
RequestContext::setContext(std::move(oldctx_));
}
private:
std::shared_ptr<RequestContext> oldctx_;
};
}
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 <chrono>
#include <stdint.h>
namespace folly {
class AsyncTimeout;
/**
* Base interface to be implemented by all classes expecting to manage
* timeouts. AsyncTimeout will use implementations of this interface
* to schedule/cancel timeouts.
*/
class TimeoutManager {
public:
enum class InternalEnum {
INTERNAL,
NORMAL
};
virtual ~TimeoutManager() {}
/**
* Attaches/detaches TimeoutManager to AsyncTimeout
*/
virtual void attachTimeoutManager(AsyncTimeout* obj,
InternalEnum internal) = 0;
virtual void detachTimeoutManager(AsyncTimeout* obj) = 0;
/**
* Schedules AsyncTimeout to fire after `timeout` milliseconds
*/
virtual bool scheduleTimeout(AsyncTimeout* obj,
std::chrono::milliseconds timeout) = 0;
/**
* Cancels the AsyncTimeout, if scheduled
*/
virtual void cancelTimeout(AsyncTimeout* obj) = 0;
/**
* This is used to mark the beginning of a new loop cycle by the
* first handler fired within that cycle.
*/
virtual bool bumpHandlingTime() = 0;
/**
* Helper method to know whether we are running in the timeout manager
* thread
*/
virtual bool isInTimeoutManagerThread() = 0;
};
} // 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