aboutsummaryrefslogtreecommitdiffhomepage
path: root/src/cpp/server/server_cc.cc
diff options
context:
space:
mode:
Diffstat (limited to 'src/cpp/server/server_cc.cc')
-rw-r--r--src/cpp/server/server_cc.cc343
1 files changed, 194 insertions, 149 deletions
diff --git a/src/cpp/server/server_cc.cc b/src/cpp/server/server_cc.cc
index 3f89275370..817d85a81c 100644
--- a/src/cpp/server/server_cc.cc
+++ b/src/cpp/server/server_cc.cc
@@ -1,5 +1,4 @@
/*
- *
* Copyright 2015, Google Inc.
* All rights reserved.
*
@@ -52,15 +51,15 @@
#include <grpc/support/log.h>
#include "src/core/lib/profiling/timers.h"
-#include "src/cpp/server/thread_pool_interface.h"
+#include "src/cpp/thread_manager/thread_manager.h"
namespace grpc {
-class DefaultGlobalCallbacks GRPC_FINAL : public Server::GlobalCallbacks {
+class DefaultGlobalCallbacks final : public Server::GlobalCallbacks {
public:
- ~DefaultGlobalCallbacks() GRPC_OVERRIDE {}
- void PreSynchronousRequest(ServerContext* context) GRPC_OVERRIDE {}
- void PostSynchronousRequest(ServerContext* context) GRPC_OVERRIDE {}
+ ~DefaultGlobalCallbacks() override {}
+ void PreSynchronousRequest(ServerContext* context) override {}
+ void PostSynchronousRequest(ServerContext* context) override {}
};
static std::shared_ptr<Server::GlobalCallbacks> g_callbacks = nullptr;
@@ -80,7 +79,7 @@ class Server::UnimplementedAsyncRequestContext {
GenericServerAsyncReaderWriter generic_stream_;
};
-class Server::UnimplementedAsyncRequest GRPC_FINAL
+class Server::UnimplementedAsyncRequest final
: public UnimplementedAsyncRequestContext,
public GenericAsyncRequest {
public:
@@ -90,7 +89,7 @@ class Server::UnimplementedAsyncRequest GRPC_FINAL
server_(server),
cq_(cq) {}
- bool FinalizeResult(void** tag, bool* status) GRPC_OVERRIDE;
+ bool FinalizeResult(void** tag, bool* status) override;
ServerContext* context() { return &server_context_; }
GenericServerAsyncReaderWriter* stream() { return &generic_stream_; }
@@ -102,13 +101,13 @@ class Server::UnimplementedAsyncRequest GRPC_FINAL
typedef SneakyCallOpSet<CallOpSendInitialMetadata, CallOpServerSendStatus>
UnimplementedAsyncResponseOp;
-class Server::UnimplementedAsyncResponse GRPC_FINAL
+class Server::UnimplementedAsyncResponse final
: public UnimplementedAsyncResponseOp {
public:
UnimplementedAsyncResponse(UnimplementedAsyncRequest* request);
~UnimplementedAsyncResponse() { delete request_; }
- bool FinalizeResult(void** tag, bool* status) GRPC_OVERRIDE {
+ bool FinalizeResult(void** tag, bool* status) override {
bool r = UnimplementedAsyncResponseOp::FinalizeResult(tag, status);
delete this;
return r;
@@ -118,15 +117,12 @@ class Server::UnimplementedAsyncResponse GRPC_FINAL
UnimplementedAsyncRequest* const request_;
};
-class Server::ShutdownRequest GRPC_FINAL : public CompletionQueueTag {
+class ShutdownTag : public CompletionQueueTag {
public:
- bool FinalizeResult(void** tag, bool* status) {
- delete this;
- return false;
- }
+ bool FinalizeResult(void** tag, bool* status) { return false; }
};
-class Server::SyncRequest GRPC_FINAL : public CompletionQueueTag {
+class Server::SyncRequest final : public CompletionQueueTag {
public:
SyncRequest(RpcServiceMethod* method, void* tag)
: method_(method),
@@ -147,36 +143,6 @@ class Server::SyncRequest GRPC_FINAL : public CompletionQueueTag {
grpc_metadata_array_destroy(&request_metadata_);
}
- static SyncRequest* Wait(CompletionQueue* cq, bool* ok) {
- void* tag = nullptr;
- *ok = false;
- if (!cq->Next(&tag, ok)) {
- return nullptr;
- }
- auto* mrd = static_cast<SyncRequest*>(tag);
- GPR_ASSERT(mrd->in_flight_);
- return mrd;
- }
-
- static bool AsyncWait(CompletionQueue* cq, SyncRequest** req, bool* ok,
- gpr_timespec deadline) {
- void* tag = nullptr;
- *ok = false;
- switch (cq->AsyncNext(&tag, ok, deadline)) {
- case CompletionQueue::TIMEOUT:
- *req = nullptr;
- return true;
- case CompletionQueue::SHUTDOWN:
- *req = nullptr;
- return false;
- case CompletionQueue::GOT_EVENT:
- *req = static_cast<SyncRequest*>(tag);
- GPR_ASSERT((*req)->in_flight_);
- return true;
- }
- GPR_UNREACHABLE_CODE(return false);
- }
-
void SetupRequest() { cq_ = grpc_completion_queue_create(nullptr); }
void TeardownRequest() {
@@ -204,7 +170,7 @@ class Server::SyncRequest GRPC_FINAL : public CompletionQueueTag {
}
}
- bool FinalizeResult(void** tag, bool* status) GRPC_OVERRIDE {
+ bool FinalizeResult(void** tag, bool* status) override {
if (!*status) {
grpc_completion_queue_destroy(cq_);
}
@@ -216,7 +182,7 @@ class Server::SyncRequest GRPC_FINAL : public CompletionQueueTag {
return true;
}
- class CallData GRPC_FINAL {
+ class CallData final {
public:
explicit CallData(Server* server, SyncRequest* mrd)
: cq_(mrd->cq_),
@@ -266,7 +232,6 @@ class Server::SyncRequest GRPC_FINAL : public CompletionQueueTag {
void* const tag_;
bool in_flight_;
const bool has_request_payload_;
- uint32_t incoming_flags_;
grpc_call* call_;
grpc_call_details* call_details_;
gpr_timespec deadline_;
@@ -275,53 +240,158 @@ class Server::SyncRequest GRPC_FINAL : public CompletionQueueTag {
grpc_completion_queue* cq_;
};
+// Implementation of ThreadManager. Each instance of SyncRequestThreadManager
+// manages a pool of threads that poll for incoming Sync RPCs and call the
+// appropriate RPC handlers
+class Server::SyncRequestThreadManager : public ThreadManager {
+ public:
+ SyncRequestThreadManager(Server* server, CompletionQueue* server_cq,
+ std::shared_ptr<GlobalCallbacks> global_callbacks,
+ int min_pollers, int max_pollers,
+ int cq_timeout_msec)
+ : ThreadManager(min_pollers, max_pollers),
+ server_(server),
+ server_cq_(server_cq),
+ cq_timeout_msec_(cq_timeout_msec),
+ global_callbacks_(global_callbacks) {}
+
+ WorkStatus PollForWork(void** tag, bool* ok) override {
+ *tag = nullptr;
+ gpr_timespec deadline =
+ gpr_time_from_millis(cq_timeout_msec_, GPR_TIMESPAN);
+
+ switch (server_cq_->AsyncNext(tag, ok, deadline)) {
+ case CompletionQueue::TIMEOUT:
+ return TIMEOUT;
+ case CompletionQueue::SHUTDOWN:
+ return SHUTDOWN;
+ case CompletionQueue::GOT_EVENT:
+ return WORK_FOUND;
+ }
+
+ GPR_UNREACHABLE_CODE(return TIMEOUT);
+ }
+
+ void DoWork(void* tag, bool ok) override {
+ SyncRequest* sync_req = static_cast<SyncRequest*>(tag);
+
+ if (!sync_req) {
+ // No tag. Nothing to work on. This is an unlikley scenario and possibly a
+ // bug in RPC Manager implementation.
+ gpr_log(GPR_ERROR, "Sync server. DoWork() was called with NULL tag");
+ return;
+ }
+
+ if (ok) {
+ // Calldata takes ownership of the completion queue inside sync_req
+ SyncRequest::CallData cd(server_, sync_req);
+ {
+ // Prepare for the next request
+ if (!IsShutdown()) {
+ sync_req->SetupRequest(); // Create new completion queue for sync_req
+ sync_req->Request(server_->c_server(), server_cq_->cq());
+ }
+ }
+
+ GPR_TIMER_SCOPE("cd.Run()", 0);
+ cd.Run(global_callbacks_);
+ }
+ // TODO (sreek) If ok is false here (which it isn't in case of
+ // grpc_request_registered_call), we should still re-queue the request
+ // object
+ }
+
+ void AddSyncMethod(RpcServiceMethod* method, void* tag) {
+ sync_requests_.emplace_back(new SyncRequest(method, tag));
+ }
+
+ void AddUnknownSyncMethod() {
+ if (!sync_requests_.empty()) {
+ unknown_method_.reset(new RpcServiceMethod(
+ "unknown", RpcMethod::BIDI_STREAMING, new UnknownMethodHandler));
+ sync_requests_.emplace_back(
+ new SyncRequest(unknown_method_.get(), nullptr));
+ }
+ }
+
+ void ShutdownAndDrainCompletionQueue() {
+ server_cq_->Shutdown();
+
+ // Drain any pending items from the queue
+ void* tag;
+ bool ok;
+ while (server_cq_->Next(&tag, &ok)) {
+ // Nothing to be done here
+ }
+ }
+
+ void Start() {
+ if (!sync_requests_.empty()) {
+ for (auto m = sync_requests_.begin(); m != sync_requests_.end(); m++) {
+ (*m)->SetupRequest();
+ (*m)->Request(server_->c_server(), server_cq_->cq());
+ }
+
+ Initialize(); // ThreadManager's Initialize()
+ }
+ }
+
+ private:
+ Server* server_;
+ CompletionQueue* server_cq_;
+ int cq_timeout_msec_;
+ std::vector<std::unique_ptr<SyncRequest>> sync_requests_;
+ std::unique_ptr<RpcServiceMethod> unknown_method_;
+ std::shared_ptr<Server::GlobalCallbacks> global_callbacks_;
+};
+
static internal::GrpcLibraryInitializer g_gli_initializer;
-Server::Server(ThreadPoolInterface* thread_pool, bool thread_pool_owned,
- int max_receive_message_size, ChannelArguments* args)
+Server::Server(
+ int max_receive_message_size, ChannelArguments* args,
+ std::shared_ptr<std::vector<std::unique_ptr<ServerCompletionQueue>>>
+ sync_server_cqs,
+ int min_pollers, int max_pollers, int sync_cq_timeout_msec)
: max_receive_message_size_(max_receive_message_size),
+ sync_server_cqs_(sync_server_cqs),
started_(false),
shutdown_(false),
shutdown_notified_(false),
- num_running_cb_(0),
- sync_methods_(new std::list<SyncRequest>),
has_generic_service_(false),
server_(nullptr),
- thread_pool_(thread_pool),
- thread_pool_owned_(thread_pool_owned),
server_initializer_(new ServerInitializer(this)) {
g_gli_initializer.summon();
gpr_once_init(&g_once_init_callbacks, InitGlobalCallbacks);
global_callbacks_ = g_callbacks;
global_callbacks_->UpdateArguments(args);
+
+ for (auto it = sync_server_cqs_->begin(); it != sync_server_cqs_->end();
+ it++) {
+ sync_req_mgrs_.emplace_back(new SyncRequestThreadManager(
+ this, (*it).get(), global_callbacks_, min_pollers, max_pollers,
+ sync_cq_timeout_msec));
+ }
+
grpc_channel_args channel_args;
args->SetChannelArgs(&channel_args);
+
server_ = grpc_server_create(&channel_args, nullptr);
- if (thread_pool_ == nullptr) {
- grpc_server_register_non_listening_completion_queue(server_, cq_.cq(),
- nullptr);
- } else {
- grpc_server_register_completion_queue(server_, cq_.cq(), nullptr);
- }
}
Server::~Server() {
{
- grpc::unique_lock<grpc::mutex> lock(mu_);
+ std::unique_lock<std::mutex> lock(mu_);
if (started_ && !shutdown_) {
lock.unlock();
Shutdown();
} else if (!started_) {
- cq_.Shutdown();
+ // Shutdown the completion queues
+ for (auto it = sync_req_mgrs_.begin(); it != sync_req_mgrs_.end(); it++) {
+ (*it)->ShutdownAndDrainCompletionQueue();
+ }
}
}
- void* got_tag;
- bool ok;
- GPR_ASSERT(!cq_.Next(&got_tag, &ok));
+
grpc_server_destroy(server_);
- if (thread_pool_owned_) {
- delete thread_pool_;
- }
- delete sync_methods_;
}
void Server::SetGlobalCallbacks(GlobalCallbacks* callbacks) {
@@ -352,12 +422,14 @@ bool Server::RegisterService(const grpc::string* host, Service* service) {
"Can only register an asynchronous service against one server.");
service->server_ = this;
}
+
const char* method_name = nullptr;
for (auto it = service->methods_.begin(); it != service->methods_.end();
++it) {
if (it->get() == nullptr) { // Handled by generic service if any.
continue;
}
+
RpcServiceMethod* method = it->get();
void* tag = grpc_server_register_method(
server_, method->name(), host ? host->c_str() : nullptr,
@@ -367,11 +439,15 @@ bool Server::RegisterService(const grpc::string* host, Service* service) {
method->name());
return false;
}
- if (method->handler() == nullptr) {
+
+ if (method->handler() == nullptr) { // Async method
method->set_server_tag(tag);
} else {
- sync_methods_->emplace_back(method, tag);
+ for (auto it = sync_req_mgrs_.begin(); it != sync_req_mgrs_.end(); it++) {
+ (*it)->AddSyncMethod(method, tag);
+ }
}
+
method_name = method->name();
}
@@ -406,60 +482,65 @@ bool Server::Start(ServerCompletionQueue** cqs, size_t num_cqs) {
grpc_server_start(server_);
if (!has_generic_service_) {
- if (!sync_methods_->empty()) {
- unknown_method_.reset(new RpcServiceMethod(
- "unknown", RpcMethod::BIDI_STREAMING, new UnknownMethodHandler));
- // Use of emplace_back with just constructor arguments is not accepted
- // here by gcc-4.4 because it can't match the anonymous nullptr with a
- // proper constructor implicitly. Construct the object and use push_back.
- sync_methods_->push_back(SyncRequest(unknown_method_.get(), nullptr));
+ for (auto it = sync_req_mgrs_.begin(); it != sync_req_mgrs_.end(); it++) {
+ (*it)->AddUnknownSyncMethod();
}
+
for (size_t i = 0; i < num_cqs; i++) {
if (cqs[i]->IsFrequentlyPolled()) {
new UnimplementedAsyncRequest(this, cqs[i]);
}
}
}
- // Start processing rpcs.
- if (!sync_methods_->empty()) {
- for (auto m = sync_methods_->begin(); m != sync_methods_->end(); m++) {
- m->SetupRequest();
- m->Request(server_, cq_.cq());
- }
- ScheduleCallback();
+ for (auto it = sync_req_mgrs_.begin(); it != sync_req_mgrs_.end(); it++) {
+ (*it)->Start();
}
return true;
}
void Server::ShutdownInternal(gpr_timespec deadline) {
- grpc::unique_lock<grpc::mutex> lock(mu_);
+ std::unique_lock<std::mutex> lock(mu_);
if (started_ && !shutdown_) {
shutdown_ = true;
- grpc_server_shutdown_and_notify(server_, cq_.cq(), new ShutdownRequest());
- cq_.Shutdown();
- lock.unlock();
- // Spin, eating requests until the completion queue is completely shutdown.
- // If the deadline expires then cancel anything that's pending and keep
- // spinning forever until the work is actually drained.
- // Since nothing else needs to touch state guarded by mu_, holding it
- // through this loop is fine.
- SyncRequest* request;
+
+ /// The completion queue to use for server shutdown completion notification
+ CompletionQueue shutdown_cq;
+ ShutdownTag shutdown_tag; // Dummy shutdown tag
+ grpc_server_shutdown_and_notify(server_, shutdown_cq.cq(), &shutdown_tag);
+
+ shutdown_cq.Shutdown();
+
+ void* tag;
bool ok;
- while (SyncRequest::AsyncWait(&cq_, &request, &ok, deadline)) {
- if (request == NULL) { // deadline expired
- grpc_server_cancel_all_calls(server_);
- deadline = gpr_inf_future(GPR_CLOCK_MONOTONIC);
- } else if (ok) {
- SyncRequest::CallData call_data(this, request);
- }
+ CompletionQueue::NextStatus status =
+ shutdown_cq.AsyncNext(&tag, &ok, deadline);
+
+ // If this timed out, it means we are done with the grace period for a clean
+ // shutdown. We should force a shutdown now by cancelling all inflight calls
+ if (status == CompletionQueue::NextStatus::TIMEOUT) {
+ grpc_server_cancel_all_calls(server_);
+ }
+ // Else in case of SHUTDOWN or GOT_EVENT, it means that the server has
+ // successfully shutdown
+
+ // Shutdown all ThreadManagers. This will try to gracefully stop all the
+ // threads in the ThreadManagers (once they process any inflight requests)
+ for (auto it = sync_req_mgrs_.begin(); it != sync_req_mgrs_.end(); it++) {
+ (*it)->Shutdown(); // ThreadManager's Shutdown()
}
- lock.lock();
- // Wait for running callbacks to finish.
- while (num_running_cb_ != 0) {
- callback_cv_.wait(lock);
+ // Wait for threads in all ThreadManagers to terminate
+ for (auto it = sync_req_mgrs_.begin(); it != sync_req_mgrs_.end(); it++) {
+ (*it)->Wait();
+ (*it)->ShutdownAndDrainCompletionQueue();
+ }
+
+ // Drain the shutdown queue (if the previous call to AsyncNext() timed out
+ // and we didn't remove the tag from the queue yet)
+ while (shutdown_cq.Next(&tag, &ok)) {
+ // Nothing to be done here. Just ignore ok and tag values
}
shutdown_notified_ = true;
@@ -468,7 +549,7 @@ void Server::ShutdownInternal(gpr_timespec deadline) {
}
void Server::Wait() {
- grpc::unique_lock<grpc::mutex> lock(mu_);
+ std::unique_lock<std::mutex> lock(mu_);
while (started_ && !shutdown_notified_) {
shutdown_cv_.wait(lock);
}
@@ -494,9 +575,14 @@ ServerInterface::BaseAsyncRequest::BaseAsyncRequest(
tag_(tag),
delete_on_finalize_(delete_on_finalize),
call_(nullptr) {
+ call_cq_->RegisterAvalanching(); // This op will trigger more ops
memset(&initial_metadata_array_, 0, sizeof(initial_metadata_array_));
}
+ServerInterface::BaseAsyncRequest::~BaseAsyncRequest() {
+ call_cq_->CompleteAvalanching();
+}
+
bool ServerInterface::BaseAsyncRequest::FinalizeResult(void** tag,
bool* status) {
if (*status) {
@@ -585,47 +671,6 @@ Server::UnimplementedAsyncResponse::UnimplementedAsyncResponse(
request_->stream()->call_.PerformOps(this);
}
-void Server::ScheduleCallback() {
- {
- grpc::unique_lock<grpc::mutex> lock(mu_);
- num_running_cb_++;
- }
- thread_pool_->Add(std::bind(&Server::RunRpc, this));
-}
-
-void Server::RunRpc() {
- // Wait for one more incoming rpc.
- bool ok;
- GPR_TIMER_SCOPE("Server::RunRpc", 0);
- auto* mrd = SyncRequest::Wait(&cq_, &ok);
- if (mrd) {
- ScheduleCallback();
- if (ok) {
- SyncRequest::CallData cd(this, mrd);
- {
- mrd->SetupRequest();
- grpc::unique_lock<grpc::mutex> lock(mu_);
- if (!shutdown_) {
- mrd->Request(server_, cq_.cq());
- } else {
- // destroy the structure that was created
- mrd->TeardownRequest();
- }
- }
- GPR_TIMER_SCOPE("cd.Run()", 0);
- cd.Run(global_callbacks_);
- }
- }
-
- {
- grpc::unique_lock<grpc::mutex> lock(mu_);
- num_running_cb_--;
- if (shutdown_) {
- callback_cv_.notify_all();
- }
- }
-}
-
ServerInitializer* Server::initializer() { return server_initializer_.get(); }
} // namespace grpc