Commit a79093b4 authored by zhujiashun's avatar zhujiashun

redis_server_protocol: revise to sync interface

parent 491a1860
...@@ -60,9 +60,7 @@ public: ...@@ -60,9 +60,7 @@ public:
: _rsimpl(rsimpl) {} : _rsimpl(rsimpl) {}
brpc::RedisCommandHandler::Result Run(const char* args[], brpc::RedisCommandHandler::Result Run(const char* args[],
brpc::RedisReply* output, brpc::RedisReply* output) {
google::protobuf::Closure* done) {
brpc::ClosureGuard done_guard(done);
if (args[1] == NULL) { if (args[1] == NULL) {
output->SetError("ERR wrong number of arguments for 'get' command"); output->SetError("ERR wrong number of arguments for 'get' command");
return brpc::RedisCommandHandler::OK; return brpc::RedisCommandHandler::OK;
...@@ -88,14 +86,12 @@ public: ...@@ -88,14 +86,12 @@ public:
: _rsimpl(rsimpl) {} : _rsimpl(rsimpl) {}
brpc::RedisCommandHandler::Result Run(const char* args[], brpc::RedisCommandHandler::Result Run(const char* args[],
brpc::RedisReply* output, brpc::RedisReply* output) {
google::protobuf::Closure* done) { if (args[1] == NULL || args[2] == NULL) {
brpc::ClosureGuard done_guard(done);
std::string key = args[1];
if (args[2] == NULL) {
output->SetError("ERR wrong number of arguments for 'set' command"); output->SetError("ERR wrong number of arguments for 'set' command");
return brpc::RedisCommandHandler::OK; return brpc::RedisCommandHandler::OK;
} }
std::string key = args[1];
_rsimpl->Set(key, args[2]); _rsimpl->Set(key, args[2]);
output->SetStatus("OK"); output->SetStatus("OK");
return brpc::RedisCommandHandler::OK; return brpc::RedisCommandHandler::OK;
......
...@@ -58,47 +58,40 @@ struct InputResponse : public InputMessageBase { ...@@ -58,47 +58,40 @@ struct InputResponse : public InputMessageBase {
} }
}; };
static const char** ParseArgs(const RedisReply& message) { static bool ParseArgs(const RedisReply& message, std::unique_ptr<const char*[]>* args_out) {
if (!message.is_array() || message.size() == 0) { if (!message.is_array() || message.size() == 0) {
LOG(WARNING) << "request message is not array or size equals to zero"; LOG(WARNING) << "request message is not array or size equals to zero";
return NULL; return false;
} }
const char** args = (const char**) args_out->reset(new const char*[message.size() + 1 /* NULL */]);
malloc(sizeof(const char*) * (message.size() + 1 /* NULL */));
for (size_t i = 0; i < message.size(); ++i) { for (size_t i = 0; i < message.size(); ++i) {
if (!message[i].is_string()) { if (!message[i].is_string()) {
LOG(WARNING) << "request message[" << i << "] is not array"; LOG(WARNING) << "request message[" << i << "] is not array";
free(args); return false;
return NULL;
} }
args[i] = message[i].c_str(); (*args_out)[i] = message[i].c_str();
} }
args[message.size()] = NULL; (*args_out)[message.size()] = NULL;
return args; return true;
} }
// One redis command corresponding to one ConsumeTaskDone. Whenever user struct RedisTask {
// has completed the process of handling command and call done->Run() RedisReply input_message;
// (read redis.h for more details), RedisConnContext::Flush() will be butil::Arena arena;
// called and flush the response to client by the order that commands arrive. };
class ConsumeTaskDone;
// This class is as parsing_context in socket. // This class is as parsing_context in socket.
class RedisConnContext : public SharedObject class RedisConnContext : public SharedObject
, public Destroyable { , public Destroyable {
public: public:
RedisConnContext() RedisConnContext()
: handler_continue(NULL) : handler_continue(NULL) {}
, message_count(0) {}
~RedisConnContext(); ~RedisConnContext();
// @Destroyable // @Destroyable
void Destroy(); void Destroy();
int Init(); int Init();
// Push `done` to a queue which is read by Flush().
void Push(ConsumeTaskDone* done);
void Flush();
void ClearSentDones();
SocketId socket_id; SocketId socket_id;
RedisService::CommandMap command_map; RedisService::CommandMap command_map;
...@@ -106,57 +99,23 @@ public: ...@@ -106,57 +99,23 @@ public:
// first handler pointer that triggers the transaction. // first handler pointer that triggers the transaction.
RedisCommandHandler* handler_continue; RedisCommandHandler* handler_continue;
// The redis command are parsed and pushed into this queue // The redis command are parsed and pushed into this queue
bthread::ExecutionQueueId<ConsumeTaskDone*> queue; bthread::ExecutionQueueId<RedisTask*> queue;
RedisReply parsing_message; RedisReply parsing_message;
butil::Arena arena; butil::Arena arena;
int64_t message_count;
private:
void AddSentDone(ConsumeTaskDone* done);
bool _writing = false;
butil::Mutex _mutex;
std::queue<ConsumeTaskDone*> _dones;
butil::Mutex _dones_sent_mutex;
std::queue<ConsumeTaskDone*> _dones_sent;
}; };
class ConsumeTaskDone : public google::protobuf::Closure { int ConsumeTask(RedisConnContext* ctx, RedisTask* task, butil::IOBuf* sendbuf) {
public: RedisReply output(&task->arena);
ConsumeTaskDone() std::unique_ptr<const char*[]> args;
: _ready(false) if (!ParseArgs(task->input_message, &args)) {
, output_message(&arena) LOG(ERROR) << "ERR command not string";
, ctx(NULL) {}
void Run() override;
bool IsReady() { return _ready.load(butil::memory_order_acquire); }
private:
butil::atomic<bool> _ready;
public:
RedisReply input_message;
RedisReply output_message;
RedisConnContext* ctx;
butil::IOBuf sendbuf;
butil::Arena arena;
};
int ConsumeTask(RedisConnContext* ctx, ConsumeTaskDone* done) {
ClosureGuard done_guard(done);
done->ctx = ctx;
ctx->Push(done);
RedisReply& output = done->output_message;
const char** args = ParseArgs(done->input_message);
if (!args) {
output.SetError("ERR command not string"); output.SetError("ERR command not string");
return -1; return -1;
} }
if (ctx->handler_continue) { if (ctx->handler_continue) {
RedisCommandHandler::Result result = ctx->handler_continue->Run( RedisCommandHandler::Result result =
args, &output, done_guard.release()); ctx->handler_continue->Run(args.get(), &output);
if (result == RedisCommandHandler::OK) { if (result == RedisCommandHandler::OK) {
ctx->handler_continue = NULL; ctx->handler_continue = NULL;
} }
...@@ -172,39 +131,50 @@ int ConsumeTask(RedisConnContext* ctx, ConsumeTaskDone* done) { ...@@ -172,39 +131,50 @@ int ConsumeTask(RedisConnContext* ctx, ConsumeTaskDone* done) {
snprintf(buf, sizeof(buf), "ERR unknown command `%s`", comm.c_str()); snprintf(buf, sizeof(buf), "ERR unknown command `%s`", comm.c_str());
output.SetError(buf); output.SetError(buf);
} else { } else {
RedisCommandHandler::Result result = RedisCommandHandler::Result result = it->second->Run(args.get(), &output);
it->second->Run(args, &output, done_guard.release());
if (result == RedisCommandHandler::CONTINUE) { if (result == RedisCommandHandler::CONTINUE) {
ctx->handler_continue = it->second.get(); ctx->handler_continue = it->second.get();
} }
} }
} }
free(args); output.SerializeToIOBuf(sendbuf);
return 0; return 0;
} }
int Consume(void* ctx, bthread::TaskIterator<ConsumeTaskDone*>& iter) { int Consume(void* ctx, bthread::TaskIterator<RedisTask*>& iter) {
RedisConnContext* qctx = static_cast<RedisConnContext*>(ctx); RedisConnContext* qctx = static_cast<RedisConnContext*>(ctx);
if (iter.is_queue_stopped()) { if (iter.is_queue_stopped()) {
qctx->RemoveRefManually(); qctx->RemoveRefManually();
return 0; return 0;
} }
SocketUniquePtr s;
bool has_err = false;
if (Socket::Address(qctx->socket_id, &s) != 0) {
LOG(WARNING) << "Fail to address redis socket";
has_err = true;
}
Socket::WriteOptions wopt;
wopt.ignore_eovercrowded = true;
butil::IOBuf sendbuf;
for (; iter; ++iter) { for (; iter; ++iter) {
ConsumeTask(qctx, *iter); std::unique_ptr<RedisTask> guard(*iter);
if (has_err) {
continue;
}
if (ConsumeTask(qctx, *iter, &sendbuf) != 0) {
has_err = true;
continue;
}
}
if (!has_err) {
LOG_IF(WARNING, s->Write(&sendbuf, &wopt) != 0) << "Fail to send redis reply";
} }
return 0; return 0;
} }
// ========== impl of RedisConnContext ========== // ========== impl of RedisConnContext ==========
RedisConnContext::~RedisConnContext() { RedisConnContext::~RedisConnContext() { }
ClearSentDones();
while (!_dones.empty()) {
ConsumeTaskDone* head = _dones.front();
_dones.pop();
delete head;
}
}
void RedisConnContext::Destroy() { void RedisConnContext::Destroy() {
bthread::execution_queue_stop(queue); bthread::execution_queue_stop(queue);
...@@ -221,88 +191,8 @@ int RedisConnContext::Init() { ...@@ -221,88 +191,8 @@ int RedisConnContext::Init() {
return 0; return 0;
} }
void RedisConnContext::Push(ConsumeTaskDone* done) {
std::unique_lock<butil::Mutex> m(_mutex);
_dones.push(done);
}
void RedisConnContext::Flush() {
SocketUniquePtr s;
if (Socket::Address(socket_id, &s) != 0) {
LOG(WARNING) << "Fail to address redis socket";
return;
}
{
std::unique_lock<butil::Mutex> m(_mutex);
if (_writing) return;
_writing = true;
}
std::queue<ConsumeTaskDone*> ready_to_write;
butil::IOBuf buf;
Socket::WriteOptions wopt;
wopt.ignore_eovercrowded = true;
while (true) {
std::unique_lock<butil::Mutex> m(_mutex);
while (!_dones.empty() && _dones.front()->IsReady()) {
ready_to_write.push(_dones.front());
_dones.pop();
}
if (ready_to_write.empty()) {
_writing = false;
if (!buf.empty()) {
LOG_IF(WARNING, s->Write(&buf, &wopt) != 0)
<< "Fail to send redis reply";
}
break;
}
m.unlock();
while (!ready_to_write.empty()) {
ConsumeTaskDone* head = ready_to_write.front();
ready_to_write.pop();
buf.append(head->sendbuf);
AddSentDone(head);
}
if ((int)buf.size() > FLAGS_redis_batch_flush_max_size) {
// In extreme cases, there are always tasks that are ready in every check
// loop and the buf size continues to grow, then we will never have chance
// to write the buffer. To solve this issue, just add a limit to the maximum
// size of buf.
LOG_IF(WARNING, s->Write(&buf, &wopt) != 0)
<< "Fail to send redis reply";
CHECK(buf.empty());
}
}
}
void RedisConnContext::AddSentDone(ConsumeTaskDone* done) {
std::unique_lock<butil::Mutex> m(_dones_sent_mutex);
_dones_sent.push(done);
}
void RedisConnContext::ClearSentDones() {
std::queue<ConsumeTaskDone*> dones_sent;
{
std::unique_lock<butil::Mutex> m(_dones_sent_mutex);
_dones_sent.swap(dones_sent);
}
while (!dones_sent.empty()) {
ConsumeTaskDone* head = dones_sent.front();
dones_sent.pop();
delete head;
}
}
// ========== impl of RedisConnContext ========== // ========== impl of RedisConnContext ==========
void ConsumeTaskDone::Run() {
butil::intrusive_ptr<RedisConnContext> delete_ctx(ctx, false);
output_message.SerializeToIOBuf(&sendbuf);
_ready.store(true, butil::memory_order_release);
ctx->Flush();
// After Flush(), this object may be deleted and should never be
// touched.
}
ParseResult ParseRedisMessage(butil::IOBuf* source, Socket* socket, ParseResult ParseRedisMessage(butil::IOBuf* source, Socket* socket,
bool read_eof, const void* arg) { bool read_eof, const void* arg) {
if (read_eof || source->empty()) { if (read_eof || source->empty()) {
...@@ -332,19 +222,15 @@ ParseResult ParseRedisMessage(butil::IOBuf* source, Socket* socket, ...@@ -332,19 +222,15 @@ ParseResult ParseRedisMessage(butil::IOBuf* source, Socket* socket,
if (err != PARSE_OK) { if (err != PARSE_OK) {
return MakeParseError(err); return MakeParseError(err);
} }
std::unique_ptr<ConsumeTaskDone> done(new ConsumeTaskDone); std::unique_ptr<RedisTask> task(new RedisTask);
done->input_message.CopyFromDifferentArena(ctx->parsing_message, &done->arena); task->input_message.CopyFromDifferentArena(ctx->parsing_message, &task->arena);
ctx->parsing_message.Clear(); ctx->parsing_message.Clear();
ctx->arena.clear(); ctx->arena.clear();
// Add a ref that removed in ConsumeTaskDone::Run if (bthread::execution_queue_execute(ctx->queue, task.get()) != 0) {
ctx->AddRefManually();
if (bthread::execution_queue_execute(ctx->queue, done.get()) != 0) {
ctx->RemoveRefManually();
LOG(ERROR) << "Fail to push execution queue"; LOG(ERROR) << "Fail to push execution queue";
return MakeParseError(PARSE_ERROR_NO_RESOURCE); return MakeParseError(PARSE_ERROR_NO_RESOURCE);
} }
ctx->ClearSentDones(); task.release();
done.release();
return MakeMessage(NULL); return MakeMessage(NULL);
} else { } else {
// NOTE(gejun): PopPipelinedInfo() is actually more contended than what // NOTE(gejun): PopPipelinedInfo() is actually more contended than what
......
...@@ -248,19 +248,17 @@ public: ...@@ -248,19 +248,17 @@ public:
// args[2] == "bar" and args[3] == nullptr. // args[2] == "bar" and args[3] == nullptr.
// `output`, which should be filled by user, is the content that sent to client side. // `output`, which should be filled by user, is the content that sent to client side.
// Read brpc/src/redis_reply.h for more usage. // Read brpc/src/redis_reply.h for more usage.
// Remember to call `done->Run()` when everything is set up into `output`. The return // The return value should be RedisCommandHandler::OK for normal cases. If you want
// value should be RedisCommandHandler::OK for normal cases. If you want to implement // to implement transaction, return RedisCommandHandler::CONTINUE until server receives
// transaction, return RedisCommandHandler::CONTINUE until server receives an ending // an ending marker. The first handler that return RedisCommandHandler::CONTINUE will
// marker. The first handler that return RedisCommandHandler::CONTINUE will continue // continue receiving the following commands until it receives an ending marker and
// receiving the following commands until it receives a ending marker and return // return RedisCommandHandler::OK to end transaction. For example, the return value
// RedisCommandHandler::OK to end transaction. For example, the return value of // of commands "multi; set k1 v1; set k2 v2; set k3 v3; exec" should be four
// commands "multi; set k1 v1; set k2 v2; set k3 v3; exec" should be four
// RedisCommandHandler::CONTINUE and one RedisCommandHandler::OK since exec is the // RedisCommandHandler::CONTINUE and one RedisCommandHandler::OK since exec is the
// marker that ends the transaction. User may queue the commands and execute them // marker that ends the transaction. User should queue the commands and execute them
// all once an ending marker is received. // all once the ending marker is received.
virtual RedisCommandHandler::Result Run(const char* args[], virtual RedisCommandHandler::Result Run(const char* args[],
RedisReply* output, RedisReply* output) = 0;
google::protobuf::Closure* done) = 0;
// Whenever a tcp connection is established, a bunch of new handlers would be created // Whenever a tcp connection is established, a bunch of new handlers would be created
// using New() of the corresponding handler and brpc makes sure that all requests from // using New() of the corresponding handler and brpc makes sure that all requests from
......
...@@ -665,60 +665,31 @@ butil::Mutex s_mutex; ...@@ -665,60 +665,31 @@ butil::Mutex s_mutex;
std::unordered_map<std::string, std::string> m; std::unordered_map<std::string, std::string> m;
std::unordered_map<std::string, int64_t> int_map; std::unordered_map<std::string, int64_t> int_map;
struct SleepArgs {
int sleep_ms;
google::protobuf::Closure* done;
};
void* sleep(void *arg) {
SleepArgs* args = static_cast<SleepArgs*>(arg);
bthread_usleep(args->sleep_ms * 1000);
args->done->Run();
delete args;
return NULL;
}
class SetCommandHandler : public brpc::RedisCommandHandler { class SetCommandHandler : public brpc::RedisCommandHandler {
public: public:
SetCommandHandler(bool sleep = false) SetCommandHandler() {}
: _sleep(sleep) {}
brpc::RedisCommandHandler::Result Run(const char* args[], brpc::RedisCommandHandler::Result Run(const char* args[],
brpc::RedisReply* output, brpc::RedisReply* output) {
google::protobuf::Closure* done) {
brpc::ClosureGuard done_guard(done);
std::string key = args[1]; std::string key = args[1];
std::string value = args[2]; std::string value = args[2];
m[key] = value; m[key] = value;
output->SetStatus("OK"); output->SetStatus("OK");
if (_sleep) {
SleepArgs *args = new SleepArgs;
args->sleep_ms = _sleep_ms;
args->done = done_guard.release();
bthread_t bth;
EXPECT_EQ(0, bthread_start_background(&bth, NULL, sleep, args));
if (_sleep_ms > 20) _sleep_ms -= 20;
}
return brpc::RedisCommandHandler::OK; return brpc::RedisCommandHandler::OK;
} }
RedisCommandHandler* New() { _new_count++; return new SetCommandHandler(_sleep); } RedisCommandHandler* New() { _new_count++; return new SetCommandHandler(); }
int new_count() { return _new_count; } int new_count() { return _new_count; }
private: private:
int _sleep_ms = 100;
int _new_count = 0; int _new_count = 0;
bool _sleep = false;
}; };
class GetCommandHandler : public brpc::RedisCommandHandler { class GetCommandHandler : public brpc::RedisCommandHandler {
public: public:
GetCommandHandler(bool sleep = false) GetCommandHandler() {}
: _sleep(sleep) {}
brpc::RedisCommandHandler::Result Run(const char* args[], brpc::RedisCommandHandler::Result Run(const char* args[],
brpc::RedisReply* output, brpc::RedisReply* output) {
google::protobuf::Closure* done) {
brpc::ClosureGuard done_guard(done);
std::string key = args[1]; std::string key = args[1];
auto it = m.find(key); auto it = m.find(key);
if (it != m.end()) { if (it != m.end()) {
...@@ -726,56 +697,33 @@ public: ...@@ -726,56 +697,33 @@ public:
} else { } else {
output->SetNilString(); output->SetNilString();
} }
if (_sleep) {
SleepArgs *args = new SleepArgs;
args->sleep_ms = _sleep_ms;
args->done = done_guard.release();
bthread_t bth;
EXPECT_EQ(0, bthread_start_background(&bth, NULL, sleep, args));
if (_sleep_ms > 20) _sleep_ms -= 20;
}
return brpc::RedisCommandHandler::OK; return brpc::RedisCommandHandler::OK;
} }
RedisCommandHandler* New() { _new_count++; return new GetCommandHandler(_sleep); } RedisCommandHandler* New() { _new_count++; return new GetCommandHandler(); }
int new_count() { return _new_count; } int new_count() { return _new_count; }
private: private:
int _sleep_ms = 100;
int _new_count = 0; int _new_count = 0;
bool _sleep = false;
}; };
class IncrCommandHandler : public brpc::RedisCommandHandler { class IncrCommandHandler : public brpc::RedisCommandHandler {
public: public:
IncrCommandHandler(bool sleep = false) IncrCommandHandler() {}
: _sleep(sleep) {}
brpc::RedisCommandHandler::Result Run(const char* args[], brpc::RedisCommandHandler::Result Run(const char* args[],
brpc::RedisReply* output, brpc::RedisReply* output) {
google::protobuf::Closure* done) {
brpc::ClosureGuard done_guard(done);
int64_t value; int64_t value;
s_mutex.lock(); s_mutex.lock();
value = ++int_map[args[1]]; value = ++int_map[args[1]];
s_mutex.unlock(); s_mutex.unlock();
output->SetInteger(value); output->SetInteger(value);
if (_sleep) {
SleepArgs *args = new SleepArgs;
args->sleep_ms = _sleep_ms;
args->done = done_guard.release();
bthread_t bth;
EXPECT_EQ(0, bthread_start_background(&bth, NULL, sleep, args));
if (_sleep_ms > 20) _sleep_ms -= 20;
}
return brpc::RedisCommandHandler::OK; return brpc::RedisCommandHandler::OK;
} }
RedisCommandHandler* New() { _new_count++; return new IncrCommandHandler(_sleep); } RedisCommandHandler* New() { _new_count++; return new IncrCommandHandler(); }
int new_count() { return _new_count; } int new_count() { return _new_count; }
private: private:
int _sleep_ms = 100;
int _new_count = 0; int _new_count = 0;
bool _sleep = false;
}; };
class RedisServiceImpl : public brpc::RedisService { }; class RedisServiceImpl : public brpc::RedisService { };
...@@ -784,9 +732,9 @@ TEST_F(RedisTest, server_sanity) { ...@@ -784,9 +732,9 @@ TEST_F(RedisTest, server_sanity) {
brpc::Server server; brpc::Server server;
brpc::ServerOptions server_options; brpc::ServerOptions server_options;
RedisServiceImpl* rsimpl = new RedisServiceImpl; RedisServiceImpl* rsimpl = new RedisServiceImpl;
GetCommandHandler *gh = new GetCommandHandler(true); GetCommandHandler *gh = new GetCommandHandler;
SetCommandHandler *sh = new SetCommandHandler(true); SetCommandHandler *sh = new SetCommandHandler;
IncrCommandHandler *ih = new IncrCommandHandler(true); IncrCommandHandler *ih = new IncrCommandHandler;
rsimpl->AddCommandHandler("get", gh); rsimpl->AddCommandHandler("get", gh);
rsimpl->AddCommandHandler("set", sh); rsimpl->AddCommandHandler("set", sh);
rsimpl->AddCommandHandler("incr", ih); rsimpl->AddCommandHandler("incr", ih);
...@@ -884,9 +832,7 @@ public: ...@@ -884,9 +832,7 @@ public:
: _started(false) {} : _started(false) {}
RedisCommandHandler::Result Run(const char* args[], RedisCommandHandler::Result Run(const char* args[],
brpc::RedisReply* output, brpc::RedisReply* output) {
google::protobuf::Closure* done) {
brpc::ClosureGuard done_guard(done);
if (strcasecmp(args[0], "multi") == 0) { if (strcasecmp(args[0], "multi") == 0) {
if (!_started) { if (!_started) {
output->SetStatus("OK"); output->SetStatus("OK");
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment