Merge changes from topic "fdevent_refactor"

* changes:
  adb: fdevent: extract Add/Del/SetTimeout.
  adb: fdevent: extract Create/Destroy from fdevent_context_poll.
  adb: fdevent: move TerminateLoop to fdevent_context.
  adb: fdevent: move CheckMainThread logic to fdevent_context.
  adb: fdevent: delete FDE_CREATED.
  adb: fdevent: move run queue to fdevent_context.
  adb: implement fdevent_reset by constructing a new context.
  adb: move fdevent implementation out to a separate file.
  adb: de-globalize fdevent context.
  adb: move fdevent to its own folder.
  adb: add superfluous include.
This commit is contained in:
Josh Gao 2019-07-09 20:02:29 +00:00 committed by Gerrit Code Review
commit 88d23135d2
19 changed files with 804 additions and 659 deletions

View file

@ -124,7 +124,8 @@ libadb_srcs = [
"adb_trace.cpp",
"adb_unique_fd.cpp",
"adb_utils.cpp",
"fdevent.cpp",
"fdevent/fdevent.cpp",
"fdevent/fdevent_poll.cpp",
"services.cpp",
"sockets.cpp",
"socket_spec.cpp",
@ -144,7 +145,7 @@ libadb_test_srcs = [
"adb_io_test.cpp",
"adb_listeners_test.cpp",
"adb_utils_test.cpp",
"fdevent_test.cpp",
"fdevent/fdevent_test.cpp",
"socket_spec_test.cpp",
"socket_test.cpp",
"sysdeps_test.cpp",

View file

@ -26,7 +26,7 @@
#include <android-base/macros.h>
#include "adb_trace.h"
#include "fdevent.h"
#include "fdevent/fdevent.h"
#include "socket.h"
#include "types.h"
#include "usb.h"

View file

@ -21,7 +21,7 @@
#include <android-base/stringprintf.h>
#include <android-base/strings.h>
#include "fdevent.h"
#include "fdevent/fdevent.h"
#include "sysdeps.h"
#include "transport.h"

View file

@ -31,7 +31,7 @@
#include "adb_mdns.h"
#include "adb_trace.h"
#include "fdevent.h"
#include "fdevent/fdevent.h"
#include "sysdeps.h"
static DNSServiceRef service_ref;

View file

@ -18,7 +18,8 @@
#include "adb.h"
#include "adb_auth.h"
#include "fdevent.h"
#include "adb_io.h"
#include "fdevent/fdevent.h"
#include "sysdeps.h"
#include "transport.h"

View file

@ -33,7 +33,6 @@
#include "adb.h"
#include "adb_io.h"
#include "adb_utils.h"
#include "fdevent.h"
/* TODO:
** - sync with vsync to avoid tearing

View file

@ -1,553 +0,0 @@
/* http://frotznet.googlecode.com/svn/trunk/utils/fdevent.c
**
** Copyright 2006, Brian Swetland <swetland@frotz.net>
**
** 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.
*/
#define TRACE_TAG FDEVENT
#include "sysdeps.h"
#include "fdevent.h"
#include <fcntl.h>
#include <inttypes.h>
#include <stdint.h>
#include <stdlib.h>
#include <string.h>
#include <unistd.h>
#include <atomic>
#include <deque>
#include <functional>
#include <list>
#include <mutex>
#include <optional>
#include <unordered_map>
#include <utility>
#include <variant>
#include <vector>
#include <android-base/chrono_utils.h>
#include <android-base/file.h>
#include <android-base/logging.h>
#include <android-base/stringprintf.h>
#include <android-base/thread_annotations.h>
#include <android-base/threads.h>
#include "adb_io.h"
#include "adb_trace.h"
#include "adb_unique_fd.h"
#include "adb_utils.h"
#include "sysdeps/chrono.h"
#define FDE_EVENTMASK 0x00ff
#define FDE_STATEMASK 0xff00
#define FDE_ACTIVE 0x0100
#define FDE_PENDING 0x0200
#define FDE_CREATED 0x0400
struct PollNode {
fdevent* fde;
adb_pollfd pollfd;
explicit PollNode(fdevent* fde) : fde(fde) {
memset(&pollfd, 0, sizeof(pollfd));
pollfd.fd = fde->fd.get();
#if defined(__linux__)
// Always enable POLLRDHUP, so the host server can take action when some clients disconnect.
// Then we can avoid leaving many sockets in CLOSE_WAIT state. See http://b/23314034.
pollfd.events = POLLRDHUP;
#endif
}
};
// All operations to fdevent should happen only in the main thread.
// That's why we don't need a lock for fdevent.
static auto& g_poll_node_map = *new std::unordered_map<int, PollNode>();
static auto& g_pending_list = *new std::list<fdevent*>();
static std::atomic<bool> terminate_loop(false);
static bool main_thread_valid;
static uint64_t main_thread_id;
static uint64_t fdevent_id;
static bool run_needs_flush = false;
static auto& run_queue_notify_fd = *new unique_fd();
static auto& run_queue_mutex = *new std::mutex();
static auto& run_queue GUARDED_BY(run_queue_mutex) = *new std::deque<std::function<void()>>();
void check_main_thread() {
if (main_thread_valid) {
CHECK_EQ(main_thread_id, android::base::GetThreadId());
}
}
void set_main_thread() {
main_thread_valid = true;
main_thread_id = android::base::GetThreadId();
}
static std::string dump_fde(const fdevent* fde) {
std::string state;
if (fde->state & FDE_ACTIVE) {
state += "A";
}
if (fde->state & FDE_PENDING) {
state += "P";
}
if (fde->state & FDE_CREATED) {
state += "C";
}
if (fde->state & FDE_READ) {
state += "R";
}
if (fde->state & FDE_WRITE) {
state += "W";
}
if (fde->state & FDE_ERROR) {
state += "E";
}
return android::base::StringPrintf("(fdevent %" PRIu64 ": fd %d %s)", fde->id, fde->fd.get(),
state.c_str());
}
template <typename F>
static fdevent* fdevent_create_impl(int fd, F func, void* arg) {
check_main_thread();
CHECK_GE(fd, 0);
fdevent* fde = new fdevent();
fde->id = fdevent_id++;
fde->state = FDE_ACTIVE;
fde->fd.reset(fd);
fde->func = func;
fde->arg = arg;
if (!set_file_block_mode(fd, false)) {
// Here is not proper to handle the error. If it fails here, some error is
// likely to be detected by poll(), then we can let the callback function
// to handle it.
LOG(ERROR) << "failed to set non-blocking mode for fd " << fd;
}
auto pair = g_poll_node_map.emplace(fde->fd.get(), PollNode(fde));
CHECK(pair.second) << "install existing fd " << fd;
fde->state |= FDE_CREATED;
return fde;
}
fdevent* fdevent_create(int fd, fd_func func, void* arg) {
return fdevent_create_impl(fd, func, arg);
}
fdevent* fdevent_create(int fd, fd_func2 func, void* arg) {
return fdevent_create_impl(fd, func, arg);
}
unique_fd fdevent_release(fdevent* fde) {
check_main_thread();
if (!fde) {
return {};
}
if (!(fde->state & FDE_CREATED)) {
LOG(FATAL) << "destroying fde not created by fdevent_create(): " << dump_fde(fde);
}
unique_fd result = std::move(fde->fd);
if (fde->state & FDE_ACTIVE) {
g_poll_node_map.erase(result.get());
if (fde->state & FDE_PENDING) {
g_pending_list.remove(fde);
}
fde->state = 0;
fde->events = 0;
}
delete fde;
return result;
}
void fdevent_destroy(fdevent* fde) {
// Release, and then let unique_fd's destructor cleanup.
fdevent_release(fde);
}
static void fdevent_update(fdevent* fde, unsigned events) {
auto it = g_poll_node_map.find(fde->fd.get());
CHECK(it != g_poll_node_map.end());
PollNode& node = it->second;
if (events & FDE_READ) {
node.pollfd.events |= POLLIN;
} else {
node.pollfd.events &= ~POLLIN;
}
if (events & FDE_WRITE) {
node.pollfd.events |= POLLOUT;
} else {
node.pollfd.events &= ~POLLOUT;
}
fde->state = (fde->state & FDE_STATEMASK) | events;
}
void fdevent_set(fdevent* fde, unsigned events) {
check_main_thread();
events &= FDE_EVENTMASK;
if ((fde->state & FDE_EVENTMASK) == events) {
return;
}
CHECK(fde->state & FDE_ACTIVE);
fdevent_update(fde, events);
D("fdevent_set: %s, events = %u", dump_fde(fde).c_str(), events);
if (fde->state & FDE_PENDING) {
// If we are pending, make sure we don't signal an event that is no longer wanted.
fde->events &= events;
if (fde->events == 0) {
g_pending_list.remove(fde);
fde->state &= ~FDE_PENDING;
}
}
}
void fdevent_add(fdevent* fde, unsigned events) {
check_main_thread();
CHECK(!(events & FDE_TIMEOUT));
fdevent_set(fde, (fde->state & FDE_EVENTMASK) | events);
}
void fdevent_del(fdevent* fde, unsigned events) {
check_main_thread();
CHECK(!(events & FDE_TIMEOUT));
fdevent_set(fde, (fde->state & FDE_EVENTMASK) & ~events);
}
void fdevent_set_timeout(fdevent* fde, std::optional<std::chrono::milliseconds> timeout) {
check_main_thread();
fde->timeout = timeout;
fde->last_active = std::chrono::steady_clock::now();
}
static std::string dump_pollfds(const std::vector<adb_pollfd>& pollfds) {
std::string result;
for (const auto& pollfd : pollfds) {
std::string op;
if (pollfd.events & POLLIN) {
op += "R";
}
if (pollfd.events & POLLOUT) {
op += "W";
}
android::base::StringAppendF(&result, " %d(%s)", pollfd.fd, op.c_str());
}
return result;
}
static std::optional<std::chrono::milliseconds> calculate_timeout() {
std::optional<std::chrono::milliseconds> result = std::nullopt;
auto now = std::chrono::steady_clock::now();
check_main_thread();
for (const auto& [fd, pollnode] : g_poll_node_map) {
UNUSED(fd);
auto timeout_opt = pollnode.fde->timeout;
if (timeout_opt) {
auto deadline = pollnode.fde->last_active + *timeout_opt;
auto time_left = std::chrono::duration_cast<std::chrono::milliseconds>(deadline - now);
if (time_left < std::chrono::milliseconds::zero()) {
time_left = std::chrono::milliseconds::zero();
}
if (!result) {
result = time_left;
} else {
result = std::min(*result, time_left);
}
}
}
return result;
}
static void fdevent_process() {
std::vector<adb_pollfd> pollfds;
for (const auto& pair : g_poll_node_map) {
pollfds.push_back(pair.second.pollfd);
}
CHECK_GT(pollfds.size(), 0u);
D("poll(), pollfds = %s", dump_pollfds(pollfds).c_str());
auto timeout = calculate_timeout();
int timeout_ms;
if (!timeout) {
timeout_ms = -1;
} else {
timeout_ms = timeout->count();
}
int ret = adb_poll(&pollfds[0], pollfds.size(), timeout_ms);
if (ret == -1) {
PLOG(ERROR) << "poll(), ret = " << ret;
return;
}
auto post_poll = std::chrono::steady_clock::now();
for (const auto& pollfd : pollfds) {
if (pollfd.revents != 0) {
D("for fd %d, revents = %x", pollfd.fd, pollfd.revents);
}
unsigned events = 0;
if (pollfd.revents & POLLIN) {
events |= FDE_READ;
}
if (pollfd.revents & POLLOUT) {
events |= FDE_WRITE;
}
if (pollfd.revents & (POLLERR | POLLHUP | POLLNVAL)) {
// We fake a read, as the rest of the code assumes that errors will
// be detected at that point.
events |= FDE_READ | FDE_ERROR;
}
#if defined(__linux__)
if (pollfd.revents & POLLRDHUP) {
events |= FDE_READ | FDE_ERROR;
}
#endif
auto it = g_poll_node_map.find(pollfd.fd);
CHECK(it != g_poll_node_map.end());
fdevent* fde = it->second.fde;
if (events == 0) {
// Check for timeout.
if (fde->timeout) {
auto deadline = fde->last_active + *fde->timeout;
if (deadline < post_poll) {
events |= FDE_TIMEOUT;
}
}
}
if (events != 0) {
CHECK_EQ(fde->fd.get(), pollfd.fd);
fde->events |= events;
fde->last_active = post_poll;
D("%s got events %x", dump_fde(fde).c_str(), events);
fde->state |= FDE_PENDING;
g_pending_list.push_back(fde);
}
}
}
template <class T>
struct always_false : std::false_type {};
static void fdevent_call_fdfunc(fdevent* fde) {
unsigned events = fde->events;
fde->events = 0;
CHECK(fde->state & FDE_PENDING);
fde->state &= (~FDE_PENDING);
D("fdevent_call_fdfunc %s", dump_fde(fde).c_str());
std::visit(
[&](auto&& f) {
using F = std::decay_t<decltype(f)>;
if constexpr (std::is_same_v<fd_func, F>) {
f(fde->fd.get(), events, fde->arg);
} else if constexpr (std::is_same_v<fd_func2, F>) {
f(fde, events, fde->arg);
} else {
static_assert(always_false<F>::value, "non-exhaustive visitor");
}
},
fde->func);
}
static void fdevent_run_flush() EXCLUDES(run_queue_mutex) {
// We need to be careful around reentrancy here, since a function we call can queue up another
// function.
while (true) {
std::function<void()> fn;
{
std::lock_guard<std::mutex> lock(run_queue_mutex);
if (run_queue.empty()) {
break;
}
fn = run_queue.front();
run_queue.pop_front();
}
fn();
}
}
static void fdevent_run_func(int fd, unsigned ev, void* /* userdata */) {
CHECK_GE(fd, 0);
CHECK(ev & FDE_READ);
char buf[1024];
// Empty the fd.
if (adb_read(fd, buf, sizeof(buf)) == -1) {
PLOG(FATAL) << "failed to empty run queue notify fd";
}
// Mark that we need to flush, and then run it at the end of fdevent_loop.
run_needs_flush = true;
}
static void fdevent_run_setup() {
{
std::lock_guard<std::mutex> lock(run_queue_mutex);
CHECK(run_queue_notify_fd.get() == -1);
int s[2];
if (adb_socketpair(s) != 0) {
PLOG(FATAL) << "failed to create run queue notify socketpair";
}
if (!set_file_block_mode(s[0], false) || !set_file_block_mode(s[1], false)) {
PLOG(FATAL) << "failed to make run queue notify socket nonblocking";
}
run_queue_notify_fd.reset(s[0]);
fdevent* fde = fdevent_create(s[1], fdevent_run_func, nullptr);
CHECK(fde != nullptr);
fdevent_add(fde, FDE_READ);
}
fdevent_run_flush();
}
void fdevent_run_on_main_thread(std::function<void()> fn) {
std::lock_guard<std::mutex> lock(run_queue_mutex);
run_queue.push_back(std::move(fn));
// run_queue_notify_fd could still be -1 if we're called before fdevent has finished setting up.
// In that case, rely on the setup code to flush the queue without a notification being needed.
if (run_queue_notify_fd != -1) {
int rc = adb_write(run_queue_notify_fd.get(), "", 1);
// It's possible that we get EAGAIN here, if lots of notifications came in while handling.
if (rc == 0) {
PLOG(FATAL) << "run queue notify fd was closed?";
} else if (rc == -1 && errno != EAGAIN) {
PLOG(FATAL) << "failed to write to run queue notify fd";
}
}
}
static void fdevent_check_spin(uint64_t cycle) {
// Check to see if we're spinning because we forgot about an fdevent
// by keeping track of how long fdevents have been continuously pending.
struct SpinCheck {
fdevent* fde;
android::base::boot_clock::time_point timestamp;
uint64_t cycle;
};
static auto& g_continuously_pending = *new std::unordered_map<uint64_t, SpinCheck>();
static auto last_cycle = android::base::boot_clock::now();
auto now = android::base::boot_clock::now();
if (now - last_cycle > 10ms) {
// We're not spinning.
g_continuously_pending.clear();
last_cycle = now;
return;
}
last_cycle = now;
for (auto* fde : g_pending_list) {
auto it = g_continuously_pending.find(fde->id);
if (it == g_continuously_pending.end()) {
g_continuously_pending[fde->id] =
SpinCheck{.fde = fde, .timestamp = now, .cycle = cycle};
} else {
it->second.cycle = cycle;
}
}
for (auto it = g_continuously_pending.begin(); it != g_continuously_pending.end();) {
if (it->second.cycle != cycle) {
it = g_continuously_pending.erase(it);
} else {
// Use an absurdly long window, since all we really care about is
// getting a bugreport eventually.
if (now - it->second.timestamp > 300s) {
LOG(FATAL_WITHOUT_ABORT)
<< "detected spin in fdevent: " << dump_fde(it->second.fde);
#if defined(__linux__)
int fd = it->second.fde->fd.get();
std::string fd_path = android::base::StringPrintf("/proc/self/fd/%d", fd);
std::string path;
if (!android::base::Readlink(fd_path, &path)) {
PLOG(FATAL_WITHOUT_ABORT) << "readlink of fd " << fd << " failed";
}
LOG(FATAL_WITHOUT_ABORT) << "fd " << fd << " = " << path;
#endif
abort();
}
++it;
}
}
}
void fdevent_loop() {
set_main_thread();
fdevent_run_setup();
uint64_t cycle = 0;
while (true) {
if (terminate_loop) {
return;
}
D("--- --- waiting for events");
fdevent_process();
fdevent_check_spin(cycle++);
while (!g_pending_list.empty()) {
fdevent* fde = g_pending_list.front();
g_pending_list.pop_front();
fdevent_call_fdfunc(fde);
}
if (run_needs_flush) {
fdevent_run_flush();
run_needs_flush = false;
}
}
}
void fdevent_terminate_loop() {
terminate_loop = true;
}
size_t fdevent_installed_count() {
return g_poll_node_map.size();
}
void fdevent_reset() {
g_poll_node_map.clear();
g_pending_list.clear();
std::lock_guard<std::mutex> lock(run_queue_mutex);
run_queue_notify_fd.reset();
run_queue.clear();
main_thread_valid = false;
terminate_loop = false;
}

View file

@ -1,90 +0,0 @@
/*
* Copyright (C) 2006 The Android Open Source Project
*
* 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.
*/
#ifndef __FDEVENT_H
#define __FDEVENT_H
#include <stddef.h>
#include <stdint.h>
#include <chrono>
#include <functional>
#include <optional>
#include <variant>
#include "adb_unique_fd.h"
// Events that may be observed
#define FDE_READ 0x0001
#define FDE_WRITE 0x0002
#define FDE_ERROR 0x0004
#define FDE_TIMEOUT 0x0008
typedef void (*fd_func)(int fd, unsigned events, void *userdata);
typedef void (*fd_func2)(struct fdevent* fde, unsigned events, void* userdata);
struct fdevent {
uint64_t id;
unique_fd fd;
int force_eof = 0;
uint16_t state = 0;
uint16_t events = 0;
std::optional<std::chrono::milliseconds> timeout;
std::chrono::steady_clock::time_point last_active;
std::variant<fd_func, fd_func2> func;
void* arg = nullptr;
};
// Allocate and initialize a new fdevent object
// TODO: Switch these to unique_fd.
fdevent *fdevent_create(int fd, fd_func func, void *arg);
fdevent* fdevent_create(int fd, fd_func2 func, void* arg);
// Deallocate an fdevent object that was created by fdevent_create.
void fdevent_destroy(fdevent *fde);
// fdevent_destroy, except releasing the file descriptor previously owned by the fdevent.
unique_fd fdevent_release(fdevent* fde);
// Change which events should cause notifications
void fdevent_set(fdevent *fde, unsigned events);
void fdevent_add(fdevent *fde, unsigned events);
void fdevent_del(fdevent *fde, unsigned events);
// Set a timeout on an fdevent.
// If no events are triggered by the timeout, an FDE_TIMEOUT will be generated.
// Note timeouts are not defused automatically; if a timeout is set on an fdevent, it will
// trigger repeatedly every |timeout| ms.
void fdevent_set_timeout(fdevent* fde, std::optional<std::chrono::milliseconds> timeout);
// Loop forever, handling events.
void fdevent_loop();
void check_main_thread();
// Queue an operation to run on the main thread.
void fdevent_run_on_main_thread(std::function<void()> fn);
// The following functions are used only for tests.
void fdevent_terminate_loop();
size_t fdevent_installed_count();
void fdevent_reset();
void set_main_thread();
#endif

201
adb/fdevent/fdevent.cpp Normal file
View file

@ -0,0 +1,201 @@
/*
* Copyright 2006, Brian Swetland <swetland@frotz.net>
*
* 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.
*/
#define TRACE_TAG FDEVENT
#include "sysdeps.h"
#include <inttypes.h>
#include <android-base/logging.h>
#include <android-base/stringprintf.h>
#include <android-base/threads.h>
#include "adb_utils.h"
#include "fdevent.h"
#include "fdevent_poll.h"
std::string dump_fde(const fdevent* fde) {
std::string state;
if (fde->state & FDE_ACTIVE) {
state += "A";
}
if (fde->state & FDE_PENDING) {
state += "P";
}
if (fde->state & FDE_READ) {
state += "R";
}
if (fde->state & FDE_WRITE) {
state += "W";
}
if (fde->state & FDE_ERROR) {
state += "E";
}
return android::base::StringPrintf("(fdevent %" PRIu64 ": fd %d %s)", fde->id, fde->fd.get(),
state.c_str());
}
fdevent* fdevent_context::Create(unique_fd fd, std::variant<fd_func, fd_func2> func, void* arg) {
CheckMainThread();
CHECK_GE(fd.get(), 0);
fdevent* fde = new fdevent();
fde->id = fdevent_id_++;
fde->state = FDE_ACTIVE;
fde->fd = std::move(fd);
fde->func = func;
fde->arg = arg;
if (!set_file_block_mode(fde->fd, false)) {
// Here is not proper to handle the error. If it fails here, some error is
// likely to be detected by poll(), then we can let the callback function
// to handle it.
LOG(ERROR) << "failed to set non-blocking mode for fd " << fde->fd.get();
}
this->Register(fde);
return fde;
}
unique_fd fdevent_context::Destroy(fdevent* fde) {
CheckMainThread();
if (!fde) {
return {};
}
this->Unregister(fde);
unique_fd result = std::move(fde->fd);
delete fde;
return result;
}
void fdevent_context::Add(fdevent* fde, unsigned events) {
Set(fde, (fde->state & FDE_EVENTMASK) | events);
}
void fdevent_context::Del(fdevent* fde, unsigned events) {
CHECK(!(events & FDE_TIMEOUT));
Set(fde, (fde->state & FDE_EVENTMASK) & ~events);
}
void fdevent_context::SetTimeout(fdevent* fde, std::optional<std::chrono::milliseconds> timeout) {
CheckMainThread();
fde->timeout = timeout;
fde->last_active = std::chrono::steady_clock::now();
}
void fdevent_context::CheckMainThread() {
if (main_thread_id_) {
CHECK_EQ(*main_thread_id_, android::base::GetThreadId());
}
}
void fdevent_context::Run(std::function<void()> fn) {
{
std::lock_guard<std::mutex> lock(run_queue_mutex_);
run_queue_.push_back(std::move(fn));
}
Interrupt();
}
void fdevent_context::TerminateLoop() {
terminate_loop_ = true;
Interrupt();
}
void fdevent_context::FlushRunQueue() {
// We need to be careful around reentrancy here, since a function we call can queue up another
// function.
while (true) {
std::function<void()> fn;
{
std::lock_guard<std::mutex> lock(this->run_queue_mutex_);
if (this->run_queue_.empty()) {
break;
}
fn = this->run_queue_.front();
this->run_queue_.pop_front();
}
fn();
}
}
static auto& g_ambient_fdevent_context =
*new std::unique_ptr<fdevent_context>(new fdevent_context_poll());
static fdevent_context* fdevent_get_ambient() {
return g_ambient_fdevent_context.get();
}
fdevent* fdevent_create(int fd, fd_func func, void* arg) {
unique_fd ufd(fd);
return fdevent_get_ambient()->Create(std::move(ufd), func, arg);
}
fdevent* fdevent_create(int fd, fd_func2 func, void* arg) {
unique_fd ufd(fd);
return fdevent_get_ambient()->Create(std::move(ufd), func, arg);
}
unique_fd fdevent_release(fdevent* fde) {
return fdevent_get_ambient()->Destroy(fde);
}
void fdevent_destroy(fdevent* fde) {
fdevent_get_ambient()->Destroy(fde);
}
void fdevent_set(fdevent* fde, unsigned events) {
fdevent_get_ambient()->Set(fde, events);
}
void fdevent_add(fdevent* fde, unsigned events) {
fdevent_get_ambient()->Add(fde, events);
}
void fdevent_del(fdevent* fde, unsigned events) {
fdevent_get_ambient()->Del(fde, events);
}
void fdevent_set_timeout(fdevent* fde, std::optional<std::chrono::milliseconds> timeout) {
fdevent_get_ambient()->SetTimeout(fde, timeout);
}
void fdevent_run_on_main_thread(std::function<void()> fn) {
fdevent_get_ambient()->Run(std::move(fn));
}
void fdevent_loop() {
fdevent_get_ambient()->Loop();
}
void check_main_thread() {
fdevent_get_ambient()->CheckMainThread();
}
void fdevent_terminate_loop() {
fdevent_get_ambient()->TerminateLoop();
}
size_t fdevent_installed_count() {
return fdevent_get_ambient()->InstalledCount();
}
void fdevent_reset() {
g_ambient_fdevent_context.reset(new fdevent_context_poll());
}

151
adb/fdevent/fdevent.h Normal file
View file

@ -0,0 +1,151 @@
/*
* Copyright (C) 2006 The Android Open Source Project
*
* 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.
*/
#ifndef __FDEVENT_H
#define __FDEVENT_H
#include <stddef.h>
#include <stdint.h>
#include <chrono>
#include <deque>
#include <functional>
#include <mutex>
#include <optional>
#include <variant>
#include <android-base/thread_annotations.h>
#include "adb_unique_fd.h"
// Events that may be observed
#define FDE_READ 0x0001
#define FDE_WRITE 0x0002
#define FDE_ERROR 0x0004
#define FDE_TIMEOUT 0x0008
// Internal states.
#define FDE_EVENTMASK 0x00ff
#define FDE_STATEMASK 0xff00
#define FDE_ACTIVE 0x0100
#define FDE_PENDING 0x0200
typedef void (*fd_func)(int fd, unsigned events, void *userdata);
typedef void (*fd_func2)(struct fdevent* fde, unsigned events, void* userdata);
struct fdevent;
std::string dump_fde(const fdevent* fde);
struct fdevent_context {
public:
virtual ~fdevent_context() = default;
// Allocate and initialize a new fdevent object.
fdevent* Create(unique_fd fd, std::variant<fd_func, fd_func2> func, void* arg);
// Deallocate an fdevent object, returning the file descriptor that was owned by it.
unique_fd Destroy(fdevent* fde);
protected:
// Register an fdevent that is being created by Create with the fdevent_context.
virtual void Register(fdevent* fde) = 0;
// Unregister an fdevent that is being destroyed by Destroy with the fdevent_context.
virtual void Unregister(fdevent* fde) = 0;
public:
// Change which events should cause notifications.
virtual void Set(fdevent* fde, unsigned events) = 0;
void Add(fdevent* fde, unsigned events);
void Del(fdevent* fde, unsigned events);
// Set a timeout on an fdevent.
// If no events are triggered by the timeout, an FDE_TIMEOUT will be generated.
// Note timeouts are not defused automatically; if a timeout is set on an fdevent, it will
// trigger repeatedly every |timeout| ms.
void SetTimeout(fdevent* fde, std::optional<std::chrono::milliseconds> timeout);
// Loop until TerminateLoop is called, handling events.
// Implementations should call FlushRunQueue on every iteration, and check the value of
// terminate_loop_ to determine whether to stop.
virtual void Loop() = 0;
// Assert that the caller is either running on the context's main thread, or that there is no
// active main thread.
void CheckMainThread();
// Queue an operation to be run on the main thread.
void Run(std::function<void()> fn);
// Test-only functionality:
void TerminateLoop();
virtual size_t InstalledCount() = 0;
protected:
// Interrupt the run loop.
virtual void Interrupt() = 0;
// Run all pending functions enqueued via Run().
void FlushRunQueue() EXCLUDES(run_queue_mutex_);
std::optional<uint64_t> main_thread_id_ = std::nullopt;
std::atomic<bool> terminate_loop_ = false;
private:
uint64_t fdevent_id_ = 0;
std::mutex run_queue_mutex_;
std::deque<std::function<void()>> run_queue_ GUARDED_BY(run_queue_mutex_);
};
struct fdevent {
uint64_t id;
unique_fd fd;
int force_eof = 0;
uint16_t state = 0;
uint16_t events = 0;
std::optional<std::chrono::milliseconds> timeout;
std::chrono::steady_clock::time_point last_active;
std::variant<fd_func, fd_func2> func;
void* arg = nullptr;
};
// Backwards compatibility shims that forward to the global fdevent_context.
fdevent* fdevent_create(int fd, fd_func func, void* arg);
fdevent* fdevent_create(int fd, fd_func2 func, void* arg);
unique_fd fdevent_release(fdevent* fde);
void fdevent_destroy(fdevent* fde);
void fdevent_set(fdevent *fde, unsigned events);
void fdevent_add(fdevent *fde, unsigned events);
void fdevent_del(fdevent *fde, unsigned events);
void fdevent_set_timeout(fdevent* fde, std::optional<std::chrono::milliseconds> timeout);
void fdevent_loop();
void check_main_thread();
// Queue an operation to run on the main thread.
void fdevent_run_on_main_thread(std::function<void()> fn);
// The following functions are used only for tests.
void fdevent_terminate_loop();
size_t fdevent_installed_count();
void fdevent_reset();
#endif

View file

@ -0,0 +1,366 @@
/*
* Copyright (C) 2019 The Android Open Source Project
*
* 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.
*/
#define TRACE_TAG FDEVENT
#include "sysdeps.h"
#include "fdevent_poll.h"
#include <fcntl.h>
#include <inttypes.h>
#include <stdint.h>
#include <stdlib.h>
#include <string.h>
#include <unistd.h>
#include <atomic>
#include <deque>
#include <functional>
#include <list>
#include <mutex>
#include <optional>
#include <unordered_map>
#include <utility>
#include <variant>
#include <vector>
#include <android-base/chrono_utils.h>
#include <android-base/file.h>
#include <android-base/logging.h>
#include <android-base/stringprintf.h>
#include <android-base/threads.h>
#include "adb_io.h"
#include "adb_trace.h"
#include "adb_unique_fd.h"
#include "adb_utils.h"
#include "fdevent.h"
#include "sysdeps/chrono.h"
static void fdevent_interrupt(int fd, unsigned, void*) {
char buf[BUFSIZ];
ssize_t rc = TEMP_FAILURE_RETRY(adb_read(fd, buf, sizeof(buf)));
if (rc == -1) {
PLOG(FATAL) << "failed to read from fdevent interrupt fd";
}
}
fdevent_context_poll::fdevent_context_poll() {
int s[2];
if (adb_socketpair(s) != 0) {
PLOG(FATAL) << "failed to create fdevent interrupt socketpair";
}
if (!set_file_block_mode(s[0], false) || !set_file_block_mode(s[1], false)) {
PLOG(FATAL) << "failed to make fdevent interrupt socket nonblocking";
}
this->interrupt_fd_.reset(s[0]);
fdevent* fde = this->Create(unique_fd(s[1]), fdevent_interrupt, nullptr);
CHECK(fde != nullptr);
this->Add(fde, FDE_READ);
}
fdevent_context_poll::~fdevent_context_poll() {
this->Destroy(this->interrupt_fde_);
}
void fdevent_context_poll::Register(fdevent* fde) {
auto pair = poll_node_map_.emplace(fde->fd.get(), PollNode(fde));
CHECK(pair.second) << "install existing fd " << fde->fd.get();
}
void fdevent_context_poll::Unregister(fdevent* fde) {
if (fde->state & FDE_ACTIVE) {
poll_node_map_.erase(fde->fd.get());
if (fde->state & FDE_PENDING) {
pending_list_.remove(fde);
}
fde->state = 0;
fde->events = 0;
}
}
void fdevent_context_poll::Set(fdevent* fde, unsigned events) {
CheckMainThread();
events &= FDE_EVENTMASK;
if ((fde->state & FDE_EVENTMASK) == events) {
return;
}
CHECK(fde->state & FDE_ACTIVE);
auto it = poll_node_map_.find(fde->fd.get());
CHECK(it != poll_node_map_.end());
PollNode& node = it->second;
if (events & FDE_READ) {
node.pollfd.events |= POLLIN;
} else {
node.pollfd.events &= ~POLLIN;
}
if (events & FDE_WRITE) {
node.pollfd.events |= POLLOUT;
} else {
node.pollfd.events &= ~POLLOUT;
}
fde->state = (fde->state & FDE_STATEMASK) | events;
D("fdevent_set: %s, events = %u", dump_fde(fde).c_str(), events);
if (fde->state & FDE_PENDING) {
// If we are pending, make sure we don't signal an event that is no longer wanted.
fde->events &= events;
if (fde->events == 0) {
pending_list_.remove(fde);
fde->state &= ~FDE_PENDING;
}
}
}
static std::string dump_pollfds(const std::vector<adb_pollfd>& pollfds) {
std::string result;
for (const auto& pollfd : pollfds) {
std::string op;
if (pollfd.events & POLLIN) {
op += "R";
}
if (pollfd.events & POLLOUT) {
op += "W";
}
android::base::StringAppendF(&result, " %d(%s)", pollfd.fd, op.c_str());
}
return result;
}
static std::optional<std::chrono::milliseconds> calculate_timeout(fdevent_context_poll* ctx) {
std::optional<std::chrono::milliseconds> result = std::nullopt;
auto now = std::chrono::steady_clock::now();
ctx->CheckMainThread();
for (const auto& [fd, pollnode] : ctx->poll_node_map_) {
UNUSED(fd);
auto timeout_opt = pollnode.fde->timeout;
if (timeout_opt) {
auto deadline = pollnode.fde->last_active + *timeout_opt;
auto time_left = std::chrono::duration_cast<std::chrono::milliseconds>(deadline - now);
if (time_left < std::chrono::milliseconds::zero()) {
time_left = std::chrono::milliseconds::zero();
}
if (!result) {
result = time_left;
} else {
result = std::min(*result, time_left);
}
}
}
return result;
}
static void fdevent_process(fdevent_context_poll* ctx) {
std::vector<adb_pollfd> pollfds;
for (const auto& pair : ctx->poll_node_map_) {
pollfds.push_back(pair.second.pollfd);
}
CHECK_GT(pollfds.size(), 0u);
D("poll(), pollfds = %s", dump_pollfds(pollfds).c_str());
auto timeout = calculate_timeout(ctx);
int timeout_ms;
if (!timeout) {
timeout_ms = -1;
} else {
timeout_ms = timeout->count();
}
int ret = adb_poll(&pollfds[0], pollfds.size(), timeout_ms);
if (ret == -1) {
PLOG(ERROR) << "poll(), ret = " << ret;
return;
}
auto post_poll = std::chrono::steady_clock::now();
for (const auto& pollfd : pollfds) {
if (pollfd.revents != 0) {
D("for fd %d, revents = %x", pollfd.fd, pollfd.revents);
}
unsigned events = 0;
if (pollfd.revents & POLLIN) {
events |= FDE_READ;
}
if (pollfd.revents & POLLOUT) {
events |= FDE_WRITE;
}
if (pollfd.revents & (POLLERR | POLLHUP | POLLNVAL)) {
// We fake a read, as the rest of the code assumes that errors will
// be detected at that point.
events |= FDE_READ | FDE_ERROR;
}
#if defined(__linux__)
if (pollfd.revents & POLLRDHUP) {
events |= FDE_READ | FDE_ERROR;
}
#endif
auto it = ctx->poll_node_map_.find(pollfd.fd);
CHECK(it != ctx->poll_node_map_.end());
fdevent* fde = it->second.fde;
if (events == 0) {
// Check for timeout.
if (fde->timeout) {
auto deadline = fde->last_active + *fde->timeout;
if (deadline < post_poll) {
events |= FDE_TIMEOUT;
}
}
}
if (events != 0) {
CHECK_EQ(fde->fd.get(), pollfd.fd);
fde->events |= events;
fde->last_active = post_poll;
D("%s got events %x", dump_fde(fde).c_str(), events);
fde->state |= FDE_PENDING;
ctx->pending_list_.push_back(fde);
}
}
}
template <class T>
struct always_false : std::false_type {};
static void fdevent_call_fdfunc(fdevent* fde) {
unsigned events = fde->events;
fde->events = 0;
CHECK(fde->state & FDE_PENDING);
fde->state &= (~FDE_PENDING);
D("fdevent_call_fdfunc %s", dump_fde(fde).c_str());
std::visit(
[&](auto&& f) {
using F = std::decay_t<decltype(f)>;
if constexpr (std::is_same_v<fd_func, F>) {
f(fde->fd.get(), events, fde->arg);
} else if constexpr (std::is_same_v<fd_func2, F>) {
f(fde, events, fde->arg);
} else {
static_assert(always_false<F>::value, "non-exhaustive visitor");
}
},
fde->func);
}
static void fdevent_check_spin(fdevent_context_poll* ctx, uint64_t cycle) {
// Check to see if we're spinning because we forgot about an fdevent
// by keeping track of how long fdevents have been continuously pending.
struct SpinCheck {
fdevent* fde;
android::base::boot_clock::time_point timestamp;
uint64_t cycle;
};
// TODO: Move this into the base fdevent_context.
static auto& g_continuously_pending = *new std::unordered_map<uint64_t, SpinCheck>();
static auto last_cycle = android::base::boot_clock::now();
auto now = android::base::boot_clock::now();
if (now - last_cycle > 10ms) {
// We're not spinning.
g_continuously_pending.clear();
last_cycle = now;
return;
}
last_cycle = now;
for (auto* fde : ctx->pending_list_) {
auto it = g_continuously_pending.find(fde->id);
if (it == g_continuously_pending.end()) {
g_continuously_pending[fde->id] =
SpinCheck{.fde = fde, .timestamp = now, .cycle = cycle};
} else {
it->second.cycle = cycle;
}
}
for (auto it = g_continuously_pending.begin(); it != g_continuously_pending.end();) {
if (it->second.cycle != cycle) {
it = g_continuously_pending.erase(it);
} else {
// Use an absurdly long window, since all we really care about is
// getting a bugreport eventually.
if (now - it->second.timestamp > 300s) {
LOG(FATAL_WITHOUT_ABORT)
<< "detected spin in fdevent: " << dump_fde(it->second.fde);
#if defined(__linux__)
int fd = it->second.fde->fd.get();
std::string fd_path = android::base::StringPrintf("/proc/self/fd/%d", fd);
std::string path;
if (!android::base::Readlink(fd_path, &path)) {
PLOG(FATAL_WITHOUT_ABORT) << "readlink of fd " << fd << " failed";
}
LOG(FATAL_WITHOUT_ABORT) << "fd " << fd << " = " << path;
#endif
abort();
}
++it;
}
}
}
void fdevent_context_poll::Loop() {
main_thread_id_ = android::base::GetThreadId();
uint64_t cycle = 0;
while (true) {
if (terminate_loop_) {
break;
}
D("--- --- waiting for events");
fdevent_process(this);
fdevent_check_spin(this, cycle++);
while (!pending_list_.empty()) {
fdevent* fde = pending_list_.front();
pending_list_.pop_front();
fdevent_call_fdfunc(fde);
}
this->FlushRunQueue();
}
main_thread_id_.reset();
}
size_t fdevent_context_poll::InstalledCount() {
// We always have an installed fde for interrupt.
return poll_node_map_.size() - 1;
}
void fdevent_context_poll::Interrupt() {
int rc = adb_write(this->interrupt_fd_, "", 1);
// It's possible that we get EAGAIN here, if lots of notifications came in while handling.
if (rc == 0) {
PLOG(FATAL) << "fdevent interrupt fd was closed?";
} else if (rc == -1 && errno != EAGAIN) {
PLOG(FATAL) << "failed to write to fdevent interrupt fd";
}
}

View file

@ -0,0 +1,71 @@
#pragma once
/*
* Copyright (C) 2019 The Android Open Source Project
*
* 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 "sysdeps.h"
#include <deque>
#include <list>
#include <mutex>
#include <unordered_map>
#include <android-base/thread_annotations.h>
#include "adb_unique_fd.h"
#include "fdevent.h"
struct PollNode {
fdevent* fde;
adb_pollfd pollfd;
explicit PollNode(fdevent* fde) : fde(fde) {
memset(&pollfd, 0, sizeof(pollfd));
pollfd.fd = fde->fd.get();
#if defined(__linux__)
// Always enable POLLRDHUP, so the host server can take action when some clients disconnect.
// Then we can avoid leaving many sockets in CLOSE_WAIT state. See http://b/23314034.
pollfd.events = POLLRDHUP;
#endif
}
};
struct fdevent_context_poll : public fdevent_context {
fdevent_context_poll();
virtual ~fdevent_context_poll();
virtual void Register(fdevent* fde) final;
virtual void Unregister(fdevent* fde) final;
virtual void Set(fdevent* fde, unsigned events) final;
virtual void Loop() final;
virtual size_t InstalledCount() final;
protected:
virtual void Interrupt() final;
public:
// All operations to fdevent should happen only in the main thread.
// That's why we don't need a lock for fdevent.
std::unordered_map<int, PollNode> poll_node_map_;
std::list<fdevent*> pending_list_;
unique_fd interrupt_fd_;
fdevent* interrupt_fde_ = nullptr;
};

View file

@ -78,8 +78,8 @@ class FdeventTest : public ::testing::Test {
}
size_t GetAdditionalLocalSocketCount() {
// dummy socket installed in PrepareThread() + fdevent_run_on_main_thread socket
return 2;
// dummy socket installed in PrepareThread()
return 1;
}
void TerminateThread() {

View file

@ -24,7 +24,7 @@
#include <string>
#include "adb_unique_fd.h"
#include "fdevent.h"
#include "fdevent/fdevent.h"
#include "types.h"
class atransport;

View file

@ -14,7 +14,7 @@
* limitations under the License.
*/
#include "fdevent.h"
#include "fdevent/fdevent.h"
#include <gtest/gtest.h>
@ -29,7 +29,7 @@
#include "adb.h"
#include "adb_io.h"
#include "fdevent_test.h"
#include "fdevent/fdevent_test.h"
#include "socket.h"
#include "sysdeps.h"
#include "sysdeps/chrono.h"

View file

@ -64,8 +64,6 @@
#include <memory> // unique_ptr
#include <string>
#include "fdevent.h"
#define OS_PATH_SEPARATORS "\\/"
#define OS_PATH_SEPARATOR '\\'
#define OS_PATH_SEPARATOR_STR "\\"

View file

@ -49,7 +49,7 @@
#include "adb_io.h"
#include "adb_trace.h"
#include "adb_utils.h"
#include "fdevent.h"
#include "fdevent/fdevent.h"
#include "sysdeps/chrono.h"
using android::base::ScopedLockAssertion;

View file

@ -19,7 +19,7 @@
#include <gtest/gtest.h>
#include "adb.h"
#include "fdevent_test.h"
#include "fdevent/fdevent_test.h"
struct TransportTest : public FdeventTest {};