1
0
Fork 0

[3.5] Feature/decoupled io (#6281)

* Decoupled IO from Scheduler.
* Fixed SSL start up bug.
* Updated messages and thread names. Fixed missing code from cherry-pick.
* Reintroduced checks for executing thread to be correct. Modifed default value for io-context depending on cores.
* Fixed memory leak caused by cyclic references.
* Actually distribute endpoints. Move handlers into function and do not copy them for each encapsulation.
* Inserted debug output.
* BUG FIXED! One has to call drain() on every queue as temporary work around.
* Added some flags and output for testing.
* More debug output!!!
* Manuel is right.
* Removed debug output.
This commit is contained in:
Lars Maier 2018-10-08 13:05:12 +02:00 committed by Max Neunhöffer
parent 94805aad7c
commit fac7b48c74
35 changed files with 529 additions and 216 deletions

View File

@ -331,6 +331,7 @@ SET(ARANGOD_SOURCES
GeneralServer/GeneralServer.cpp GeneralServer/GeneralServer.cpp
GeneralServer/GeneralServerFeature.cpp GeneralServer/GeneralServerFeature.cpp
GeneralServer/HttpCommTask.cpp GeneralServer/HttpCommTask.cpp
GeneralServer/IoTask.cpp
GeneralServer/RestHandler.cpp GeneralServer/RestHandler.cpp
GeneralServer/RestHandlerFactory.cpp GeneralServer/RestHandlerFactory.cpp
GeneralServer/VstCommTask.cpp GeneralServer/VstCommTask.cpp

View File

@ -64,15 +64,16 @@ static std::string const Open("/_open/");
// --SECTION-- constructors and destructors // --SECTION-- constructors and destructors
// ----------------------------------------------------------------------------- // -----------------------------------------------------------------------------
GeneralCommTask::GeneralCommTask(Scheduler* scheduler, GeneralServer* server, GeneralCommTask::GeneralCommTask(GeneralServer &server,
GeneralServer::IoContext &context,
std::unique_ptr<Socket> socket, std::unique_ptr<Socket> socket,
ConnectionInfo&& info, double keepAliveTimeout, ConnectionInfo&& info, double keepAliveTimeout,
bool skipSocketInit) bool skipSocketInit)
: Task(scheduler, "GeneralCommTask"), : IoTask(server, context, "GeneralCommTask"),
SocketTask(scheduler, std::move(socket), std::move(info), SocketTask(server, context, std::move(socket), std::move(info),
keepAliveTimeout, skipSocketInit), keepAliveTimeout, skipSocketInit),
_server(server),
_auth(AuthenticationFeature::instance()) { _auth(AuthenticationFeature::instance()) {
TRI_ASSERT(_auth != nullptr); TRI_ASSERT(_auth != nullptr);
} }
@ -154,6 +155,7 @@ GeneralCommTask::RequestFlow GeneralCommTask::prepareExecution(GeneralRequest& r
// Step 2: Handle server-modes, i.e. bootstrap/ Active-Failover / DC2DC stunts // Step 2: Handle server-modes, i.e. bootstrap/ Active-Failover / DC2DC stunts
std::string const& path = req.requestPath(); std::string const& path = req.requestPath();
ServerState::Mode mode = ServerState::mode(); ServerState::Mode mode = ServerState::mode();
switch (mode) { switch (mode) {
case ServerState::Mode::MAINTENANCE: { case ServerState::Mode::MAINTENANCE: {
@ -459,18 +461,14 @@ void GeneralCommTask::handleRequestDirectly(
auto self = shared_from_this(); auto self = shared_from_this();
handler->runHandler([self, this, doLock](rest::RestHandler* handler) { handler->runHandler([self, this, doLock](rest::RestHandler* handler) {
RequestStatistics* stat = handler->stealStatistics();
// TODO we could reduce all of this to strand::dispatch ?
if (doLock || !_peer->runningInThisThread()) {
// Note that the latter is for the case that a handler was put to sleep
// and woke up in a different thread.
auto h = handler->shared_from_this();
RequestStatistics* stat = handler->stealStatistics();
auto h = handler->shared_from_this();
// Pass the response the io context
_peer->post( _peer->post(
[self, this, stat, h]() { addResponse(*(h->response()), stat); }); [self, this, stat, h]() {
} else { addResponse(*(h->response()), stat);
addResponse(*handler->response(), stat); });
}
}); });
} }

View File

@ -28,6 +28,7 @@
#include "Scheduler/SocketTask.h" #include "Scheduler/SocketTask.h"
#include <openssl/ssl.h> #include <openssl/ssl.h>
#include "GeneralServer/GeneralServer.h"
#include "Basics/Mutex.h" #include "Basics/Mutex.h"
#include "Basics/MutexLocker.h" #include "Basics/MutexLocker.h"
@ -40,7 +41,6 @@ class GeneralRequest;
class GeneralResponse; class GeneralResponse;
namespace rest { namespace rest {
class GeneralServer;
class RestHandler; class RestHandler;
// //
@ -85,7 +85,7 @@ class GeneralCommTask : public SocketTask {
GeneralCommTask const& operator=(GeneralCommTask const&) = delete; GeneralCommTask const& operator=(GeneralCommTask const&) = delete;
public: public:
GeneralCommTask(Scheduler*, GeneralServer*, std::unique_ptr<Socket>, GeneralCommTask(GeneralServer &server, GeneralServer::IoContext&, std::unique_ptr<Socket>,
ConnectionInfo&&, double keepAliveTimeout, ConnectionInfo&&, double keepAliveTimeout,
bool skipSocketInit = false); bool skipSocketInit = false);
@ -135,7 +135,6 @@ class GeneralCommTask : public SocketTask {
uint64_t messageId, int errorNum); uint64_t messageId, int errorNum);
protected: protected:
GeneralServer* const _server;
AuthenticationFeature* _auth; AuthenticationFeature* _auth;
// protocol to use http, vst // protocol to use http, vst

View File

@ -38,12 +38,11 @@ using namespace arangodb::rest;
/// @brief listen to given port /// @brief listen to given port
//////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////
GeneralListenTask::GeneralListenTask(Scheduler* scheduler, GeneralServer* server, GeneralListenTask::GeneralListenTask(GeneralServer &server, GeneralServer::IoContext& context,
Endpoint* endpoint, Endpoint* endpoint,
ProtocolType connectionType) ProtocolType connectionType)
: Task(scheduler, "GeneralListenTask"), : IoTask(server, context, "GeneralListenTask"),
ListenTask(scheduler, endpoint), ListenTask(server, context, endpoint),
_server(server),
_connectionType(connectionType) { _connectionType(connectionType) {
_keepAliveTimeout = GeneralServerFeature::keepAliveTimeout(); _keepAliveTimeout = GeneralServerFeature::keepAliveTimeout();
@ -52,7 +51,7 @@ GeneralListenTask::GeneralListenTask(Scheduler* scheduler, GeneralServer* server
void GeneralListenTask::handleConnected(std::unique_ptr<Socket> socket, void GeneralListenTask::handleConnected(std::unique_ptr<Socket> socket,
ConnectionInfo&& info) { ConnectionInfo&& info) {
auto commTask = std::make_shared<HttpCommTask>(_scheduler, _server, std::move(socket), auto commTask = std::make_shared<HttpCommTask>(_server, _context, std::move(socket),
std::move(info), _keepAliveTimeout); std::move(info), _keepAliveTimeout);
bool res = commTask->start(); bool res = commTask->start();
LOG_TOPIC_IF(DEBUG, Logger::COMMUNICATION, res) << "Started comm task"; LOG_TOPIC_IF(DEBUG, Logger::COMMUNICATION, res) << "Started comm task";

View File

@ -25,11 +25,13 @@
#ifndef ARANGOD_HTTP_SERVER_HTTP_LISTEN_TASK_H #ifndef ARANGOD_HTTP_SERVER_HTTP_LISTEN_TASK_H
#define ARANGOD_HTTP_SERVER_HTTP_LISTEN_TASK_H 1 #define ARANGOD_HTTP_SERVER_HTTP_LISTEN_TASK_H 1
#include "Scheduler/ListenTask.h"
#include <openssl/ssl.h> #include <openssl/ssl.h>
#include "GeneralServer/GeneralDefinitions.h" #include "GeneralServer/GeneralDefinitions.h"
#include "GeneralServer/GeneralServer.h"
#include "Scheduler/ListenTask.h"
namespace arangodb { namespace arangodb {
class Endpoint; class Endpoint;
@ -42,7 +44,7 @@ class GeneralListenTask final : public ListenTask {
GeneralListenTask& operator=(GeneralListenTask const&) = delete; GeneralListenTask& operator=(GeneralListenTask const&) = delete;
public: public:
GeneralListenTask(Scheduler*, GeneralServer*, Endpoint*, GeneralListenTask(GeneralServer &server, GeneralServer::IoContext&, Endpoint*,
ProtocolType connectionType); ProtocolType connectionType);
protected: protected:
@ -50,7 +52,6 @@ class GeneralListenTask final : public ListenTask {
ConnectionInfo&&) override; ConnectionInfo&&) override;
private: private:
GeneralServer* _server;
ProtocolType const _connectionType; ProtocolType const _connectionType;
double _keepAliveTimeout = 300.0; double _keepAliveTimeout = 300.0;
}; };

View File

@ -40,16 +40,25 @@ using namespace arangodb::rest;
// --SECTION-- public methods // --SECTION-- public methods
// ----------------------------------------------------------------------------- // -----------------------------------------------------------------------------
GeneralServer::GeneralServer(uint64_t numIoThreads) :
_numIoThreads(numIoThreads),
_contexts(numIoThreads)
{}
void GeneralServer::setEndpointList(EndpointList const* list) { void GeneralServer::setEndpointList(EndpointList const* list) {
_endpointList = list; _endpointList = list;
} }
void GeneralServer::startListening() { void GeneralServer::startListening() {
unsigned int i = 0;
for (auto& it : _endpointList->allEndpoints()) { for (auto& it : _endpointList->allEndpoints()) {
LOG_TOPIC(TRACE, arangodb::Logger::FIXME) << "trying to bind to endpoint '" LOG_TOPIC(TRACE, arangodb::Logger::FIXME) << "trying to bind to endpoint '"
<< it.first << "' for requests"; << it.first << "' for requests";
bool ok = openEndpoint(it.second); // distribute endpoints across all io contexts
IoContext &ioContext = _contexts[i++ % _numIoThreads];
bool ok = openEndpoint(ioContext, it.second);
if (ok) { if (ok) {
LOG_TOPIC(DEBUG, arangodb::Logger::FIXME) << "bound to endpoint '" LOG_TOPIC(DEBUG, arangodb::Logger::FIXME) << "bound to endpoint '"
@ -66,8 +75,9 @@ void GeneralServer::startListening() {
} }
void GeneralServer::stopListening() { void GeneralServer::stopListening() {
for (auto& task : _listenTasks) { LOG_TOPIC(ERR, Logger::FIXME) << "GeneralServer::stopListening()";
task->stop(); for (auto& context : _contexts) {
context.stop();
} }
} }
@ -75,7 +85,7 @@ void GeneralServer::stopListening() {
// --SECTION-- protected methods // --SECTION-- protected methods
// ----------------------------------------------------------------------------- // -----------------------------------------------------------------------------
bool GeneralServer::openEndpoint(Endpoint* endpoint) { bool GeneralServer::openEndpoint(IoContext &ioContext, Endpoint* endpoint) {
ProtocolType protocolType; ProtocolType protocolType;
if (endpoint->encryption() == Endpoint::EncryptionType::SSL) { if (endpoint->encryption() == Endpoint::EncryptionType::SSL) {
@ -84,13 +94,61 @@ bool GeneralServer::openEndpoint(Endpoint* endpoint) {
protocolType = ProtocolType::HTTP; protocolType = ProtocolType::HTTP;
} }
std::unique_ptr<ListenTask> task;
task.reset(new GeneralListenTask(SchedulerFeature::SCHEDULER, this, endpoint, auto task = std::make_shared<GeneralListenTask> (*this, ioContext, endpoint, protocolType);
protocolType));
if (!task->start()) { if (!task->start()) {
return false; return false;
} }
_listenTasks.emplace_back(std::move(task));
return true; return true;
} }
GeneralServer::IoThread::~IoThread() {
shutdown();
}
GeneralServer::IoThread::IoThread(IoContext &iocontext) :
Thread("Io"), _iocontext(iocontext) {}
void GeneralServer::IoThread::run() {
// run the asio io context
_iocontext._asioIoContext.run();
}
GeneralServer::IoContext::IoContext() :
_clients(0),
_thread(*this),
_asioIoContext(1), // only a single thread per context
_asioWork(_asioIoContext),
_stopped(false)
{
_thread.start();
}
GeneralServer::IoContext::~IoContext() {
stop();
}
void GeneralServer::IoContext::stop() {
_asioIoContext.stop();
}
GeneralServer::IoContext &GeneralServer::selectIoContext()
{
uint32_t low = _contexts[0]._clients.load();
size_t lowpos = 0;
for (size_t i = 1; i < _contexts.size(); ++i) {
uint32_t x = _contexts[i]._clients.load();
if (x < low) {
low = x;
lowpos = i;
}
}
return _contexts[lowpos];
}

View File

@ -27,32 +27,126 @@
#define ARANGOD_HTTP_SERVER_HTTP_SERVER_H 1 #define ARANGOD_HTTP_SERVER_HTTP_SERVER_H 1
#include "Basics/Common.h" #include "Basics/Common.h"
#include "Basics/asio_ns.h"
#include "GeneralServer/HttpCommTask.h" #include "Basics/Thread.h"
#include "Scheduler/ListenTask.h" #include "Endpoint/Endpoint.h"
namespace arangodb { namespace arangodb {
class EndpointList; class EndpointList;
class ListenTask;
namespace rest { namespace rest {
class GeneralServer { class GeneralServer {
GeneralServer(GeneralServer const&) = delete; GeneralServer(GeneralServer const&) = delete;
GeneralServer const& operator=(GeneralServer const&) = delete; GeneralServer const& operator=(GeneralServer const&) = delete;
public: public:
GeneralServer() = default; GeneralServer(uint64_t numIoThreads);
public: public:
void setEndpointList(EndpointList const* list); void setEndpointList(EndpointList const* list);
void startListening(); void startListening();
void stopListening(); void stopListening();
protected: class IoContext;
bool openEndpoint(Endpoint* endpoint);
private: private:
std::vector<std::unique_ptr<ListenTask>> _listenTasks; class IoThread final : public Thread {
public:
IoThread(IoContext &iocontext);
~IoThread();
void run();
private:
IoContext &_iocontext;
};
public:
class IoContext {
friend class IoThread;
friend class GeneralServer;
public:
std::atomic<uint64_t> _clients;
private:
IoThread _thread;
asio_ns::io_context _asioIoContext;
asio_ns::io_context::work _asioWork;
std::atomic<bool> _stopped;
public:
IoContext();
~IoContext();
template <typename T>
asio_ns::deadline_timer* newDeadlineTimer(T timeout) {
return new asio_ns::deadline_timer(_asioIoContext, timeout);
}
asio_ns::steady_timer* newSteadyTimer() {
return new asio_ns::steady_timer(_asioIoContext);
}
asio_ns::io_context::strand* newStrand() {
return new asio_ns::io_context::strand(_asioIoContext);
}
asio_ns::ip::tcp::acceptor* newAcceptor() {
return new asio_ns::ip::tcp::acceptor(_asioIoContext);
}
#ifndef _WIN32
asio_ns::local::stream_protocol::acceptor* newDomainAcceptor() {
return new asio_ns::local::stream_protocol::acceptor(_asioIoContext);
}
#endif
asio_ns::ip::tcp::socket* newSocket() {
return new asio_ns::ip::tcp::socket(_asioIoContext);
}
#ifndef _WIN32
asio_ns::local::stream_protocol::socket* newDomainSocket() {
return new asio_ns::local::stream_protocol::socket(_asioIoContext);
}
#endif
asio_ns::ssl::stream<asio_ns::ip::tcp::socket>* newSslSocket(
asio_ns::ssl::context& sslContext) {
return new asio_ns::ssl::stream<asio_ns::ip::tcp::socket>(_asioIoContext,
sslContext);
}
asio_ns::ip::tcp::resolver* newResolver() {
return new asio_ns::ip::tcp::resolver(_asioIoContext);
}
void post(std::function<void()> && handler) {
_asioIoContext.post(std::move(handler));
}
void start();
void stop();
bool runningInThisThread() { return _thread.runningInThisThread(); }
private:
};
GeneralServer::IoContext &selectIoContext();
protected:
bool openEndpoint(IoContext &ioContext, Endpoint* endpoint);
private:
friend class IoThread;
friend class IoContext;
uint64_t _numIoThreads;
std::vector<IoContext> _contexts;
EndpointList const* _endpointList = nullptr; EndpointList const* _endpointList = nullptr;
}; };
} }

View File

@ -100,6 +100,8 @@ using namespace arangodb::options;
namespace arangodb { namespace arangodb {
static uint64_t const _maxIoThreads = 64;
rest::RestHandlerFactory* GeneralServerFeature::HANDLER_FACTORY = nullptr; rest::RestHandlerFactory* GeneralServerFeature::HANDLER_FACTORY = nullptr;
rest::AsyncJobManager* GeneralServerFeature::JOB_MANAGER = nullptr; rest::AsyncJobManager* GeneralServerFeature::JOB_MANAGER = nullptr;
GeneralServerFeature* GeneralServerFeature::GENERAL_SERVER = nullptr; GeneralServerFeature* GeneralServerFeature::GENERAL_SERVER = nullptr;
@ -109,12 +111,20 @@ GeneralServerFeature::GeneralServerFeature(
) )
: ApplicationFeature(server, "GeneralServer"), : ApplicationFeature(server, "GeneralServer"),
_allowMethodOverride(false), _allowMethodOverride(false),
_proxyCheck(true) { _proxyCheck(true),
_numIoThreads(0) {
setOptional(true); setOptional(true);
startsAfter("AQLPhase"); startsAfter("AQLPhase");
startsAfter("Endpoint"); startsAfter("Endpoint");
startsAfter("Upgrade"); startsAfter("Upgrade");
startsAfter("SslServer");
_numIoThreads = (std::max)(static_cast<uint64_t>(1),
static_cast<uint64_t>(TRI_numberProcessors() / 4));
if (_numIoThreads > _maxIoThreads) {
_numIoThreads = _maxIoThreads;
}
// TODO The following features are too high // TODO The following features are too high
// startsAfter("Agency"); Only need to know if it is enabled during start that is clear before // startsAfter("Agency"); Only need to know if it is enabled during start that is clear before
@ -133,6 +143,11 @@ void GeneralServerFeature::collectOptions(
options->addOldOption("server.default-api-compatibility", ""); options->addOldOption("server.default-api-compatibility", "");
options->addOldOption("no-server", "server.rest-server"); options->addOldOption("no-server", "server.rest-server");
options->addOption(
"--server.io-threads",
"Number of threads used to handle IO",
new UInt64Parameter(&_numIoThreads));
options->addSection("http", "HttpServer features"); options->addSection("http", "HttpServer features");
options->addHiddenOption("--http.allow-method-override", options->addHiddenOption("--http.allow-method-override",
@ -193,6 +208,17 @@ void GeneralServerFeature::validateOptions(std::shared_ptr<ProgramOptions>) {
}), }),
_accessControlAllowOrigins.end()); _accessControlAllowOrigins.end());
} }
// we need at least one io thread and context
if (_numIoThreads == 0) {
LOG_TOPIC(WARN, Logger::FIXME)
<< "Need at least one io-context thread.";
_numIoThreads = 1;
} else if (_numIoThreads > _maxIoThreads) {
LOG_TOPIC(WARN, Logger::FIXME)
<< "IO-contexts are limited to " << _maxIoThreads;
_numIoThreads = _maxIoThreads;
}
} }
void GeneralServerFeature::prepare() { void GeneralServerFeature::prepare() {
@ -261,7 +287,7 @@ void GeneralServerFeature::buildServers() {
ssl->SSL->verifySslOptions(); ssl->SSL->verifySslOptions();
} }
GeneralServer* server = new GeneralServer(); GeneralServer* server = new GeneralServer(_numIoThreads);
server->setEndpointList(&endpointList); server->setEndpointList(&endpointList);
_servers.push_back(server); _servers.push_back(server);

View File

@ -122,6 +122,7 @@ class GeneralServerFeature final
std::pair<aql::QueryRegistry*, traverser::TraverserEngineRegistry*>> std::pair<aql::QueryRegistry*, traverser::TraverserEngineRegistry*>>
_combinedRegistries; _combinedRegistries;
std::vector<rest::GeneralServer*> _servers; std::vector<rest::GeneralServer*> _servers;
uint64_t _numIoThreads;
}; };
} }

View File

@ -46,11 +46,11 @@ size_t const HttpCommTask::MaximalBodySize = 1024 * 1024 * 1024; // 1024 MB
size_t const HttpCommTask::MaximalPipelineSize = 1024 * 1024 * 1024; // 1024 MB size_t const HttpCommTask::MaximalPipelineSize = 1024 * 1024 * 1024; // 1024 MB
size_t const HttpCommTask::RunCompactEvery = 500; size_t const HttpCommTask::RunCompactEvery = 500;
HttpCommTask::HttpCommTask(Scheduler* scheduler, GeneralServer* server, HttpCommTask::HttpCommTask(GeneralServer &server, GeneralServer::IoContext &context,
std::unique_ptr<Socket> socket, std::unique_ptr<Socket> socket,
ConnectionInfo&& info, double timeout) ConnectionInfo&& info, double timeout)
: Task(scheduler, "HttpCommTask"), : IoTask(server, context, "HttpCommTask"),
GeneralCommTask(scheduler, server, std::move(socket), std::move(info), GeneralCommTask(server, context, std::move(socket), std::move(info),
timeout), timeout),
_readPosition(0), _readPosition(0),
_startPosition(0), _startPosition(0),
@ -299,7 +299,7 @@ bool HttpCommTask::processRead(double startTime) {
} }
std::shared_ptr<GeneralCommTask> commTask = std::make_shared<VstCommTask>( std::shared_ptr<GeneralCommTask> commTask = std::make_shared<VstCommTask>(
_scheduler, _server, std::move(_peer), std::move(_connectionInfo), _server, _context, std::move(_peer), std::move(_connectionInfo),
GeneralServerFeature::keepAliveTimeout(), GeneralServerFeature::keepAliveTimeout(),
protocolVersion, /*skipSocketInit*/ true); protocolVersion, /*skipSocketInit*/ true);
commTask->addToReadBuffer(_readBuffer.c_str() + 11, commTask->addToReadBuffer(_readBuffer.c_str() + 11,

View File

@ -17,7 +17,7 @@ class HttpCommTask final : public GeneralCommTask {
static size_t const RunCompactEvery; static size_t const RunCompactEvery;
public: public:
HttpCommTask(Scheduler*, GeneralServer*, std::unique_ptr<Socket> socket, HttpCommTask(GeneralServer &server, GeneralServer::IoContext &context, std::unique_ptr<Socket> socket,
ConnectionInfo&&, double timeout); ConnectionInfo&&, double timeout);
arangodb::Endpoint::TransportType transportType() override { arangodb::Endpoint::TransportType transportType() override {

View File

@ -0,0 +1,42 @@
////////////////////////////////////////////////////////////////////////////////
/// DISCLAIMER
///
/// Copyright 2014-2016 ArangoDB GmbH, Cologne, Germany
/// Copyright 2004-2014 triAGENS GmbH, Cologne, Germany
///
/// 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.
///
/// Copyright holder is ArangoDB GmbH, Cologne, Germany
///
/// @author Dr. Frank Celler
/// @author Achim Brandt
////////////////////////////////////////////////////////////////////////////////
#include "IoTask.h"
#include <velocypack/Builder.h>
#include <velocypack/velocypack-aliases.h>
using namespace arangodb::rest;
namespace {
std::atomic_uint_fast64_t NEXT_IO_TASK_ID(static_cast<uint64_t>(TRI_microtime() *
100000.0));
}
IoTask::IoTask(GeneralServer &server, GeneralServer::IoContext &context,
std::string const& name)
: _context(context),
_server(server),
_taskId(NEXT_IO_TASK_ID++),
_name(name) {}

View File

@ -0,0 +1,64 @@
////////////////////////////////////////////////////////////////////////////////
/// DISCLAIMER
///
/// Copyright 2014-2016 ArangoDB GmbH, Cologne, Germany
/// Copyright 2004-2014 triAGENS GmbH, Cologne, Germany
///
/// 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.
///
/// Copyright holder is ArangoDB GmbH, Cologne, Germany
///
/// @author Dr. Frank Celler
/// @author Achim Brandt
////////////////////////////////////////////////////////////////////////////////
#ifndef ARANGOD_SCHEDULER_IO_TASK_H
#define ARANGOD_SCHEDULER_IO_TASK_H 1
#include "Basics/Common.h"
#include "GeneralServer/GeneralServer.h"
namespace arangodb {
namespace velocypack {
class Builder;
}
namespace rest {
class IoTask : public std::enable_shared_from_this<IoTask> {
IoTask(IoTask const&) = delete;
IoTask& operator=(IoTask const&) = delete;
public:
IoTask(GeneralServer &server, GeneralServer::IoContext&, std::string const& name);
virtual ~IoTask() = default;
public:
std::string const& name() const { return _name; }
// get a VelocyPack representation of the IoTask for reporting
std::shared_ptr<arangodb::velocypack::Builder> toVelocyPack() const;
void toVelocyPack(arangodb::velocypack::Builder&) const;
protected:
GeneralServer::IoContext &_context;
GeneralServer &_server;
uint64_t const _taskId;
private:
std::string const _name;
};
}
}
#endif

View File

@ -75,12 +75,12 @@ inline void validateMessage(char const* vpStart, char const* vpEnd) {
} }
VstCommTask::VstCommTask(Scheduler* scheduler, GeneralServer* server, VstCommTask::VstCommTask(GeneralServer &server, GeneralServer::IoContext &context,
std::unique_ptr<Socket> socket, ConnectionInfo&& info, std::unique_ptr<Socket> socket, ConnectionInfo&& info,
double timeout, ProtocolVersion protocolVersion, double timeout, ProtocolVersion protocolVersion,
bool skipInit) bool skipInit)
: Task(scheduler, "VstCommTask"), : IoTask(server, context, "VstCommTask"),
GeneralCommTask(scheduler, server, std::move(socket), std::move(info), timeout, GeneralCommTask(server, context, std::move(socket), std::move(info), timeout,
skipInit), skipInit),
_authorized(!_auth->isActive()), _authorized(!_auth->isActive()),
_authMethod(rest::AuthenticationMethod::NONE), _authMethod(rest::AuthenticationMethod::NONE),
@ -299,6 +299,7 @@ void VstCommTask::handleAuthHeader(VPackSlice const& header,
if (_authorized || !_auth->isActive()) { if (_authorized || !_auth->isActive()) {
_authenticatedUser = std::move(entry._username); _authenticatedUser = std::move(entry._username);
// simon: drivers expect a response for their auth request // simon: drivers expect a response for their auth request
addErrorResponse(ResponseCode::OK, rest::ContentType::VPACK, messageId, TRI_ERROR_NO_ERROR, addErrorResponse(ResponseCode::OK, rest::ContentType::VPACK, messageId, TRI_ERROR_NO_ERROR,
"auth successful"); "auth successful");
} else { } else {
@ -375,6 +376,7 @@ bool VstCommTask::processRead(double startTime) {
TRI_ASSERT(header.at(1).isNumber<int>()); // va TRI_ASSERT(header.at(1).isNumber<int>()); // va
int type = header.at(1).getNumber<int>(); int type = header.at(1).getNumber<int>();
// handle request types // handle request types
if (type == 1000) { // auth if (type == 1000) { // auth
handleAuthHeader(header, chunkHeader._messageID); handleAuthHeader(header, chunkHeader._messageID);
@ -392,6 +394,7 @@ bool VstCommTask::processRead(double startTime) {
} }
RequestFlow cont = prepareExecution(*req.get()); RequestFlow cont = prepareExecution(*req.get());
if (cont == RequestFlow::Continue) { if (cont == RequestFlow::Continue) {
auto resp = std::make_unique<VstResponse>(rest::ResponseCode::SERVER_ERROR, auto resp = std::make_unique<VstResponse>(rest::ResponseCode::SERVER_ERROR,
chunkHeader._messageID); chunkHeader._messageID);

View File

@ -38,7 +38,8 @@ namespace rest {
class VstCommTask final : public GeneralCommTask { class VstCommTask final : public GeneralCommTask {
public: public:
VstCommTask(Scheduler*, GeneralServer*, std::unique_ptr<Socket> socket, VstCommTask(GeneralServer &server, GeneralServer::IoContext &context,
std::unique_ptr<Socket> socket,
ConnectionInfo&&, double timeout, ProtocolVersion protocolVersion, ConnectionInfo&&, double timeout, ProtocolVersion protocolVersion,
bool skipSocketInit = false); bool skipSocketInit = false);

View File

@ -31,15 +31,17 @@
using namespace arangodb; using namespace arangodb;
Acceptor::Acceptor(rest::Scheduler* scheduler, Endpoint* endpoint) Acceptor::Acceptor(rest::GeneralServer &server,
: _scheduler(scheduler), _endpoint(endpoint) {} rest::GeneralServer::IoContext &context, Endpoint* endpoint)
: _server(server), _context(context), _endpoint(endpoint) {}
std::unique_ptr<Acceptor> Acceptor::factory(rest::Scheduler* scheduler, std::unique_ptr<Acceptor> Acceptor::factory(rest::GeneralServer &server,
rest::GeneralServer::IoContext &context,
Endpoint* endpoint) { Endpoint* endpoint) {
#ifdef ARANGODB_HAVE_DOMAIN_SOCKETS #ifdef ARANGODB_HAVE_DOMAIN_SOCKETS
if (endpoint->domainType() == Endpoint::DomainType::UNIX) { if (endpoint->domainType() == Endpoint::DomainType::UNIX) {
return std::make_unique<AcceptorUnixDomain>(scheduler, endpoint); return std::make_unique<AcceptorUnixDomain>(server, context, endpoint);
} }
#endif #endif
return std::make_unique<AcceptorTcp>(scheduler, endpoint); return std::make_unique<AcceptorTcp>(server, context, endpoint);
} }

View File

@ -36,7 +36,8 @@ class Acceptor {
typedef std::function<void(asio_ns::error_code const&)> AcceptHandler; typedef std::function<void(asio_ns::error_code const&)> AcceptHandler;
public: public:
Acceptor(rest::Scheduler*, Endpoint* endpoint); Acceptor(rest::GeneralServer &server,
rest::GeneralServer::IoContext &context, Endpoint* endpoint);
virtual ~Acceptor() {} virtual ~Acceptor() {}
public: public:
@ -46,10 +47,13 @@ class Acceptor {
std::unique_ptr<Socket> movePeer() { return std::move(_peer); }; std::unique_ptr<Socket> movePeer() { return std::move(_peer); };
public: public:
static std::unique_ptr<Acceptor> factory(rest::Scheduler*, Endpoint*); static std::unique_ptr<Acceptor> factory(rest::GeneralServer &server,
rest::GeneralServer::IoContext &context, Endpoint*);
protected: protected:
rest::Scheduler* _scheduler;
rest::GeneralServer &_server;
rest::GeneralServer::IoContext &_context;
Endpoint* _endpoint; Endpoint* _endpoint;
std::unique_ptr<Socket> _peer; std::unique_ptr<Socket> _peer;
}; };

View File

@ -31,7 +31,7 @@
using namespace arangodb; using namespace arangodb;
void AcceptorTcp::open() { void AcceptorTcp::open() {
std::unique_ptr<asio_ns::ip::tcp::resolver> resolver(_scheduler->newResolver()); std::unique_ptr<asio_ns::ip::tcp::resolver> resolver(_context.newResolver());
std::string hostname = _endpoint->host(); std::string hostname = _endpoint->host();
int portNumber = _endpoint->port(); int portNumber = _endpoint->port();
@ -109,13 +109,18 @@ void AcceptorTcp::open() {
void AcceptorTcp::asyncAccept(AcceptHandler const& handler) { void AcceptorTcp::asyncAccept(AcceptHandler const& handler) {
TRI_ASSERT(!_peer); TRI_ASSERT(!_peer);
// select the io context for this socket
auto &context = _server.selectIoContext();
if (_endpoint->encryption() == Endpoint::EncryptionType::SSL) { if (_endpoint->encryption() == Endpoint::EncryptionType::SSL) {
_peer.reset(new SocketSslTcp(_scheduler,
SslServerFeature::SSL->createSslContext())); auto sslContext = SslServerFeature::SSL->createSslContext();
_peer.reset(new SocketSslTcp(context, std::move(sslContext)));
SocketSslTcp* peer = static_cast<SocketSslTcp*>(_peer.get()); SocketSslTcp* peer = static_cast<SocketSslTcp*>(_peer.get());
_acceptor->async_accept(peer->_socket, peer->_peerEndpoint, handler); _acceptor->async_accept(peer->_socket, peer->_peerEndpoint, handler);
} else { } else {
_peer.reset(new SocketTcp(_scheduler)); _peer.reset(new SocketTcp(context));
SocketTcp* peer = static_cast<SocketTcp*>(_peer.get()); SocketTcp* peer = static_cast<SocketTcp*>(_peer.get());
_acceptor->async_accept(*peer->_socket, peer->_peerEndpoint, handler); _acceptor->async_accept(*peer->_socket, peer->_peerEndpoint, handler);
} }

View File

@ -28,8 +28,9 @@
namespace arangodb { namespace arangodb {
class AcceptorTcp final : public Acceptor { class AcceptorTcp final : public Acceptor {
public: public:
AcceptorTcp(rest::Scheduler* scheduler, Endpoint* endpoint) AcceptorTcp(rest::GeneralServer &server,
: Acceptor(scheduler, endpoint), _acceptor(scheduler->newAcceptor()) {} rest::GeneralServer::IoContext &context, Endpoint* endpoint)
: Acceptor(server, context, endpoint), _acceptor(context.newAcceptor()) {}
public: public:
void open() override; void open() override;

View File

@ -52,7 +52,9 @@ void AcceptorUnixDomain::open() {
void AcceptorUnixDomain::asyncAccept(AcceptHandler const& handler) { void AcceptorUnixDomain::asyncAccept(AcceptHandler const& handler) {
TRI_ASSERT(!_peer); TRI_ASSERT(!_peer);
_peer.reset(new SocketUnixDomain(_scheduler)); auto &context = _server.selectIoContext();
_peer.reset(new SocketUnixDomain(context));
auto peer = dynamic_cast<SocketUnixDomain*>(_peer.get()); auto peer = dynamic_cast<SocketUnixDomain*>(_peer.get());
if (peer == nullptr) { if (peer == nullptr) {
THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_INTERNAL, "unexpected socket type"); THROW_ARANGO_EXCEPTION_MESSAGE(TRI_ERROR_INTERNAL, "unexpected socket type");

View File

@ -28,9 +28,10 @@
namespace arangodb { namespace arangodb {
class AcceptorUnixDomain final : public Acceptor { class AcceptorUnixDomain final : public Acceptor {
public: public:
AcceptorUnixDomain(rest::Scheduler* scheduler, Endpoint* endpoint) AcceptorUnixDomain(rest::GeneralServer &server,
: Acceptor(scheduler, endpoint), rest::GeneralServer::IoContext &context, Endpoint* endpoint)
_acceptor(scheduler->newDomainAcceptor()) {} : Acceptor(server, context, endpoint),
_acceptor(context.newDomainAcceptor()) {}
public: public:
void open() override; void open() override;

View File

@ -37,11 +37,11 @@ using namespace arangodb::rest;
// --SECTION-- constructors and destructors // --SECTION-- constructors and destructors
// ----------------------------------------------------------------------------- // -----------------------------------------------------------------------------
ListenTask::ListenTask(Scheduler* scheduler, Endpoint* endpoint) ListenTask::ListenTask(GeneralServer &server, GeneralServer::IoContext& context, Endpoint* endpoint)
: Task(scheduler, "ListenTask"), : IoTask(server, context, "ListenTask"),
_endpoint(endpoint), _endpoint(endpoint),
_bound(false), _bound(false),
_acceptor(Acceptor::factory(scheduler, endpoint)) {} _acceptor(Acceptor::factory(server, context, endpoint)) {}
ListenTask::~ListenTask() {} ListenTask::~ListenTask() {}
@ -50,7 +50,6 @@ ListenTask::~ListenTask() {}
// ----------------------------------------------------------------------------- // -----------------------------------------------------------------------------
bool ListenTask::start() { bool ListenTask::start() {
MUTEX_LOCKER(mutex, _shutdownMutex);
TRI_ASSERT(_acceptor); TRI_ASSERT(_acceptor);
try { try {
@ -67,17 +66,23 @@ bool ListenTask::start() {
return false; return false;
} }
_handler = [this](asio_ns::error_code const& ec) {
MUTEX_LOCKER(mutex, _shutdownMutex); _bound = true;
JobGuard guard(_scheduler); this->accept();
guard.work(); return true;
}
void ListenTask::accept() {
auto self(shared_from_this());
auto handler = [this, self](asio_ns::error_code const& ec) {
if (!_bound) { if (!_bound) {
_handler = nullptr; _handler = nullptr;
return; return;
} }
TRI_ASSERT(_handler != nullptr);
TRI_ASSERT(_acceptor != nullptr); TRI_ASSERT(_acceptor != nullptr);
if (ec) { if (ec) {
@ -115,16 +120,14 @@ bool ListenTask::start() {
handleConnected(std::move(peer), std::move(info)); handleConnected(std::move(peer), std::move(info));
_acceptor->asyncAccept(_handler); this->accept();
}; };
_bound = true; _acceptor->asyncAccept(handler);
_acceptor->asyncAccept(_handler);
return true;
} }
void ListenTask::stop() { void ListenTask::stop() {
MUTEX_LOCKER(mutex, _shutdownMutex);
if (!_bound) { if (!_bound) {
return; return;

View File

@ -25,6 +25,9 @@
#ifndef ARANGOD_SCHEDULER_LISTEN_TASK_H #ifndef ARANGOD_SCHEDULER_LISTEN_TASK_H
#define ARANGOD_SCHEDULER_LISTEN_TASK_H 1 #define ARANGOD_SCHEDULER_LISTEN_TASK_H 1
#include "GeneralServer/GeneralServer.h"
#include "GeneralServer/IoTask.h"
#include "Scheduler/Task.h" #include "Scheduler/Task.h"
#include "Basics/Mutex.h" #include "Basics/Mutex.h"
@ -33,13 +36,17 @@
#include "Scheduler/Acceptor.h" #include "Scheduler/Acceptor.h"
#include "Scheduler/Socket.h" #include "Scheduler/Socket.h"
namespace arangodb { namespace arangodb {
class ListenTask : virtual public rest::Task {
class ListenTask : virtual public rest::IoTask {
public: public:
static size_t const MAX_ACCEPT_ERRORS = 128; static size_t const MAX_ACCEPT_ERRORS = 128;
public: public:
ListenTask(rest::Scheduler*, Endpoint*); ListenTask(rest::GeneralServer &server, rest::GeneralServer::IoContext&, Endpoint*);
~ListenTask(); ~ListenTask();
public: public:
@ -52,10 +59,11 @@ class ListenTask : virtual public rest::Task {
void stop(); void stop();
private: private:
void accept();
Endpoint* _endpoint; Endpoint* _endpoint;
size_t _acceptFailures = 0; size_t _acceptFailures = 0;
Mutex _shutdownMutex;
bool _bound; bool _bound;
std::unique_ptr<Acceptor> _acceptor; std::unique_ptr<Acceptor> _acceptor;
@ -63,4 +71,5 @@ class ListenTask : virtual public rest::Task {
}; };
} }
#endif #endif

View File

@ -351,6 +351,12 @@ bool Scheduler::queue(RequestPriority prio,
break; break;
} }
// THIS IS A UGLY HACK TO SUPPORT THE NEW IO CONTEXT INFRASTRUCTURE
// This is needed, since a post on the scheduler does no longer result in a
// drain immerdiately. The reason for that is, that no worker thread returns
// from `run_once`.
this->drain();
return ok; return ok;
} }

View File

@ -109,6 +109,45 @@ class Scheduler : public std::enable_shared_from_this<Scheduler> {
bool isRunning() const { return numRunning(_counters) > 0; } bool isRunning() const { return numRunning(_counters) > 0; }
bool isStopping() const noexcept { return (_counters & (1ULL << 63)) != 0; } bool isStopping() const noexcept { return (_counters & (1ULL << 63)) != 0; }
public:
template <typename T>
asio_ns::deadline_timer* newDeadlineTimer(T timeout) {
return new asio_ns::deadline_timer(*_ioContext, timeout);
}
asio_ns::steady_timer* newSteadyTimer() {
return new asio_ns::steady_timer(*_ioContext);
}
asio_ns::io_context::strand* newStrand() {
return new asio_ns::io_context::strand(*_ioContext);
}
asio_ns::ip::tcp::acceptor* newAcceptor() {
return new asio_ns::ip::tcp::acceptor(*_ioContext);
}
#ifndef _WIN32
asio_ns::local::stream_protocol::acceptor* newDomainAcceptor() {
return new asio_ns::local::stream_protocol::acceptor(*_ioContext);
}
#endif
asio_ns::ip::tcp::socket* newSocket() {
return new asio_ns::ip::tcp::socket(*_ioContext);
}
#ifndef _WIN32
asio_ns::local::stream_protocol::socket* newDomainSocket() {
return new asio_ns::local::stream_protocol::socket(*_ioContext);
}
#endif
asio_ns::ssl::stream<asio_ns::ip::tcp::socket>* newSslSocket(
asio_ns::ssl::context& context) {
return new asio_ns::ssl::stream<asio_ns::ip::tcp::socket>(*_ioContext,
context);
}
asio_ns::ip::tcp::resolver* newResolver() {
return new asio_ns::ip::tcp::resolver(*_ioContext);
}
asio_ns::signal_set* newSignalSet() {
return new asio_ns::signal_set(*_managerContext);
}
private: private:
inline void setStopping() noexcept { _counters |= (1ULL << 63); } inline void setStopping() noexcept { _counters |= (1ULL << 63); }
@ -209,54 +248,6 @@ class Scheduler : public std::enable_shared_from_this<Scheduler> {
// The `io_context` itself is not exposed because everything // The `io_context` itself is not exposed because everything
// should use the method `post` of the Scheduler. // should use the method `post` of the Scheduler.
public:
template <typename T>
asio_ns::deadline_timer* newDeadlineTimer(T timeout) {
return new asio_ns::deadline_timer(*_ioContext, timeout);
}
asio_ns::steady_timer* newSteadyTimer() {
return new asio_ns::steady_timer(*_ioContext);
}
asio_ns::io_context::strand* newStrand() {
return new asio_ns::io_context::strand(*_ioContext);
}
asio_ns::ip::tcp::acceptor* newAcceptor() {
return new asio_ns::ip::tcp::acceptor(*_ioContext);
}
#ifndef _WIN32
asio_ns::local::stream_protocol::acceptor* newDomainAcceptor() {
return new asio_ns::local::stream_protocol::acceptor(*_ioContext);
}
#endif
asio_ns::ip::tcp::socket* newSocket() {
return new asio_ns::ip::tcp::socket(*_ioContext);
}
#ifndef _WIN32
asio_ns::local::stream_protocol::socket* newDomainSocket() {
return new asio_ns::local::stream_protocol::socket(*_ioContext);
}
#endif
asio_ns::ssl::stream<asio_ns::ip::tcp::socket>* newSslSocket(
asio_ns::ssl::context& context) {
return new asio_ns::ssl::stream<asio_ns::ip::tcp::socket>(*_ioContext,
context);
}
asio_ns::ip::tcp::resolver* newResolver() {
return new asio_ns::ip::tcp::resolver(*_ioContext);
}
asio_ns::signal_set* newSignalSet() {
return new asio_ns::signal_set(*_managerContext);
}
private: private:
static void initializeSignalHandlers(); static void initializeSignalHandlers();

View File

@ -120,7 +120,7 @@ void SchedulerFeature::start() {
if (_nrMaximalThreads > 8 * N) { if (_nrMaximalThreads > 8 * N) {
LOG_TOPIC(WARN, arangodb::Logger::THREADS) LOG_TOPIC(WARN, arangodb::Logger::THREADS)
<< "--server.threads (" << _nrMaximalThreads << "--server.maximal-threads (" << _nrMaximalThreads
<< ") is more than eight times the number of cores (" << N << ") is more than eight times the number of cores (" << N
<< "), this might overload the server"; << "), this might overload the server";
} }
@ -134,7 +134,7 @@ void SchedulerFeature::start() {
if (_nrMinimalThreads >= _nrMaximalThreads) { if (_nrMinimalThreads >= _nrMaximalThreads) {
LOG_TOPIC(WARN, arangodb::Logger::THREADS) LOG_TOPIC(WARN, arangodb::Logger::THREADS)
<< "--server.threads (" << _nrMaximalThreads << ") should be at least " << "--server.maximal-threads (" << _nrMaximalThreads << ") should be at least "
<< (_nrMinimalThreads + 1) << ", raising it"; << (_nrMinimalThreads + 1) << ", raising it";
_nrMaximalThreads = _nrMinimalThreads + 1; _nrMaximalThreads = _nrMinimalThreads + 1;
} }

View File

@ -30,6 +30,8 @@
#include "Logger/Logger.h" #include "Logger/Logger.h"
#include "Scheduler/JobGuard.h" #include "Scheduler/JobGuard.h"
#include "GeneralServer/GeneralServer.h"
namespace arangodb { namespace arangodb {
namespace rest { namespace rest {
class Scheduler; class Scheduler;
@ -41,17 +43,16 @@ typedef std::function<void(const asio_ns::error_code& ec,
class Socket { class Socket {
public: public:
Socket(rest::Scheduler* scheduler, bool encrypted) Socket(rest::GeneralServer::IoContext &context, bool encrypted)
: _strand(scheduler->newStrand()), : _context(context),
_encrypted(encrypted), _encrypted(encrypted) {
_scheduler(scheduler) { _context._clients++;
TRI_ASSERT(_scheduler != nullptr);
} }
Socket(Socket const& that) = delete; Socket(Socket const& that) = delete;
Socket(Socket&& that) = delete; Socket(Socket&& that) = delete;
virtual ~Socket() {} virtual ~Socket() { _context._clients--; }
bool isEncrypted() const { return _encrypted; } bool isEncrypted() const { return _encrypted; }
@ -84,11 +85,11 @@ class Socket {
} }
} }
void post(std::function<void()> handler) { void post(std::function<void()> && handler) {
_scheduler->post(*_strand, handler); _context.post(std::move(handler));
} }
bool runningInThisThread() { return _strand->running_in_this_thread(); } bool runningInThisThread() { return _context.runningInThisThread(); }
public: public:
virtual std::string peerAddress() const = 0; virtual std::string peerAddress() const = 0;
@ -111,13 +112,12 @@ class Socket {
virtual void shutdownSend(asio_ns::error_code& ec) = 0; virtual void shutdownSend(asio_ns::error_code& ec) = 0;
protected: protected:
// strand to ensure the connection's handlers are not called concurrently. rest::GeneralServer::IoContext &_context;
std::unique_ptr<asio_ns::io_context::strand> _strand;
private: private:
bool const _encrypted; bool const _encrypted;
bool _handshakeDone = false; bool _handshakeDone = false;
rest::Scheduler* _scheduler;
}; };
} }

View File

@ -33,10 +33,10 @@ class SocketSslTcp final : public Socket {
friend class AcceptorTcp; friend class AcceptorTcp;
public: public:
SocketSslTcp(rest::Scheduler* scheduler, asio_ns::ssl::context&& context) SocketSslTcp(rest::GeneralServer::IoContext &context, asio_ns::ssl::context&& sslContext)
: Socket(scheduler, /*encrypted*/ true), : Socket(context, /*encrypted*/ true),
_sslContext(std::move(context)), _sslContext(std::move(sslContext)),
_sslSocket(scheduler->newSslSocket(_sslContext)), _sslSocket(context.newSslSocket(_sslContext)),
_socket(_sslSocket->next_layer()), _socket(_sslSocket->next_layer()),
_peerEndpoint() {} _peerEndpoint() {}
@ -61,7 +61,7 @@ class SocketSslTcp final : public Socket {
void asyncWrite(asio_ns::mutable_buffers_1 const& buffer, void asyncWrite(asio_ns::mutable_buffers_1 const& buffer,
AsyncHandler const& handler) override { AsyncHandler const& handler) override {
return asio_ns::async_write(*_sslSocket, buffer, _strand->wrap(handler)); return asio_ns::async_write(*_sslSocket, buffer, handler);
} }
size_t readSome(asio_ns::mutable_buffers_1 const& buffer, size_t readSome(asio_ns::mutable_buffers_1 const& buffer,
@ -71,7 +71,7 @@ class SocketSslTcp final : public Socket {
void asyncRead(asio_ns::mutable_buffers_1 const& buffer, void asyncRead(asio_ns::mutable_buffers_1 const& buffer,
AsyncHandler const& handler) override { AsyncHandler const& handler) override {
return _sslSocket->async_read_some(buffer, _strand->wrap(handler)); return _sslSocket->async_read_some(buffer, handler);
} }
std::size_t available(asio_ns::error_code& ec) override { std::size_t available(asio_ns::error_code& ec) override {

View File

@ -44,11 +44,11 @@ using namespace arangodb::rest;
// --SECTION-- constructors and destructors // --SECTION-- constructors and destructors
// ----------------------------------------------------------------------------- // -----------------------------------------------------------------------------
SocketTask::SocketTask(Scheduler* scheduler, SocketTask::SocketTask(GeneralServer &server, GeneralServer::IoContext &context,
std::unique_ptr<arangodb::Socket> socket, std::unique_ptr<arangodb::Socket> socket,
arangodb::ConnectionInfo&& connectionInfo, arangodb::ConnectionInfo&& connectionInfo,
double keepAliveTimeout, bool skipInit = false) double keepAliveTimeout, bool skipInit = false)
: Task(scheduler, "SocketTask"), : IoTask(server, context, "SocketTask"),
_peer(std::move(socket)), _peer(std::move(socket)),
_connectionInfo(std::move(connectionInfo)), _connectionInfo(std::move(connectionInfo)),
_connectionStatistics(nullptr), _connectionStatistics(nullptr),
@ -56,7 +56,7 @@ SocketTask::SocketTask(Scheduler* scheduler,
_stringBuffers{_stringBuffersArena}, _stringBuffers{_stringBuffersArena},
_writeBuffer(nullptr, nullptr), _writeBuffer(nullptr, nullptr),
_keepAliveTimeout(static_cast<long>(keepAliveTimeout * 1000)), _keepAliveTimeout(static_cast<long>(keepAliveTimeout * 1000)),
_keepAliveTimer(scheduler->newDeadlineTimer(_keepAliveTimeout)), _keepAliveTimer(context.newDeadlineTimer(_keepAliveTimeout)),
_useKeepAliveTimer(keepAliveTimeout > 0.0), _useKeepAliveTimer(keepAliveTimeout > 0.0),
_keepAliveTimerActive(false), _keepAliveTimerActive(false),
_closeRequested(false), _closeRequested(false),
@ -436,8 +436,6 @@ void SocketTask::asyncReadSome() {
_peer->asyncRead( _peer->asyncRead(
asio_ns::buffer(_readBuffer.end(), READ_BLOCK_SIZE), asio_ns::buffer(_readBuffer.end(), READ_BLOCK_SIZE),
[self, this](const asio_ns::error_code& ec, std::size_t transferred) { [self, this](const asio_ns::error_code& ec, std::size_t transferred) {
JobGuard guard(_scheduler);
guard.work();
if (_abandoned.load(std::memory_order_acquire)) { if (_abandoned.load(std::memory_order_acquire)) {
return; return;
@ -535,9 +533,6 @@ void SocketTask::asyncWriteSome() {
_peer->asyncWrite( _peer->asyncWrite(
asio_ns::buffer(_writeBuffer._buffer->begin() + written, total - written), asio_ns::buffer(_writeBuffer._buffer->begin() + written, total - written),
[self, this](const asio_ns::error_code& ec, std::size_t transferred) { [self, this](const asio_ns::error_code& ec, std::size_t transferred) {
JobGuard guard(_scheduler);
guard.work();
if (_abandoned.load(std::memory_order_acquire)) { if (_abandoned.load(std::memory_order_acquire)) {
return; return;
} }

View File

@ -34,11 +34,13 @@
#include "Scheduler/Socket.h" #include "Scheduler/Socket.h"
#include "Statistics/RequestStatistics.h" #include "Statistics/RequestStatistics.h"
#include "GeneralServer/IoTask.h"
namespace arangodb { namespace arangodb {
class ConnectionStatistics; class ConnectionStatistics;
namespace rest { namespace rest {
class SocketTask : virtual public Task { class SocketTask : virtual public IoTask {
friend class HttpCommTask; friend class HttpCommTask;
explicit SocketTask(SocketTask const&) = delete; explicit SocketTask(SocketTask const&) = delete;
@ -48,7 +50,8 @@ class SocketTask : virtual public Task {
static size_t const READ_BLOCK_SIZE = 10000; static size_t const READ_BLOCK_SIZE = 10000;
public: public:
SocketTask(Scheduler*, std::unique_ptr<Socket>, ConnectionInfo&&, SocketTask(GeneralServer &server, GeneralServer::IoContext &context,
std::unique_ptr<Socket>, ConnectionInfo&&,
double keepAliveTimeout, bool skipInit); double keepAliveTimeout, bool skipInit);
virtual ~SocketTask(); virtual ~SocketTask();

View File

@ -33,9 +33,9 @@ class SocketTcp final : public Socket {
friend class AcceptorTcp; friend class AcceptorTcp;
public: public:
SocketTcp(rest::Scheduler* scheduler) SocketTcp(rest::GeneralServer::IoContext &context)
: Socket(scheduler, /*encrypted*/ false), : Socket(context, /*encrypted*/ false),
_socket(scheduler->newSocket()), _socket(context.newSocket()),
_peerEndpoint() {} _peerEndpoint() {}
SocketTcp(SocketTcp const& that) = delete; SocketTcp(SocketTcp const& that) = delete;
@ -58,7 +58,7 @@ class SocketTcp final : public Socket {
void asyncWrite(asio_ns::mutable_buffers_1 const& buffer, void asyncWrite(asio_ns::mutable_buffers_1 const& buffer,
AsyncHandler const& handler) override { AsyncHandler const& handler) override {
return asio_ns::async_write(*_socket, buffer, _strand->wrap(handler)); return asio_ns::async_write(*_socket, buffer, handler);
} }
size_t readSome(asio_ns::mutable_buffers_1 const& buffer, size_t readSome(asio_ns::mutable_buffers_1 const& buffer,
@ -68,7 +68,7 @@ class SocketTcp final : public Socket {
void asyncRead(asio_ns::mutable_buffers_1 const& buffer, void asyncRead(asio_ns::mutable_buffers_1 const& buffer,
AsyncHandler const& handler) override { AsyncHandler const& handler) override {
return _socket->async_read_some(buffer, _strand->wrap(handler)); return _socket->async_read_some(buffer, handler);
} }
void close(asio_ns::error_code& ec) override { void close(asio_ns::error_code& ec) override {

View File

@ -34,7 +34,7 @@ size_t SocketUnixDomain::writeSome(basics::StringBuffer* buffer,
void SocketUnixDomain::asyncWrite(asio_ns::mutable_buffers_1 const& buffer, void SocketUnixDomain::asyncWrite(asio_ns::mutable_buffers_1 const& buffer,
AsyncHandler const& handler) { AsyncHandler const& handler) {
return asio_ns::async_write(*_socket, buffer, _strand->wrap(handler)); return asio_ns::async_write(*_socket, buffer, handler);
} }
size_t SocketUnixDomain::readSome(asio_ns::mutable_buffers_1 const& buffer, size_t SocketUnixDomain::readSome(asio_ns::mutable_buffers_1 const& buffer,
@ -48,7 +48,7 @@ std::size_t SocketUnixDomain::available(asio_ns::error_code& ec) {
void SocketUnixDomain::asyncRead(asio_ns::mutable_buffers_1 const& buffer, void SocketUnixDomain::asyncRead(asio_ns::mutable_buffers_1 const& buffer,
AsyncHandler const& handler) { AsyncHandler const& handler) {
return _socket->async_read_some(buffer, _strand->wrap(handler)); return _socket->async_read_some(buffer, handler);
} }
void SocketUnixDomain::shutdownReceive(asio_ns::error_code& ec) { void SocketUnixDomain::shutdownReceive(asio_ns::error_code& ec) {

View File

@ -36,9 +36,9 @@ class SocketUnixDomain final : public Socket {
friend class AcceptorUnixDomain; friend class AcceptorUnixDomain;
public: public:
explicit SocketUnixDomain(rest::Scheduler* scheduler) explicit SocketUnixDomain(rest::GeneralServer::IoContext &context)
: Socket(scheduler, false), : Socket(context, false),
_socket(scheduler->newDomainSocket()) {} _socket(context.newDomainSocket()) {}
SocketUnixDomain(SocketUnixDomain&& that) = default; SocketUnixDomain(SocketUnixDomain&& that) = default;

View File

@ -144,16 +144,16 @@ UpgradeResult Upgrade::startup(
LOG_TOPIC(ERR, Logger::STARTUP) LOG_TOPIC(ERR, Logger::STARTUP)
<< "It seems like you have upgraded the ArangoDB binary."; << "It seems like you have upgraded the ArangoDB binary.";
LOG_TOPIC(ERR, Logger::STARTUP) LOG_TOPIC(ERR, Logger::STARTUP)
<< "If this is what you wanted to do, please restart with the'"; << "If this is what you wanted to do, please restart with the";
LOG_TOPIC(ERR, Logger::STARTUP) << " --database.auto-upgrade true'"; LOG_TOPIC(ERR, Logger::STARTUP) << " --database.auto-upgrade true";
LOG_TOPIC(ERR, Logger::STARTUP) LOG_TOPIC(ERR, Logger::STARTUP)
<< "option to upgrade the data in the database directory.'"; << "option to upgrade the data in the database directory.";
LOG_TOPIC(ERR, Logger::STARTUP) << "Normally you can use the control " LOG_TOPIC(ERR, Logger::STARTUP) << "Normally you can use the control "
"script to upgrade your database'"; "script to upgrade your database'";
LOG_TOPIC(ERR, Logger::STARTUP) << " /etc/init.d/arangodb stop'"; LOG_TOPIC(ERR, Logger::STARTUP) << " /etc/init.d/arangodb stop";
LOG_TOPIC(ERR, Logger::STARTUP) << " /etc/init.d/arangodb upgrade'"; LOG_TOPIC(ERR, Logger::STARTUP) << " /etc/init.d/arangodb upgrade";
LOG_TOPIC(ERR, Logger::STARTUP) << " /etc/init.d/arangodb start'"; LOG_TOPIC(ERR, Logger::STARTUP) << " /etc/init.d/arangodb start";
LOG_TOPIC(ERR, Logger::STARTUP) LOG_TOPIC(ERR, Logger::STARTUP)
<< "----------------------------------------------------------------------'"; << "----------------------------------------------------------------------'";
return UpgradeResult(TRI_ERROR_BAD_PARAMETER, vinfo.status); return UpgradeResult(TRI_ERROR_BAD_PARAMETER, vinfo.status);

View File

@ -96,6 +96,10 @@ class Thread {
/// @brief flags the thread as stopping /// @brief flags the thread as stopping
virtual void beginShutdown(); virtual void beginShutdown();
bool runningInThisThread() {
return currentThreadNumber() == this->threadNumber();
}
protected: protected:
/// @brief called from the destructor /// @brief called from the destructor
void shutdown(); void shutdown();