Factor out client connections module into a separate library called facade
This commit is contained in:
parent
28a2db1044
commit
3f0fcbf99f
|
@ -1,3 +1,4 @@
|
||||||
add_subdirectory(redis)
|
add_subdirectory(redis)
|
||||||
add_subdirectory(core)
|
add_subdirectory(core)
|
||||||
|
add_subdirectory(facade)
|
||||||
add_subdirectory(server)
|
add_subdirectory(server)
|
||||||
|
|
|
@ -4,28 +4,11 @@
|
||||||
|
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
#include "core/intent_lock.h"
|
#include "core/intent_lock.h"
|
||||||
#include "core/resp_expr.h"
|
|
||||||
|
#include <absl/base/macros.h>
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
const char* RespExpr::TypeName(Type t) {
|
|
||||||
switch (t) {
|
|
||||||
case STRING:
|
|
||||||
return "string";
|
|
||||||
case INT64:
|
|
||||||
return "int";
|
|
||||||
case ARRAY:
|
|
||||||
return "array";
|
|
||||||
case NIL_ARRAY:
|
|
||||||
return "nil-array";
|
|
||||||
case NIL:
|
|
||||||
return "nil";
|
|
||||||
case ERROR:
|
|
||||||
return "error";
|
|
||||||
}
|
|
||||||
ABSL_INTERNAL_UNREACHABLE;
|
|
||||||
}
|
|
||||||
|
|
||||||
const char* IntentLock::ModeName(Mode m) {
|
const char* IntentLock::ModeName(Mode m) {
|
||||||
switch (m) {
|
switch (m) {
|
||||||
case IntentLock::SHARED:
|
case IntentLock::SHARED:
|
||||||
|
@ -43,47 +26,3 @@ void IntentLock::VerifyDebug() {
|
||||||
}
|
}
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace dfly
|
||||||
|
|
||||||
namespace std {
|
|
||||||
ostream& operator<<(ostream& os, const dfly::RespExpr& e) {
|
|
||||||
using dfly::RespExpr;
|
|
||||||
using dfly::ToSV;
|
|
||||||
|
|
||||||
switch (e.type) {
|
|
||||||
case RespExpr::INT64:
|
|
||||||
os << "i" << get<int64_t>(e.u);
|
|
||||||
break;
|
|
||||||
case RespExpr::STRING:
|
|
||||||
os << "'" << ToSV(get<RespExpr::Buffer>(e.u)) << "'";
|
|
||||||
break;
|
|
||||||
case RespExpr::NIL:
|
|
||||||
os << "nil";
|
|
||||||
break;
|
|
||||||
case RespExpr::NIL_ARRAY:
|
|
||||||
os << "[]";
|
|
||||||
break;
|
|
||||||
case RespExpr::ARRAY:
|
|
||||||
os << dfly::RespSpan{*get<RespExpr::Vec*>(e.u)};
|
|
||||||
break;
|
|
||||||
case RespExpr::ERROR:
|
|
||||||
os << "e(" << ToSV(get<RespExpr::Buffer>(e.u)) << ")";
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
|
|
||||||
return os;
|
|
||||||
}
|
|
||||||
|
|
||||||
ostream& operator<<(ostream& os, dfly::RespSpan ras) {
|
|
||||||
os << "[";
|
|
||||||
if (!ras.empty()) {
|
|
||||||
for (size_t i = 0; i < ras.size() - 1; ++i) {
|
|
||||||
os << ras[i] << ",";
|
|
||||||
}
|
|
||||||
os << ras.back();
|
|
||||||
}
|
|
||||||
os << "]";
|
|
||||||
|
|
||||||
return os;
|
|
||||||
}
|
|
||||||
|
|
||||||
} // namespace std
|
|
||||||
|
|
|
@ -0,0 +1,13 @@
|
||||||
|
add_library(dfly_facade dragonfly_listener.cc dragonfly_connection.cc facade.cc
|
||||||
|
memcache_parser.cc redis_parser.cc reply_builder.cc)
|
||||||
|
cxx_link(dfly_facade base uring_fiber_lib fibers_ext strings_lib http_server_lib
|
||||||
|
tls_lib TRDP::mimalloc)
|
||||||
|
|
||||||
|
add_library(facade_test facade_test.cc)
|
||||||
|
cxx_link(facade_test dfly_facade gtest_main_ext)
|
||||||
|
|
||||||
|
cxx_test(memcache_parser_test dfly_facade LABELS DFLY)
|
||||||
|
cxx_test(redis_parser_test facade_test LABELS DFLY)
|
||||||
|
|
||||||
|
add_executable(ok_backend ok_main.cc)
|
||||||
|
cxx_link(ok_backend dfly_facade)
|
|
@ -0,0 +1,8 @@
|
||||||
|
## A facade library
|
||||||
|
|
||||||
|
The library is responsible for opening dragonfly-like TCP client connections.
|
||||||
|
I call it facade because "client" term is often abused.
|
||||||
|
|
||||||
|
It should be separated from the rest of dragonfly server logic and should be self-contained, i.e
|
||||||
|
no redis-lib or server dependencies are allowed.
|
||||||
|
|
|
@ -0,0 +1,51 @@
|
||||||
|
// Copyright 2021, Roman Gershman. All rights reserved.
|
||||||
|
// See LICENSE for licensing terms.
|
||||||
|
//
|
||||||
|
|
||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <absl/container/flat_hash_set.h>
|
||||||
|
|
||||||
|
#include "facade/facade_types.h"
|
||||||
|
#include "facade/reply_builder.h"
|
||||||
|
|
||||||
|
namespace facade {
|
||||||
|
|
||||||
|
class Connection;
|
||||||
|
|
||||||
|
class ConnectionContext {
|
||||||
|
public:
|
||||||
|
ConnectionContext(::io::Sink* stream, Connection* owner);
|
||||||
|
|
||||||
|
Connection* owner() {
|
||||||
|
return owner_;
|
||||||
|
}
|
||||||
|
|
||||||
|
Protocol protocol() const;
|
||||||
|
|
||||||
|
// A convenient proxy for redis interface.
|
||||||
|
RedisReplyBuilder* operator->();
|
||||||
|
|
||||||
|
ReplyBuilderInterface* reply_builder() {
|
||||||
|
return rbuilder_.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Allows receiving the output data from the commands called from scripts.
|
||||||
|
ReplyBuilderInterface* Inject(ReplyBuilderInterface* new_i) {
|
||||||
|
ReplyBuilderInterface* res = rbuilder_.release();
|
||||||
|
rbuilder_.reset(new_i);
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
// connection state / properties.
|
||||||
|
bool async_dispatch: 1;
|
||||||
|
bool conn_closing: 1;
|
||||||
|
bool req_auth: 1;
|
||||||
|
bool replica_conn: 1;
|
||||||
|
bool authenticated: 1;
|
||||||
|
private:
|
||||||
|
Connection* owner_;
|
||||||
|
std::unique_ptr<ReplyBuilderInterface> rbuilder_;
|
||||||
|
};
|
||||||
|
|
||||||
|
} // namespace facade
|
|
@ -2,7 +2,7 @@
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
#include "server/dragonfly_connection.h"
|
#include "facade/dragonfly_connection.h"
|
||||||
|
|
||||||
#include <absl/container/flat_hash_map.h>
|
#include <absl/container/flat_hash_map.h>
|
||||||
#include <absl/strings/match.h>
|
#include <absl/strings/match.h>
|
||||||
|
@ -11,13 +11,11 @@
|
||||||
#include <boost/fiber/operations.hpp>
|
#include <boost/fiber/operations.hpp>
|
||||||
|
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
#include "server/command_registry.h"
|
|
||||||
#include "server/conn_context.h"
|
#include "facade/conn_context.h"
|
||||||
#include "server/main_service.h"
|
#include "facade/memcache_parser.h"
|
||||||
#include "server/memcache_parser.h"
|
#include "facade/redis_parser.h"
|
||||||
#include "server/redis_parser.h"
|
#include "facade/service_interface.h"
|
||||||
#include "server/server_state.h"
|
|
||||||
#include "server/transaction.h"
|
|
||||||
#include "util/fiber_sched_algo.h"
|
#include "util/fiber_sched_algo.h"
|
||||||
#include "util/tls/tls_socket.h"
|
#include "util/tls/tls_socket.h"
|
||||||
#include "util/uring/uring_socket.h"
|
#include "util/uring/uring_socket.h"
|
||||||
|
@ -30,7 +28,7 @@ using nonstd::make_unexpected;
|
||||||
namespace this_fiber = boost::this_fiber;
|
namespace this_fiber = boost::this_fiber;
|
||||||
namespace fibers = boost::fibers;
|
namespace fibers = boost::fibers;
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
namespace {
|
namespace {
|
||||||
|
|
||||||
void SendProtocolError(RedisParser::Result pres, FiberSocketBase* peer) {
|
void SendProtocolError(RedisParser::Result pres, FiberSocketBase* peer) {
|
||||||
|
@ -94,8 +92,9 @@ struct Connection::Request {
|
||||||
Request(const Request&) = delete;
|
Request(const Request&) = delete;
|
||||||
};
|
};
|
||||||
|
|
||||||
Connection::Connection(Protocol protocol, Service* service, SSL_CTX* ctx)
|
Connection::Connection(Protocol protocol, util::HttpListenerBase* http_listener, SSL_CTX* ctx,
|
||||||
: io_buf_{kMinReadSize}, service_(service), ctx_(ctx) {
|
ServiceInterface* service)
|
||||||
|
: io_buf_(kMinReadSize), http_listener_(http_listener), ctx_(ctx), service_(service) {
|
||||||
protocol_ = protocol;
|
protocol_ = protocol;
|
||||||
|
|
||||||
switch (protocol) {
|
switch (protocol) {
|
||||||
|
@ -167,7 +166,7 @@ void Connection::HandleRequests() {
|
||||||
if (http_res) {
|
if (http_res) {
|
||||||
if (*http_res) {
|
if (*http_res) {
|
||||||
VLOG(1) << "HTTP1.1 identified";
|
VLOG(1) << "HTTP1.1 identified";
|
||||||
HttpConnection http_conn{service_->http_listener()};
|
HttpConnection http_conn{http_listener_};
|
||||||
http_conn.SetSocket(peer);
|
http_conn.SetSocket(peer);
|
||||||
auto ec = http_conn.ParseFromBuffer(io_buf_.InputBuffer());
|
auto ec = http_conn.ParseFromBuffer(io_buf_.InputBuffer());
|
||||||
io_buf_.ConsumeInput(io_buf_.InputLen());
|
io_buf_.ConsumeInput(io_buf_.InputLen());
|
||||||
|
@ -176,30 +175,27 @@ void Connection::HandleRequests() {
|
||||||
}
|
}
|
||||||
http_conn.ReleaseSocket();
|
http_conn.ReleaseSocket();
|
||||||
} else {
|
} else {
|
||||||
cc_.reset(new ConnectionContext(peer, this));
|
cc_.reset(service_->CreateContext(peer, this));
|
||||||
cc_->shard_set = &service_->shard_set();
|
|
||||||
|
|
||||||
if (service_->IsPassProtected())
|
|
||||||
cc_->conn_state.mask |= ConnectionState::REQ_AUTH;
|
|
||||||
|
|
||||||
|
bool should_disarm_poller = false;
|
||||||
// TODO: to move this interface to LinuxSocketBase so we won't need to cast.
|
// TODO: to move this interface to LinuxSocketBase so we won't need to cast.
|
||||||
uring::UringSocket* us = static_cast<uring::UringSocket*>(socket_.get());
|
uring::UringSocket* us = static_cast<uring::UringSocket*>(socket_.get());
|
||||||
|
uint32_t poll_id = 0;
|
||||||
|
if (breaker_cb_) {
|
||||||
|
should_disarm_poller = true;
|
||||||
|
|
||||||
bool poll_armed = true;
|
poll_id = us->PollEvent(POLLERR | POLLHUP, [&](uint32_t mask) {
|
||||||
uint32_t poll_id = us->PollEvent(POLLERR | POLLHUP, [&](uint32_t mask) {
|
|
||||||
VLOG(1) << "Got event " << mask;
|
VLOG(1) << "Got event " << mask;
|
||||||
cc_->conn_state.mask |= ConnectionState::CONN_CLOSING;
|
cc_->conn_closing = true;
|
||||||
if (cc_->transaction) {
|
breaker_cb_(mask);
|
||||||
cc_->transaction->BreakOnClose();
|
|
||||||
}
|
|
||||||
|
|
||||||
evc_.notify(); // Notify dispatch fiber.
|
evc_.notify(); // Notify dispatch fiber.
|
||||||
poll_armed = false;
|
should_disarm_poller = false;
|
||||||
});
|
});
|
||||||
|
}
|
||||||
|
|
||||||
ConnectionFlow(peer);
|
ConnectionFlow(peer);
|
||||||
|
|
||||||
if (poll_armed) {
|
if (should_disarm_poller) {
|
||||||
us->CancelPoll(poll_id);
|
us->CancelPoll(poll_id);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -208,6 +204,10 @@ void Connection::HandleRequests() {
|
||||||
VLOG(1) << "Closed connection for peer " << remote_ep;
|
VLOG(1) << "Closed connection for peer " << remote_ep;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void Connection::RegisterOnBreak(BreakerCb breaker_cb) {
|
||||||
|
breaker_cb_ = breaker_cb;
|
||||||
|
}
|
||||||
|
|
||||||
io::Result<bool> Connection::CheckForHttpProto(util::FiberSocketBase* peer) {
|
io::Result<bool> Connection::CheckForHttpProto(util::FiberSocketBase* peer) {
|
||||||
size_t last_len = 0;
|
size_t last_len = 0;
|
||||||
do {
|
do {
|
||||||
|
@ -235,7 +235,7 @@ io::Result<bool> Connection::CheckForHttpProto(util::FiberSocketBase* peer) {
|
||||||
|
|
||||||
void Connection::ConnectionFlow(FiberSocketBase* peer) {
|
void Connection::ConnectionFlow(FiberSocketBase* peer) {
|
||||||
auto dispatch_fb = fibers::fiber(fibers::launch::dispatch, [&] { DispatchFiber(peer); });
|
auto dispatch_fb = fibers::fiber(fibers::launch::dispatch, [&] { DispatchFiber(peer); });
|
||||||
ConnectionStats* stats = ServerState::tl_connection_stats();
|
ConnectionStats* stats = service_->GetThreadLocalConnectionStats();
|
||||||
stats->num_conns++;
|
stats->num_conns++;
|
||||||
stats->read_buf_capacity += io_buf_.Capacity();
|
stats->read_buf_capacity += io_buf_.Capacity();
|
||||||
|
|
||||||
|
@ -265,14 +265,14 @@ void Connection::ConnectionFlow(FiberSocketBase* peer) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
cc_->conn_state.mask |= ConnectionState::CONN_CLOSING; // Signal dispatch to close.
|
cc_->conn_closing = true; // Signal dispatch to close.
|
||||||
evc_.notify();
|
evc_.notify();
|
||||||
dispatch_fb.join();
|
dispatch_fb.join();
|
||||||
|
|
||||||
stats->read_buf_capacity -= io_buf_.Capacity();
|
stats->read_buf_capacity -= io_buf_.Capacity();
|
||||||
|
|
||||||
// Update num_replicas if this was a replica connection.
|
// Update num_replicas if this was a replica connection.
|
||||||
if (cc_->conn_state.mask & ConnectionState::REPL_CONNECTION) {
|
if (cc_->replica_conn) {
|
||||||
--stats->num_replicas;
|
--stats->num_replicas;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -321,7 +321,7 @@ auto Connection::ParseRedis() -> ParserStatus {
|
||||||
// We use ASYNC_DISPATCH as a lock to avoid out-of-order replies when the
|
// We use ASYNC_DISPATCH as a lock to avoid out-of-order replies when the
|
||||||
// dispatch fiber pulls the last record but is still processing the command and then this
|
// dispatch fiber pulls the last record but is still processing the command and then this
|
||||||
// fiber enters the condition below and executes out of order.
|
// fiber enters the condition below and executes out of order.
|
||||||
bool is_sync_dispatch = !cc_->conn_state.IsRunViaDispatch();
|
bool is_sync_dispatch = !cc_->async_dispatch;
|
||||||
if (dispatch_q_.empty() && is_sync_dispatch && consumed >= io_buf_.InputLen()) {
|
if (dispatch_q_.empty() && is_sync_dispatch && consumed >= io_buf_.InputLen()) {
|
||||||
RespToArgList(args, &arg_vec);
|
RespToArgList(args, &arg_vec);
|
||||||
service_->DispatchCommand(CmdArgList{arg_vec.data(), arg_vec.size()}, cc_.get());
|
service_->DispatchCommand(CmdArgList{arg_vec.data(), arg_vec.size()}, cc_.get());
|
||||||
|
@ -382,7 +382,7 @@ auto Connection::ParseMemcache() -> ParserStatus {
|
||||||
// We use ASYNC_DISPATCH as a lock to avoid out-of-order replies when the
|
// We use ASYNC_DISPATCH as a lock to avoid out-of-order replies when the
|
||||||
// dispatch fiber pulls the last record but is still processing the command and then this
|
// dispatch fiber pulls the last record but is still processing the command and then this
|
||||||
// fiber enters the condition below and executes out of order.
|
// fiber enters the condition below and executes out of order.
|
||||||
bool is_sync_dispatch = (cc_->conn_state.mask & ConnectionState::ASYNC_DISPATCH) == 0;
|
bool is_sync_dispatch = !cc_->async_dispatch;
|
||||||
if (dispatch_q_.empty() && is_sync_dispatch) {
|
if (dispatch_q_.empty() && is_sync_dispatch) {
|
||||||
service_->DispatchMC(cmd, value, cc_.get());
|
service_->DispatchMC(cmd, value, cc_.get());
|
||||||
}
|
}
|
||||||
|
@ -408,7 +408,7 @@ auto Connection::ParseMemcache() -> ParserStatus {
|
||||||
|
|
||||||
auto Connection::IoLoop(util::FiberSocketBase* peer) -> variant<error_code, ParserStatus> {
|
auto Connection::IoLoop(util::FiberSocketBase* peer) -> variant<error_code, ParserStatus> {
|
||||||
SinkReplyBuilder* builder = static_cast<SinkReplyBuilder*>(cc_->reply_builder());
|
SinkReplyBuilder* builder = static_cast<SinkReplyBuilder*>(cc_->reply_builder());
|
||||||
ConnectionStats* stats = ServerState::tl_connection_stats();
|
ConnectionStats* stats = service_->GetThreadLocalConnectionStats();
|
||||||
error_code ec;
|
error_code ec;
|
||||||
ParserStatus parse_status = OK;
|
ParserStatus parse_status = OK;
|
||||||
|
|
||||||
|
@ -484,12 +484,12 @@ auto Connection::IoLoop(util::FiberSocketBase* peer) -> variant<error_code, Pars
|
||||||
void Connection::DispatchFiber(util::FiberSocketBase* peer) {
|
void Connection::DispatchFiber(util::FiberSocketBase* peer) {
|
||||||
this_fiber::properties<FiberProps>().set_name("DispatchFiber");
|
this_fiber::properties<FiberProps>().set_name("DispatchFiber");
|
||||||
|
|
||||||
ConnectionStats* stats = ServerState::tl_connection_stats();
|
ConnectionStats* stats = service_->GetThreadLocalConnectionStats();
|
||||||
SinkReplyBuilder* builder = static_cast<SinkReplyBuilder*>(cc_->reply_builder());
|
SinkReplyBuilder* builder = static_cast<SinkReplyBuilder*>(cc_->reply_builder());
|
||||||
|
|
||||||
while (!builder->GetError()) {
|
while (!builder->GetError()) {
|
||||||
evc_.await([this] { return cc_->conn_state.IsClosing() || !dispatch_q_.empty(); });
|
evc_.await([this] { return cc_->conn_closing || !dispatch_q_.empty(); });
|
||||||
if (cc_->conn_state.IsClosing())
|
if (cc_->conn_closing)
|
||||||
break; // TODO: We have a memory leak with pending requests in the queue.
|
break; // TODO: We have a memory leak with pending requests in the queue.
|
||||||
|
|
||||||
Request* req = dispatch_q_.front();
|
Request* req = dispatch_q_.front();
|
||||||
|
@ -498,14 +498,14 @@ void Connection::DispatchFiber(util::FiberSocketBase* peer) {
|
||||||
++stats->pipelined_cmd_cnt;
|
++stats->pipelined_cmd_cnt;
|
||||||
|
|
||||||
builder->SetBatchMode(!dispatch_q_.empty());
|
builder->SetBatchMode(!dispatch_q_.empty());
|
||||||
cc_->conn_state.mask |= ConnectionState::ASYNC_DISPATCH;
|
cc_->async_dispatch = true;
|
||||||
service_->DispatchCommand(CmdArgList{req->args.data(), req->args.size()}, cc_.get());
|
service_->DispatchCommand(CmdArgList{req->args.data(), req->args.size()}, cc_.get());
|
||||||
cc_->conn_state.mask &= ~ConnectionState::ASYNC_DISPATCH;
|
cc_->async_dispatch = false;
|
||||||
req->~Request();
|
req->~Request();
|
||||||
mi_free(req);
|
mi_free(req);
|
||||||
}
|
}
|
||||||
|
|
||||||
cc_->conn_state.mask |= ConnectionState::CONN_CLOSING;
|
cc_->conn_closing = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
auto Connection::FromArgs(RespVec args, mi_heap_t* heap) -> Request* {
|
auto Connection::FromArgs(RespVec args, mi_heap_t* heap) -> Request* {
|
||||||
|
@ -536,4 +536,4 @@ auto Connection::FromArgs(RespVec args, mi_heap_t* heap) -> Request* {
|
||||||
return req;
|
return req;
|
||||||
}
|
}
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace facade
|
|
@ -10,24 +10,26 @@
|
||||||
#include <variant>
|
#include <variant>
|
||||||
|
|
||||||
#include "base/io_buf.h"
|
#include "base/io_buf.h"
|
||||||
#include "core/resp_expr.h"
|
#include "facade/facade_types.h"
|
||||||
#include "server/common_types.h"
|
#include "facade/resp_expr.h"
|
||||||
#include "util/connection.h"
|
#include "util/connection.h"
|
||||||
#include "util/fibers/event_count.h"
|
#include "util/fibers/event_count.h"
|
||||||
|
#include "util/http/http_handler.h"
|
||||||
|
|
||||||
typedef struct ssl_ctx_st SSL_CTX;
|
typedef struct ssl_ctx_st SSL_CTX;
|
||||||
typedef struct mi_heap_s mi_heap_t;
|
typedef struct mi_heap_s mi_heap_t;
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
|
|
||||||
class ConnectionContext;
|
class ConnectionContext;
|
||||||
class RedisParser;
|
class RedisParser;
|
||||||
class Service;
|
class ServiceInterface;
|
||||||
class MemcacheParser;
|
class MemcacheParser;
|
||||||
|
|
||||||
class Connection : public util::Connection {
|
class Connection : public util::Connection {
|
||||||
public:
|
public:
|
||||||
Connection(Protocol protocol, Service* service, SSL_CTX* ctx);
|
Connection(Protocol protocol, util::HttpListenerBase* http_listener,
|
||||||
|
SSL_CTX* ctx, ServiceInterface* service);
|
||||||
~Connection();
|
~Connection();
|
||||||
|
|
||||||
using error_code = std::error_code;
|
using error_code = std::error_code;
|
||||||
|
@ -41,6 +43,9 @@ class Connection : public util::Connection {
|
||||||
return protocol_;
|
return protocol_;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
using BreakerCb = std::function<void(uint32_t)>;
|
||||||
|
void RegisterOnBreak(BreakerCb breaker_cb);
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
void OnShutdown() override;
|
void OnShutdown() override;
|
||||||
|
|
||||||
|
@ -63,8 +68,10 @@ class Connection : public util::Connection {
|
||||||
base::IoBuf io_buf_;
|
base::IoBuf io_buf_;
|
||||||
std::unique_ptr<RedisParser> redis_parser_;
|
std::unique_ptr<RedisParser> redis_parser_;
|
||||||
std::unique_ptr<MemcacheParser> memcache_parser_;
|
std::unique_ptr<MemcacheParser> memcache_parser_;
|
||||||
Service* service_;
|
util::HttpListenerBase* http_listener_;
|
||||||
SSL_CTX* ctx_;
|
SSL_CTX* ctx_;
|
||||||
|
ServiceInterface* service_;
|
||||||
|
|
||||||
std::unique_ptr<ConnectionContext> cc_;
|
std::unique_ptr<ConnectionContext> cc_;
|
||||||
|
|
||||||
struct Request;
|
struct Request;
|
||||||
|
@ -77,6 +84,7 @@ class Connection : public util::Connection {
|
||||||
Protocol protocol_;
|
Protocol protocol_;
|
||||||
struct Shutdown;
|
struct Shutdown;
|
||||||
std::unique_ptr<Shutdown> shutdown_;
|
std::unique_ptr<Shutdown> shutdown_;
|
||||||
|
BreakerCb breaker_cb_;
|
||||||
};
|
};
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace facade
|
|
@ -1,14 +1,13 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
#include "server/dragonfly_listener.h"
|
#include "facade/dragonfly_listener.h"
|
||||||
|
|
||||||
#include <openssl/ssl.h>
|
#include <openssl/ssl.h>
|
||||||
|
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
#include "server/config_flags.h"
|
#include "facade/dragonfly_connection.h"
|
||||||
#include "server/dragonfly_connection.h"
|
|
||||||
#include "util/proactor_pool.h"
|
#include "util/proactor_pool.h"
|
||||||
|
|
||||||
DEFINE_uint32(conn_threads, 0, "Number of threads used for handing server connections");
|
DEFINE_uint32(conn_threads, 0, "Number of threads used for handing server connections");
|
||||||
|
@ -17,16 +16,17 @@ DEFINE_bool(conn_use_incoming_cpu, false,
|
||||||
"If true uses incoming cpu of a socket in order to distribute"
|
"If true uses incoming cpu of a socket in order to distribute"
|
||||||
" incoming connections");
|
" incoming connections");
|
||||||
|
|
||||||
CONFIG_string(tls_client_cert_file, "", "", TrueValidator);
|
DEFINE_string(tls_client_cert_file, "", "cert file for tls connections");
|
||||||
CONFIG_string(tls_client_key_file, "", "", TrueValidator);
|
DEFINE_string(tls_client_key_file, "", "key file for tls connections");
|
||||||
|
|
||||||
|
#if 0
|
||||||
enum TlsClientAuth {
|
enum TlsClientAuth {
|
||||||
CL_AUTH_NO = 0,
|
CL_AUTH_NO = 0,
|
||||||
CL_AUTH_YES = 1,
|
CL_AUTH_YES = 1,
|
||||||
CL_AUTH_OPTIONAL = 2,
|
CL_AUTH_OPTIONAL = 2,
|
||||||
};
|
};
|
||||||
|
|
||||||
dfly::ConfigEnum tls_auth_clients_enum[] = {
|
facade::ConfigEnum tls_auth_clients_enum[] = {
|
||||||
{"no", CL_AUTH_NO},
|
{"no", CL_AUTH_NO},
|
||||||
{"yes", CL_AUTH_YES},
|
{"yes", CL_AUTH_YES},
|
||||||
{"optional", CL_AUTH_OPTIONAL},
|
{"optional", CL_AUTH_OPTIONAL},
|
||||||
|
@ -35,8 +35,9 @@ dfly::ConfigEnum tls_auth_clients_enum[] = {
|
||||||
static int tls_auth_clients_opt = CL_AUTH_YES;
|
static int tls_auth_clients_opt = CL_AUTH_YES;
|
||||||
|
|
||||||
CONFIG_enum(tls_auth_clients, "yes", "", tls_auth_clients_enum, tls_auth_clients_opt);
|
CONFIG_enum(tls_auth_clients, "yes", "", tls_auth_clients_enum, tls_auth_clients_opt);
|
||||||
|
#endif
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
|
|
||||||
using namespace util;
|
using namespace util;
|
||||||
using namespace std;
|
using namespace std;
|
||||||
|
@ -85,11 +86,14 @@ static SSL_CTX* CreateSslCntx() {
|
||||||
return ctx;
|
return ctx;
|
||||||
}
|
}
|
||||||
|
|
||||||
Listener::Listener(Protocol protocol, Service* e) : engine_(e), protocol_(protocol) {
|
Listener::Listener(Protocol protocol, ServiceInterface* e) : service_(e), protocol_(protocol) {
|
||||||
if (FLAGS_tls) {
|
if (FLAGS_tls) {
|
||||||
OPENSSL_init_ssl(OPENSSL_INIT_SSL_DEFAULT, NULL);
|
OPENSSL_init_ssl(OPENSSL_INIT_SSL_DEFAULT, NULL);
|
||||||
ctx_ = CreateSslCntx();
|
ctx_ = CreateSslCntx();
|
||||||
}
|
}
|
||||||
|
http_base_.reset(new HttpListener<>);
|
||||||
|
|
||||||
|
http_base_->enable_metrics();
|
||||||
}
|
}
|
||||||
|
|
||||||
Listener::~Listener() {
|
Listener::~Listener() {
|
||||||
|
@ -97,7 +101,7 @@ Listener::~Listener() {
|
||||||
}
|
}
|
||||||
|
|
||||||
util::Connection* Listener::NewConnection(ProactorBase* proactor) {
|
util::Connection* Listener::NewConnection(ProactorBase* proactor) {
|
||||||
return new Connection{protocol_, engine_, ctx_};
|
return new Connection{protocol_, http_base_.get(), ctx_, service_};
|
||||||
}
|
}
|
||||||
|
|
||||||
void Listener::PreShutdown() {
|
void Listener::PreShutdown() {
|
||||||
|
@ -139,4 +143,4 @@ ProactorBase* Listener::PickConnectionProactor(LinuxSocketBase* sock) {
|
||||||
return pp->at(id % total);
|
return pp->at(id % total);
|
||||||
}
|
}
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace facade
|
|
@ -4,18 +4,19 @@
|
||||||
|
|
||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include "facade/facade_types.h"
|
||||||
|
#include "util/http/http_handler.h"
|
||||||
#include "util/listener_interface.h"
|
#include "util/listener_interface.h"
|
||||||
#include "server/common_types.h"
|
|
||||||
|
|
||||||
typedef struct ssl_ctx_st SSL_CTX;
|
typedef struct ssl_ctx_st SSL_CTX;
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
|
|
||||||
class Service;
|
class ServiceInterface;
|
||||||
|
|
||||||
class Listener : public util::ListenerInterface {
|
class Listener : public util::ListenerInterface {
|
||||||
public:
|
public:
|
||||||
Listener(Protocol protocol, Service*);
|
Listener(Protocol protocol, ServiceInterface*);
|
||||||
~Listener();
|
~Listener();
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
@ -26,11 +27,13 @@ class Listener : public util::ListenerInterface {
|
||||||
|
|
||||||
void PostShutdown();
|
void PostShutdown();
|
||||||
|
|
||||||
Service* engine_;
|
std::unique_ptr<util::HttpListenerBase> http_base_;
|
||||||
|
|
||||||
|
ServiceInterface* service_;
|
||||||
|
|
||||||
std::atomic_uint32_t next_id_{0};
|
std::atomic_uint32_t next_id_{0};
|
||||||
Protocol protocol_;
|
Protocol protocol_;
|
||||||
SSL_CTX* ctx_ = nullptr;
|
SSL_CTX* ctx_ = nullptr;
|
||||||
};
|
};
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace facade
|
|
@ -0,0 +1,25 @@
|
||||||
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
|
// See LICENSE for licensing terms.
|
||||||
|
//
|
||||||
|
|
||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <atomic>
|
||||||
|
#include <string>
|
||||||
|
|
||||||
|
namespace facade {
|
||||||
|
|
||||||
|
std::string WrongNumArgsError(std::string_view cmd);
|
||||||
|
|
||||||
|
extern const char kSyntaxErr[];
|
||||||
|
extern const char kWrongTypeErr[];
|
||||||
|
extern const char kKeyNotFoundErr[];
|
||||||
|
extern const char kInvalidIntErr[];
|
||||||
|
extern const char kUintErr[];
|
||||||
|
extern const char kDbIndOutOfRangeErr[];
|
||||||
|
extern const char kInvalidDbIndErr[];
|
||||||
|
extern const char kScriptNotFound[];
|
||||||
|
extern const char kAuthRejected[];
|
||||||
|
|
||||||
|
|
||||||
|
} // namespace dfly
|
|
@ -0,0 +1,157 @@
|
||||||
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
|
// See LICENSE for licensing terms.
|
||||||
|
//
|
||||||
|
|
||||||
|
#include <absl/strings/str_cat.h>
|
||||||
|
|
||||||
|
#include "base/logging.h"
|
||||||
|
|
||||||
|
#include "facade/conn_context.h"
|
||||||
|
#include "facade/dragonfly_connection.h"
|
||||||
|
#include "facade/error.h"
|
||||||
|
#include "facade/facade_types.h"
|
||||||
|
|
||||||
|
namespace facade {
|
||||||
|
|
||||||
|
using namespace std;
|
||||||
|
|
||||||
|
#define ADD(x) (x) += o.x
|
||||||
|
|
||||||
|
ConnectionStats& ConnectionStats::operator+=(const ConnectionStats& o) {
|
||||||
|
// To break this code deliberately if we add/remove a field to this struct.
|
||||||
|
static_assert(sizeof(ConnectionStats) == 64);
|
||||||
|
|
||||||
|
ADD(num_conns);
|
||||||
|
ADD(num_replicas);
|
||||||
|
ADD(read_buf_capacity);
|
||||||
|
ADD(io_read_cnt);
|
||||||
|
ADD(io_read_bytes);
|
||||||
|
ADD(io_write_cnt);
|
||||||
|
ADD(io_write_bytes);
|
||||||
|
ADD(pipelined_cmd_cnt);
|
||||||
|
ADD(command_cnt);
|
||||||
|
|
||||||
|
return *this;
|
||||||
|
}
|
||||||
|
|
||||||
|
#undef ADD
|
||||||
|
|
||||||
|
string WrongNumArgsError(std::string_view cmd) {
|
||||||
|
return absl::StrCat("wrong number of arguments for '", cmd, "' command");
|
||||||
|
}
|
||||||
|
|
||||||
|
const char kSyntaxErr[] = "syntax error";
|
||||||
|
const char kWrongTypeErr[] = "-WRONGTYPE Operation against a key holding the wrong kind of value";
|
||||||
|
const char kKeyNotFoundErr[] = "no such key";
|
||||||
|
const char kInvalidIntErr[] = "value is not an integer or out of range";
|
||||||
|
const char kUintErr[] = "value is out of range, must be positive";
|
||||||
|
const char kDbIndOutOfRangeErr[] = "DB index is out of range";
|
||||||
|
const char kInvalidDbIndErr[] = "invalid DB index";
|
||||||
|
const char kScriptNotFound[] = "-NOSCRIPT No matching script. Please use EVAL.";
|
||||||
|
const char kAuthRejected[] = "-WRONGPASS invalid username-password pair or user is disabled.";
|
||||||
|
|
||||||
|
const char* RespExpr::TypeName(Type t) {
|
||||||
|
switch (t) {
|
||||||
|
case STRING:
|
||||||
|
return "string";
|
||||||
|
case INT64:
|
||||||
|
return "int";
|
||||||
|
case ARRAY:
|
||||||
|
return "array";
|
||||||
|
case NIL_ARRAY:
|
||||||
|
return "nil-array";
|
||||||
|
case NIL:
|
||||||
|
return "nil";
|
||||||
|
case ERROR:
|
||||||
|
return "error";
|
||||||
|
}
|
||||||
|
ABSL_INTERNAL_UNREACHABLE;
|
||||||
|
}
|
||||||
|
|
||||||
|
ConnectionContext::ConnectionContext(::io::Sink* stream, Connection* owner) : owner_(owner) {
|
||||||
|
switch (owner->protocol()) {
|
||||||
|
case Protocol::REDIS:
|
||||||
|
rbuilder_.reset(new RedisReplyBuilder(stream));
|
||||||
|
break;
|
||||||
|
case Protocol::MEMCACHE:
|
||||||
|
rbuilder_.reset(new MCReplyBuilder(stream));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
async_dispatch = false;
|
||||||
|
conn_closing = false;
|
||||||
|
req_auth = false;
|
||||||
|
replica_conn = false;
|
||||||
|
authenticated = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
Protocol ConnectionContext::protocol() const {
|
||||||
|
return owner_->protocol();
|
||||||
|
}
|
||||||
|
|
||||||
|
RedisReplyBuilder* ConnectionContext::operator->() {
|
||||||
|
CHECK(Protocol::REDIS == protocol());
|
||||||
|
|
||||||
|
return static_cast<RedisReplyBuilder*>(rbuilder_.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
} // namespace facade
|
||||||
|
|
||||||
|
|
||||||
|
namespace std {
|
||||||
|
|
||||||
|
ostream& operator<<(ostream& os, facade::CmdArgList ras) {
|
||||||
|
os << "[";
|
||||||
|
if (!ras.empty()) {
|
||||||
|
for (size_t i = 0; i < ras.size() - 1; ++i) {
|
||||||
|
os << facade::ArgS(ras, i) << ",";
|
||||||
|
}
|
||||||
|
os << facade::ArgS(ras, ras.size() - 1);
|
||||||
|
}
|
||||||
|
os << "]";
|
||||||
|
|
||||||
|
return os;
|
||||||
|
}
|
||||||
|
|
||||||
|
ostream& operator<<(ostream& os, const facade::RespExpr& e) {
|
||||||
|
using facade::RespExpr;
|
||||||
|
using facade::ToSV;
|
||||||
|
|
||||||
|
switch (e.type) {
|
||||||
|
case RespExpr::INT64:
|
||||||
|
os << "i" << get<int64_t>(e.u);
|
||||||
|
break;
|
||||||
|
case RespExpr::STRING:
|
||||||
|
os << "'" << ToSV(get<RespExpr::Buffer>(e.u)) << "'";
|
||||||
|
break;
|
||||||
|
case RespExpr::NIL:
|
||||||
|
os << "nil";
|
||||||
|
break;
|
||||||
|
case RespExpr::NIL_ARRAY:
|
||||||
|
os << "[]";
|
||||||
|
break;
|
||||||
|
case RespExpr::ARRAY:
|
||||||
|
os << facade::RespSpan{*get<RespExpr::Vec*>(e.u)};
|
||||||
|
break;
|
||||||
|
case RespExpr::ERROR:
|
||||||
|
os << "e(" << ToSV(get<RespExpr::Buffer>(e.u)) << ")";
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
return os;
|
||||||
|
}
|
||||||
|
|
||||||
|
ostream& operator<<(ostream& os, facade::RespSpan ras) {
|
||||||
|
os << "[";
|
||||||
|
if (!ras.empty()) {
|
||||||
|
for (size_t i = 0; i < ras.size() - 1; ++i) {
|
||||||
|
os << ras[i] << ",";
|
||||||
|
}
|
||||||
|
os << ras.back();
|
||||||
|
}
|
||||||
|
os << "]";
|
||||||
|
|
||||||
|
return os;
|
||||||
|
}
|
||||||
|
|
||||||
|
} // namespace std
|
|
@ -0,0 +1,98 @@
|
||||||
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
|
// See LICENSE for licensing terms.
|
||||||
|
//
|
||||||
|
|
||||||
|
#include "facade/facade_test.h"
|
||||||
|
|
||||||
|
#include <absl/strings/match.h>
|
||||||
|
|
||||||
|
namespace facade {
|
||||||
|
|
||||||
|
using namespace testing;
|
||||||
|
using namespace std;
|
||||||
|
|
||||||
|
bool RespMatcher::MatchAndExplain(const RespExpr& e, MatchResultListener* listener) const {
|
||||||
|
if (e.type != type_) {
|
||||||
|
*listener << "\nWrong type: " << RespExpr::TypeName(e.type);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (type_ == RespExpr::STRING || type_ == RespExpr::ERROR) {
|
||||||
|
RespExpr::Buffer ebuf = e.GetBuf();
|
||||||
|
std::string_view actual{reinterpret_cast<char*>(ebuf.data()), ebuf.size()};
|
||||||
|
|
||||||
|
if (type_ == RespExpr::ERROR && !absl::StrContains(actual, exp_str_)) {
|
||||||
|
*listener << "Actual does not contain '" << exp_str_ << "'";
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (type_ == RespExpr::STRING && exp_str_ != actual) {
|
||||||
|
*listener << "\nActual string: " << actual;
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
} else if (type_ == RespExpr::INT64) {
|
||||||
|
auto actual = get<int64_t>(e.u);
|
||||||
|
if (exp_int_ != actual) {
|
||||||
|
*listener << "\nActual : " << actual << " expected: " << exp_int_;
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
} else if (type_ == RespExpr::ARRAY) {
|
||||||
|
size_t len = get<RespVec*>(e.u)->size();
|
||||||
|
if (len != size_t(exp_int_)) {
|
||||||
|
*listener << "Actual length " << len << ", expected: " << exp_int_;
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
void RespMatcher::DescribeTo(std::ostream* os) const {
|
||||||
|
*os << "is ";
|
||||||
|
switch (type_) {
|
||||||
|
case RespExpr::STRING:
|
||||||
|
case RespExpr::ERROR:
|
||||||
|
*os << exp_str_;
|
||||||
|
break;
|
||||||
|
|
||||||
|
case RespExpr::INT64:
|
||||||
|
*os << exp_str_;
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
*os << "TBD";
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void RespMatcher::DescribeNegationTo(std::ostream* os) const {
|
||||||
|
*os << "is not ";
|
||||||
|
}
|
||||||
|
|
||||||
|
bool RespTypeMatcher::MatchAndExplain(const RespExpr& e, MatchResultListener* listener) const {
|
||||||
|
if (e.type != type_) {
|
||||||
|
*listener << "\nWrong type: " << RespExpr::TypeName(e.type);
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
void RespTypeMatcher::DescribeTo(std::ostream* os) const {
|
||||||
|
*os << "is " << RespExpr::TypeName(type_);
|
||||||
|
}
|
||||||
|
|
||||||
|
void RespTypeMatcher::DescribeNegationTo(std::ostream* os) const {
|
||||||
|
*os << "is not " << RespExpr::TypeName(type_);
|
||||||
|
}
|
||||||
|
|
||||||
|
void PrintTo(const RespExpr::Vec& vec, std::ostream* os) {
|
||||||
|
*os << "Vec: [";
|
||||||
|
if (!vec.empty()) {
|
||||||
|
for (size_t i = 0; i < vec.size() - 1; ++i) {
|
||||||
|
*os << vec[i] << ",";
|
||||||
|
}
|
||||||
|
*os << vec.back();
|
||||||
|
}
|
||||||
|
*os << "]\n";
|
||||||
|
}
|
||||||
|
|
||||||
|
} // namespace facade
|
|
@ -0,0 +1,83 @@
|
||||||
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
|
// See LICENSE for licensing terms.
|
||||||
|
//
|
||||||
|
|
||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <gmock/gmock.h>
|
||||||
|
|
||||||
|
#include "facade/resp_expr.h"
|
||||||
|
|
||||||
|
namespace facade {
|
||||||
|
|
||||||
|
class RespMatcher {
|
||||||
|
public:
|
||||||
|
RespMatcher(std::string_view val, RespExpr::Type t = RespExpr::STRING) : type_(t), exp_str_(val) {
|
||||||
|
}
|
||||||
|
|
||||||
|
RespMatcher(int64_t val, RespExpr::Type t = RespExpr::INT64) : type_(t), exp_int_(val) {
|
||||||
|
}
|
||||||
|
|
||||||
|
using is_gtest_matcher = void;
|
||||||
|
|
||||||
|
bool MatchAndExplain(const RespExpr& e, testing::MatchResultListener*) const;
|
||||||
|
|
||||||
|
void DescribeTo(std::ostream* os) const;
|
||||||
|
|
||||||
|
void DescribeNegationTo(std::ostream* os) const;
|
||||||
|
|
||||||
|
private:
|
||||||
|
RespExpr::Type type_;
|
||||||
|
|
||||||
|
std::string exp_str_;
|
||||||
|
int64_t exp_int_;
|
||||||
|
};
|
||||||
|
|
||||||
|
class RespTypeMatcher {
|
||||||
|
public:
|
||||||
|
RespTypeMatcher(RespExpr::Type type) : type_(type) {
|
||||||
|
}
|
||||||
|
|
||||||
|
using is_gtest_matcher = void;
|
||||||
|
|
||||||
|
bool MatchAndExplain(const RespExpr& e, testing::MatchResultListener*) const;
|
||||||
|
|
||||||
|
void DescribeTo(std::ostream* os) const;
|
||||||
|
|
||||||
|
void DescribeNegationTo(std::ostream* os) const;
|
||||||
|
|
||||||
|
private:
|
||||||
|
RespExpr::Type type_;
|
||||||
|
};
|
||||||
|
|
||||||
|
inline ::testing::PolymorphicMatcher<RespMatcher> StrArg(std::string_view str) {
|
||||||
|
return ::testing::MakePolymorphicMatcher(RespMatcher(str));
|
||||||
|
}
|
||||||
|
|
||||||
|
inline ::testing::PolymorphicMatcher<RespMatcher> ErrArg(std::string_view str) {
|
||||||
|
return ::testing::MakePolymorphicMatcher(RespMatcher(str, RespExpr::ERROR));
|
||||||
|
}
|
||||||
|
|
||||||
|
inline ::testing::PolymorphicMatcher<RespMatcher> IntArg(int64_t ival) {
|
||||||
|
return ::testing::MakePolymorphicMatcher(RespMatcher(ival));
|
||||||
|
}
|
||||||
|
|
||||||
|
inline ::testing::PolymorphicMatcher<RespMatcher> ArrLen(size_t len) {
|
||||||
|
return ::testing::MakePolymorphicMatcher(RespMatcher(len, RespExpr::ARRAY));
|
||||||
|
}
|
||||||
|
|
||||||
|
inline ::testing::PolymorphicMatcher<RespTypeMatcher> ArgType(RespExpr::Type t) {
|
||||||
|
return ::testing::MakePolymorphicMatcher(RespTypeMatcher(t));
|
||||||
|
}
|
||||||
|
|
||||||
|
inline bool operator==(const RespExpr& left, const char* s) {
|
||||||
|
return left.type == RespExpr::STRING && ToSV(left.GetBuf()) == s;
|
||||||
|
}
|
||||||
|
|
||||||
|
void PrintTo(const RespExpr::Vec& vec, std::ostream* os);
|
||||||
|
|
||||||
|
MATCHER_P(RespEq, val, "") {
|
||||||
|
return ::testing::ExplainMatchResult(::testing::ElementsAre(StrArg(val)), arg, result_listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
} // namespace facade
|
|
@ -0,0 +1,58 @@
|
||||||
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
|
// See LICENSE for licensing terms.
|
||||||
|
//
|
||||||
|
|
||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <absl/types/span.h>
|
||||||
|
|
||||||
|
namespace facade {
|
||||||
|
|
||||||
|
enum class Protocol : uint8_t {
|
||||||
|
MEMCACHE = 1,
|
||||||
|
REDIS = 2
|
||||||
|
};
|
||||||
|
|
||||||
|
using MutableSlice = absl::Span<char>;
|
||||||
|
using CmdArgList = absl::Span<MutableSlice>;
|
||||||
|
using CmdArgVec = std::vector<MutableSlice>;
|
||||||
|
|
||||||
|
|
||||||
|
struct ConnectionStats {
|
||||||
|
uint32_t num_conns = 0;
|
||||||
|
uint32_t num_replicas = 0;
|
||||||
|
size_t read_buf_capacity = 0;
|
||||||
|
size_t io_read_cnt = 0;
|
||||||
|
size_t io_read_bytes = 0;
|
||||||
|
size_t io_write_cnt = 0;
|
||||||
|
size_t io_write_bytes = 0;
|
||||||
|
size_t command_cnt = 0;
|
||||||
|
size_t pipelined_cmd_cnt = 0;
|
||||||
|
|
||||||
|
ConnectionStats& operator+=(const ConnectionStats& o);
|
||||||
|
};
|
||||||
|
|
||||||
|
inline MutableSlice ToMSS(absl::Span<uint8_t> span) {
|
||||||
|
return MutableSlice{reinterpret_cast<char*>(span.data()), span.size()};
|
||||||
|
}
|
||||||
|
|
||||||
|
inline std::string_view ArgS(CmdArgList args, size_t i) {
|
||||||
|
auto arg = args[i];
|
||||||
|
return std::string_view(arg.data(), arg.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
constexpr inline unsigned long long operator""_MB(unsigned long long x) {
|
||||||
|
return 1024L * 1024L * x;
|
||||||
|
}
|
||||||
|
|
||||||
|
constexpr inline unsigned long long operator""_KB(unsigned long long x) {
|
||||||
|
return 1024L * x;
|
||||||
|
}
|
||||||
|
|
||||||
|
} // namespace facade
|
||||||
|
|
||||||
|
|
||||||
|
namespace std {
|
||||||
|
ostream& operator<<(ostream& os, facade::CmdArgList args);
|
||||||
|
|
||||||
|
} // namespace std
|
|
@ -1,7 +1,7 @@
|
||||||
// Copyright 2022, Roman Gershman. All rights reserved.
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
#include "server/memcache_parser.h"
|
#include "facade/memcache_parser.h"
|
||||||
|
|
||||||
#include <absl/container/flat_hash_map.h>
|
#include <absl/container/flat_hash_map.h>
|
||||||
#include <absl/strings/ascii.h>
|
#include <absl/strings/ascii.h>
|
||||||
|
@ -9,7 +9,7 @@
|
||||||
|
|
||||||
#include "base/stl_util.h"
|
#include "base/stl_util.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
using namespace std;
|
using namespace std;
|
||||||
using MP = MemcacheParser;
|
using MP = MemcacheParser;
|
||||||
|
|
|
@ -7,7 +7,7 @@
|
||||||
#include <string_view>
|
#include <string_view>
|
||||||
#include <vector>
|
#include <vector>
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
|
|
||||||
// Memcache parser does not parse value blobs, only the commands.
|
// Memcache parser does not parse value blobs, only the commands.
|
||||||
// The expectation is that the caller will parse the command and
|
// The expectation is that the caller will parse the command and
|
|
@ -2,23 +2,22 @@
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
#include "server/memcache_parser.h"
|
#include "facade/memcache_parser.h"
|
||||||
|
|
||||||
#include <gmock/gmock.h>
|
#include <gmock/gmock.h>
|
||||||
|
|
||||||
#include "absl/strings/str_cat.h"
|
#include "absl/strings/str_cat.h"
|
||||||
#include "base/gtest.h"
|
#include "base/gtest.h"
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
#include "server/test_utils.h"
|
#include "facade/facade_test.h"
|
||||||
|
|
||||||
using namespace testing;
|
using namespace testing;
|
||||||
using namespace std;
|
using namespace std;
|
||||||
namespace dfly {
|
|
||||||
|
namespace facade {
|
||||||
|
|
||||||
class MCParserTest : public testing::Test {
|
class MCParserTest : public testing::Test {
|
||||||
protected:
|
protected:
|
||||||
RedisParser::Result Parse(std::string_view str);
|
|
||||||
|
|
||||||
MemcacheParser parser_;
|
MemcacheParser parser_;
|
||||||
MemcacheParser::Command cmd_;
|
MemcacheParser::Command cmd_;
|
||||||
uint32_t consumed_;
|
uint32_t consumed_;
|
||||||
|
@ -26,7 +25,6 @@ class MCParserTest : public testing::Test {
|
||||||
unique_ptr<uint8_t[]> stash_;
|
unique_ptr<uint8_t[]> stash_;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
TEST_F(MCParserTest, Basic) {
|
TEST_F(MCParserTest, Basic) {
|
||||||
MemcacheParser::Result st = parser_.Parse("set a 1 20 3\r\n", &consumed_, &cmd_);
|
MemcacheParser::Result st = parser_.Parse("set a 1 20 3\r\n", &consumed_, &cmd_);
|
||||||
EXPECT_EQ(MemcacheParser::OK, st);
|
EXPECT_EQ(MemcacheParser::OK, st);
|
||||||
|
@ -80,4 +78,4 @@ TEST_F(MCParserTest, Stats) {
|
||||||
EXPECT_EQ(MemcacheParser::PARSE_ERROR, st);
|
EXPECT_EQ(MemcacheParser::PARSE_ERROR, st);
|
||||||
}
|
}
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace facade
|
|
@ -0,0 +1,70 @@
|
||||||
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
|
// See LICENSE for licensing terms.
|
||||||
|
//
|
||||||
|
|
||||||
|
#include "base/init.h"
|
||||||
|
#include "facade/conn_context.h"
|
||||||
|
#include "facade/dragonfly_listener.h"
|
||||||
|
#include "facade/service_interface.h"
|
||||||
|
#include "util/accept_server.h"
|
||||||
|
#include "util/uring/uring_pool.h"
|
||||||
|
|
||||||
|
DEFINE_uint32(port, 6379, "server port");
|
||||||
|
|
||||||
|
using namespace util;
|
||||||
|
using namespace std;
|
||||||
|
|
||||||
|
namespace facade {
|
||||||
|
|
||||||
|
namespace {
|
||||||
|
|
||||||
|
thread_local ConnectionStats tl_stats;
|
||||||
|
|
||||||
|
class OkService : public ServiceInterface {
|
||||||
|
public:
|
||||||
|
void DispatchCommand(CmdArgList args, ConnectionContext* cntx) final {
|
||||||
|
(*cntx)->SendOk();
|
||||||
|
}
|
||||||
|
|
||||||
|
void DispatchMC(const MemcacheParser::Command& cmd, std::string_view value,
|
||||||
|
ConnectionContext* cntx) final {
|
||||||
|
cntx->reply_builder()->SendError("");
|
||||||
|
}
|
||||||
|
|
||||||
|
ConnectionContext* CreateContext(util::FiberSocketBase* peer, Connection* owner) final {
|
||||||
|
return new ConnectionContext{peer, owner};
|
||||||
|
}
|
||||||
|
|
||||||
|
ConnectionStats* GetThreadLocalConnectionStats() final {
|
||||||
|
return &tl_stats;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
void RunEngine(ProactorPool* pool, AcceptServer* acceptor) {
|
||||||
|
OkService service;
|
||||||
|
|
||||||
|
acceptor->AddListener(FLAGS_port, new Listener{Protocol::REDIS, &service});
|
||||||
|
|
||||||
|
acceptor->Run();
|
||||||
|
acceptor->Wait();
|
||||||
|
}
|
||||||
|
|
||||||
|
} // namespace
|
||||||
|
|
||||||
|
} // namespace facade
|
||||||
|
|
||||||
|
int main(int argc, char* argv[]) {
|
||||||
|
MainInitGuard guard(&argc, &argv);
|
||||||
|
|
||||||
|
CHECK_GT(FLAGS_port, 0u);
|
||||||
|
|
||||||
|
uring::UringPool pp{1024};
|
||||||
|
pp.Run();
|
||||||
|
|
||||||
|
AcceptServer acceptor(&pp);
|
||||||
|
facade::RunEngine(&pp, &acceptor);
|
||||||
|
|
||||||
|
pp.Stop();
|
||||||
|
|
||||||
|
return 0;
|
||||||
|
}
|
|
@ -1,4 +1,4 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
|
@ -6,7 +6,7 @@
|
||||||
|
|
||||||
#include <ostream>
|
#include <ostream>
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
|
|
||||||
enum class OpStatus : uint16_t {
|
enum class OpStatus : uint16_t {
|
||||||
OK,
|
OK,
|
||||||
|
@ -84,16 +84,16 @@ inline bool operator==(OpStatus st, const OpResultBase& ob) {
|
||||||
return ob.operator==(st);
|
return ob.operator==(st);
|
||||||
}
|
}
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace facade
|
||||||
|
|
||||||
namespace std {
|
namespace std {
|
||||||
|
|
||||||
template <typename T> std::ostream& operator<<(std::ostream& os, const dfly::OpResult<T>& res) {
|
template <typename T> std::ostream& operator<<(std::ostream& os, const facade::OpResult<T>& res) {
|
||||||
os << int(res.status());
|
os << int(res.status());
|
||||||
return os;
|
return os;
|
||||||
}
|
}
|
||||||
|
|
||||||
inline std::ostream& operator<<(std::ostream& os, const dfly::OpStatus op) {
|
inline std::ostream& operator<<(std::ostream& os, const facade::OpStatus op) {
|
||||||
os << int(op);
|
os << int(op);
|
||||||
return os;
|
return os;
|
||||||
}
|
}
|
|
@ -1,13 +1,13 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
// Copyright 2021, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
#include "server/redis_parser.h"
|
#include "facade/redis_parser.h"
|
||||||
|
|
||||||
#include <absl/strings/numbers.h>
|
#include <absl/strings/numbers.h>
|
||||||
|
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
|
|
||||||
using namespace std;
|
using namespace std;
|
||||||
|
|
|
@ -5,9 +5,9 @@
|
||||||
|
|
||||||
#include <absl/container/inlined_vector.h>
|
#include <absl/container/inlined_vector.h>
|
||||||
|
|
||||||
#include "core/resp_expr.h"
|
#include "facade/resp_expr.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @brief Zero-copy (best-effort) parser.
|
* @brief Zero-copy (best-effort) parser.
|
||||||
|
@ -15,14 +15,7 @@ namespace dfly {
|
||||||
*/
|
*/
|
||||||
class RedisParser {
|
class RedisParser {
|
||||||
public:
|
public:
|
||||||
enum Result {
|
enum Result { OK, INPUT_PENDING, BAD_ARRAYLEN, BAD_BULKLEN, BAD_STRING, BAD_INT };
|
||||||
OK,
|
|
||||||
INPUT_PENDING,
|
|
||||||
BAD_ARRAYLEN,
|
|
||||||
BAD_BULKLEN,
|
|
||||||
BAD_STRING,
|
|
||||||
BAD_INT
|
|
||||||
};
|
|
||||||
using Buffer = RespExpr::Buffer;
|
using Buffer = RespExpr::Buffer;
|
||||||
|
|
||||||
explicit RedisParser(bool server_mode = true) : server_mode_(server_mode) {
|
explicit RedisParser(bool server_mode = true) : server_mode_(server_mode) {
|
||||||
|
@ -51,8 +44,12 @@ class RedisParser {
|
||||||
return bulk_len_;
|
return bulk_len_;
|
||||||
}
|
}
|
||||||
|
|
||||||
size_t stash_size() const { return stash_.size(); }
|
size_t stash_size() const {
|
||||||
const std::vector<std::unique_ptr<RespVec>>& stash() const { return stash_;}
|
return stash_.size();
|
||||||
|
}
|
||||||
|
const std::vector<std::unique_ptr<RespVec>>& stash() const {
|
||||||
|
return stash_;
|
||||||
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
void InitStart(uint8_t prefix_b, RespVec* res);
|
void InitStart(uint8_t prefix_b, RespVec* res);
|
||||||
|
@ -97,4 +94,4 @@ class RedisParser {
|
||||||
bool server_mode_ = true;
|
bool server_mode_ = true;
|
||||||
};
|
};
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace facade
|
|
@ -2,12 +2,7 @@
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
#include "server/redis_parser.h"
|
#include "facade/redis_parser.h"
|
||||||
|
|
||||||
extern "C" {
|
|
||||||
#include "redis/sds.h"
|
|
||||||
#include "redis/zmalloc.h"
|
|
||||||
}
|
|
||||||
|
|
||||||
#include <absl/strings/str_cat.h>
|
#include <absl/strings/str_cat.h>
|
||||||
#include <gmock/gmock.h>
|
#include <gmock/gmock.h>
|
||||||
|
@ -15,11 +10,11 @@ extern "C" {
|
||||||
#include "absl/strings/str_cat.h"
|
#include "absl/strings/str_cat.h"
|
||||||
#include "base/gtest.h"
|
#include "base/gtest.h"
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
#include "server/test_utils.h"
|
#include "facade/facade_test.h"
|
||||||
|
|
||||||
using namespace testing;
|
using namespace testing;
|
||||||
using namespace std;
|
using namespace std;
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
|
|
||||||
MATCHER_P(ArrArg, expected, absl::StrCat(negation ? "is not" : "is", " equal to:\n", expected)) {
|
MATCHER_P(ArrArg, expected, absl::StrCat(negation ? "is not" : "is", " equal to:\n", expected)) {
|
||||||
if (arg.type != RespExpr::ARRAY) {
|
if (arg.type != RespExpr::ARRAY) {
|
||||||
|
@ -39,7 +34,6 @@ MATCHER_P(ArrArg, expected, absl::StrCat(negation ? "is not" : "is", " equal to:
|
||||||
class RedisParserTest : public testing::Test {
|
class RedisParserTest : public testing::Test {
|
||||||
protected:
|
protected:
|
||||||
static void SetUpTestSuite() {
|
static void SetUpTestSuite() {
|
||||||
init_zmalloc_threadlocal();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
RedisParser::Result Parse(std::string_view str);
|
RedisParser::Result Parse(std::string_view str);
|
||||||
|
@ -91,29 +85,6 @@ TEST_F(RedisParserTest, Inline) {
|
||||||
EXPECT_EQ(2, consumed_);
|
EXPECT_EQ(2, consumed_);
|
||||||
}
|
}
|
||||||
|
|
||||||
TEST_F(RedisParserTest, Sds) {
|
|
||||||
int argc;
|
|
||||||
sds* argv = sdssplitargs("\r\n",&argc);
|
|
||||||
EXPECT_EQ(0, argc);
|
|
||||||
sdsfreesplitres(argv,argc);
|
|
||||||
|
|
||||||
argv = sdssplitargs("\026 \020 \200 \277 \r\n",&argc);
|
|
||||||
EXPECT_EQ(4, argc);
|
|
||||||
EXPECT_STREQ("\026", argv[0]);
|
|
||||||
sdsfreesplitres(argv,argc);
|
|
||||||
|
|
||||||
argv = sdssplitargs(R"(abc "oops\n" )""\r\n",&argc);
|
|
||||||
EXPECT_EQ(2, argc);
|
|
||||||
EXPECT_STREQ("oops\n", argv[1]);
|
|
||||||
sdsfreesplitres(argv,argc);
|
|
||||||
|
|
||||||
argv = sdssplitargs(R"( "abc\xf0" )" "\t'oops\n' \r\n",&argc);
|
|
||||||
ASSERT_EQ(2, argc);
|
|
||||||
EXPECT_STREQ("abc\xf0", argv[0]);
|
|
||||||
EXPECT_STREQ("oops\n", argv[1]);
|
|
||||||
sdsfreesplitres(argv,argc);
|
|
||||||
}
|
|
||||||
|
|
||||||
TEST_F(RedisParserTest, InlineEscaping) {
|
TEST_F(RedisParserTest, InlineEscaping) {
|
||||||
LOG(ERROR) << "TBD: to be compliant with sdssplitargs"; // TODO:
|
LOG(ERROR) << "TBD: to be compliant with sdssplitargs"; // TODO:
|
||||||
}
|
}
|
||||||
|
@ -217,4 +188,4 @@ TEST_F(RedisParserTest, LargeBulk) {
|
||||||
ASSERT_EQ(RedisParser::OK, Parse("\r\n"));
|
ASSERT_EQ(RedisParser::OK, Parse("\r\n"));
|
||||||
}
|
}
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace facade
|
|
@ -1,18 +1,18 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
// Copyright 2021, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
#include "server/reply_builder.h"
|
#include "facade/reply_builder.h"
|
||||||
|
|
||||||
#include <absl/strings/numbers.h>
|
#include <absl/strings/numbers.h>
|
||||||
#include <absl/strings/str_cat.h>
|
#include <absl/strings/str_cat.h>
|
||||||
|
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
#include "server/error.h"
|
#include "facade/error.h"
|
||||||
|
|
||||||
using namespace std;
|
using namespace std;
|
||||||
using absl::StrAppend;
|
using absl::StrAppend;
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
|
|
||||||
namespace {
|
namespace {
|
||||||
|
|
|
@ -4,10 +4,10 @@
|
||||||
#include <optional>
|
#include <optional>
|
||||||
#include <string_view>
|
#include <string_view>
|
||||||
|
|
||||||
#include "core/op_status.h"
|
#include "facade/op_status.h"
|
||||||
#include "io/sync_stream_interface.h"
|
#include "io/sync_stream_interface.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
|
|
||||||
class ReplyBuilderInterface {
|
class ReplyBuilderInterface {
|
||||||
public:
|
public:
|
|
@ -10,7 +10,7 @@
|
||||||
#include <variant>
|
#include <variant>
|
||||||
#include <vector>
|
#include <vector>
|
||||||
|
|
||||||
namespace dfly {
|
namespace facade {
|
||||||
|
|
||||||
class RespExpr {
|
class RespExpr {
|
||||||
public:
|
public:
|
||||||
|
@ -31,7 +31,9 @@ class RespExpr {
|
||||||
return Buffer{reinterpret_cast<uint8_t*>(s->data()), s->size()};
|
return Buffer{reinterpret_cast<uint8_t*>(s->data()), s->size()};
|
||||||
}
|
}
|
||||||
|
|
||||||
Buffer GetBuf() const { return std::get<Buffer>(u); }
|
Buffer GetBuf() const {
|
||||||
|
return std::get<Buffer>(u);
|
||||||
|
}
|
||||||
|
|
||||||
static const char* TypeName(Type t);
|
static const char* TypeName(Type t);
|
||||||
};
|
};
|
||||||
|
@ -43,11 +45,11 @@ inline std::string_view ToSV(const absl::Span<uint8_t>& s) {
|
||||||
return std::string_view{reinterpret_cast<char*>(s.data()), s.size()};
|
return std::string_view{reinterpret_cast<char*>(s.data()), s.size()};
|
||||||
}
|
}
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace facade
|
||||||
|
|
||||||
namespace std {
|
namespace std {
|
||||||
|
|
||||||
ostream& operator<<(ostream& os, const dfly::RespExpr& e);
|
ostream& operator<<(ostream& os, const facade::RespExpr& e);
|
||||||
ostream& operator<<(ostream& os, dfly::RespSpan rspan);
|
ostream& operator<<(ostream& os, facade::RespSpan rspan);
|
||||||
|
|
||||||
} // namespace std
|
} // namespace std
|
|
@ -0,0 +1,31 @@
|
||||||
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
|
// See LICENSE for licensing terms.
|
||||||
|
//
|
||||||
|
|
||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include "facade/facade_types.h"
|
||||||
|
#include "facade/memcache_parser.h"
|
||||||
|
|
||||||
|
#include "util/fiber_socket_base.h"
|
||||||
|
|
||||||
|
namespace facade {
|
||||||
|
|
||||||
|
class ConnectionContext;
|
||||||
|
class Connection;
|
||||||
|
class ConnectionStats;
|
||||||
|
|
||||||
|
class ServiceInterface {
|
||||||
|
public:
|
||||||
|
virtual ~ServiceInterface() {
|
||||||
|
}
|
||||||
|
|
||||||
|
virtual void DispatchCommand(CmdArgList args, ConnectionContext* cntx) = 0;
|
||||||
|
virtual void DispatchMC(const MemcacheParser::Command& cmd, std::string_view value,
|
||||||
|
ConnectionContext* cntx) = 0;
|
||||||
|
|
||||||
|
virtual ConnectionContext* CreateContext(util::FiberSocketBase* peer, Connection* owner) = 0;
|
||||||
|
virtual ConnectionStats* GetThreadLocalConnectionStats() = 0;
|
||||||
|
};
|
||||||
|
|
||||||
|
} // namespace facade
|
|
@ -2,25 +2,22 @@ add_executable(dragonfly dfly_main.cc)
|
||||||
cxx_link(dragonfly base dragonfly_lib)
|
cxx_link(dragonfly base dragonfly_lib)
|
||||||
|
|
||||||
add_library(dragonfly_lib command_registry.cc common.cc config_flags.cc
|
add_library(dragonfly_lib command_registry.cc common.cc config_flags.cc
|
||||||
conn_context.cc db_slice.cc debugcmd.cc dragonfly_listener.cc
|
db_slice.cc debugcmd.cc
|
||||||
dragonfly_connection.cc engine_shard_set.cc generic_family.cc hset_family.cc
|
engine_shard_set.cc generic_family.cc hset_family.cc
|
||||||
list_family.cc main_service.cc memcache_parser.cc rdb_load.cc rdb_save.cc replica.cc
|
list_family.cc main_service.cc rdb_load.cc rdb_save.cc replica.cc
|
||||||
snapshot.cc redis_parser.cc reply_builder.cc script_mgr.cc server_family.cc
|
snapshot.cc script_mgr.cc server_family.cc
|
||||||
set_family.cc
|
set_family.cc
|
||||||
string_family.cc transaction.cc zset_family.cc)
|
string_family.cc transaction.cc zset_family.cc)
|
||||||
|
|
||||||
cxx_link(dragonfly_lib dfly_core redis_lib uring_fiber_lib
|
cxx_link(dragonfly_lib dfly_core dfly_facade redis_lib strings_lib)
|
||||||
fibers_ext strings_lib http_server_lib tls_lib)
|
|
||||||
|
|
||||||
add_library(dfly_test_lib test_utils.cc)
|
add_library(dfly_test_lib test_utils.cc)
|
||||||
cxx_link(dfly_test_lib dragonfly_lib gtest_main_ext)
|
cxx_link(dfly_test_lib dragonfly_lib facade_test gtest_main_ext)
|
||||||
|
|
||||||
cxx_test(dragonfly_test dfly_test_lib LABELS DFLY)
|
cxx_test(dragonfly_test dfly_test_lib LABELS DFLY)
|
||||||
cxx_test(generic_family_test dfly_test_lib LABELS DFLY)
|
cxx_test(generic_family_test dfly_test_lib LABELS DFLY)
|
||||||
cxx_test(hset_family_test dfly_test_lib LABELS DFLY)
|
cxx_test(hset_family_test dfly_test_lib LABELS DFLY)
|
||||||
cxx_test(list_family_test dfly_test_lib LABELS DFLY)
|
cxx_test(list_family_test dfly_test_lib LABELS DFLY)
|
||||||
cxx_test(memcache_parser_test dfly_test_lib LABELS DFLY)
|
|
||||||
cxx_test(redis_parser_test dfly_test_lib LABELS DFLY)
|
|
||||||
cxx_test(set_family_test dfly_test_lib LABELS DFLY)
|
cxx_test(set_family_test dfly_test_lib LABELS DFLY)
|
||||||
cxx_test(string_family_test dfly_test_lib LABELS DFLY)
|
cxx_test(string_family_test dfly_test_lib LABELS DFLY)
|
||||||
cxx_test(rdb_test dfly_test_lib DATA testdata/empty.rdb testdata/small.rdb LABELS DFLY)
|
cxx_test(rdb_test dfly_test_lib DATA testdata/empty.rdb testdata/small.rdb LABELS DFLY)
|
||||||
|
|
|
@ -38,41 +38,6 @@ Interpreter& ServerState::GetInterpreter() {
|
||||||
return interpreter_.value();
|
return interpreter_.value();
|
||||||
}
|
}
|
||||||
|
|
||||||
#define ADD(x) (x) += o.x
|
|
||||||
|
|
||||||
ConnectionStats& ConnectionStats::operator+=(const ConnectionStats& o) {
|
|
||||||
// To break this code deliberately if we add/remove a field to this struct.
|
|
||||||
static_assert(sizeof(ConnectionStats) == 64);
|
|
||||||
|
|
||||||
ADD(num_conns);
|
|
||||||
ADD(num_replicas);
|
|
||||||
ADD(read_buf_capacity);
|
|
||||||
ADD(io_read_cnt);
|
|
||||||
ADD(io_read_bytes);
|
|
||||||
ADD(io_write_cnt);
|
|
||||||
ADD(io_write_bytes);
|
|
||||||
ADD(pipelined_cmd_cnt);
|
|
||||||
ADD(command_cnt);
|
|
||||||
|
|
||||||
return *this;
|
|
||||||
}
|
|
||||||
|
|
||||||
#undef ADD
|
|
||||||
|
|
||||||
string WrongNumArgsError(std::string_view cmd) {
|
|
||||||
return absl::StrCat("wrong number of arguments for '", cmd, "' command");
|
|
||||||
}
|
|
||||||
|
|
||||||
const char kSyntaxErr[] = "syntax error";
|
|
||||||
const char kWrongTypeErr[] = "-WRONGTYPE Operation against a key holding the wrong kind of value";
|
|
||||||
const char kKeyNotFoundErr[] = "no such key";
|
|
||||||
const char kInvalidIntErr[] = "value is not an integer or out of range";
|
|
||||||
const char kUintErr[] = "value is out of range, must be positive";
|
|
||||||
const char kDbIndOutOfRangeErr[] = "DB index is out of range";
|
|
||||||
const char kInvalidDbIndErr[] = "invalid DB index";
|
|
||||||
const char kScriptNotFound[] = "-NOSCRIPT No matching script. Please use EVAL.";
|
|
||||||
const char kAuthRejected[] = "-WRONGPASS invalid username-password pair or user is disabled.";
|
|
||||||
|
|
||||||
const char* GlobalState::Name(S s) {
|
const char* GlobalState::Name(S s) {
|
||||||
switch (s) {
|
switch (s) {
|
||||||
case GlobalState::IDLE:
|
case GlobalState::IDLE:
|
||||||
|
@ -88,20 +53,3 @@ const char* GlobalState::Name(S s) {
|
||||||
}
|
}
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace dfly
|
||||||
|
|
||||||
namespace std {
|
|
||||||
|
|
||||||
ostream& operator<<(ostream& os, dfly::CmdArgList ras) {
|
|
||||||
os << "[";
|
|
||||||
if (!ras.empty()) {
|
|
||||||
for (size_t i = 0; i < ras.size() - 1; ++i) {
|
|
||||||
os << dfly::ArgS(ras, i) << ",";
|
|
||||||
}
|
|
||||||
os << dfly::ArgS(ras, ras.size() - 1);
|
|
||||||
}
|
|
||||||
os << "]";
|
|
||||||
|
|
||||||
return os;
|
|
||||||
}
|
|
||||||
|
|
||||||
} // namespace std
|
|
||||||
|
|
|
@ -1,4 +1,4 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
|
@ -9,25 +9,24 @@
|
||||||
|
|
||||||
#include <string_view>
|
#include <string_view>
|
||||||
#include <vector>
|
#include <vector>
|
||||||
|
#include "facade/facade_types.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
enum class ListDir : uint8_t { LEFT, RIGHT };
|
enum class ListDir : uint8_t { LEFT, RIGHT };
|
||||||
|
|
||||||
enum class Protocol : uint8_t {
|
|
||||||
MEMCACHE = 1,
|
|
||||||
REDIS = 2
|
|
||||||
};
|
|
||||||
|
|
||||||
using DbIndex = uint16_t;
|
using DbIndex = uint16_t;
|
||||||
using ShardId = uint16_t;
|
using ShardId = uint16_t;
|
||||||
using TxId = uint64_t;
|
using TxId = uint64_t;
|
||||||
using TxClock = uint64_t;
|
using TxClock = uint64_t;
|
||||||
|
|
||||||
|
using facade::MutableSlice;
|
||||||
|
using facade::CmdArgList;
|
||||||
|
using facade::CmdArgVec;
|
||||||
|
using facade::ArgS;
|
||||||
|
|
||||||
using ArgSlice = absl::Span<const std::string_view>;
|
using ArgSlice = absl::Span<const std::string_view>;
|
||||||
using MutableSlice = absl::Span<char>;
|
|
||||||
using CmdArgList = absl::Span<MutableSlice>;
|
|
||||||
using CmdArgVec = std::vector<MutableSlice>;
|
|
||||||
|
|
||||||
constexpr DbIndex kInvalidDbId = DbIndex(-1);
|
constexpr DbIndex kInvalidDbId = DbIndex(-1);
|
||||||
constexpr ShardId kInvalidSid = ShardId(-1);
|
constexpr ShardId kInvalidSid = ShardId(-1);
|
||||||
|
@ -50,42 +49,11 @@ struct KeyIndex {
|
||||||
unsigned step; // 1 for commands like mget. 2 for commands like mset.
|
unsigned step; // 1 for commands like mget. 2 for commands like mset.
|
||||||
};
|
};
|
||||||
|
|
||||||
struct ConnectionStats {
|
|
||||||
uint32_t num_conns = 0;
|
|
||||||
uint32_t num_replicas = 0;
|
|
||||||
size_t read_buf_capacity = 0;
|
|
||||||
size_t io_read_cnt = 0;
|
|
||||||
size_t io_read_bytes = 0;
|
|
||||||
size_t io_write_cnt = 0;
|
|
||||||
size_t io_write_bytes = 0;
|
|
||||||
size_t command_cnt = 0;
|
|
||||||
size_t pipelined_cmd_cnt = 0;
|
|
||||||
|
|
||||||
ConnectionStats& operator+=(const ConnectionStats& o);
|
|
||||||
};
|
|
||||||
|
|
||||||
struct OpArgs {
|
struct OpArgs {
|
||||||
EngineShard* shard;
|
EngineShard* shard;
|
||||||
DbIndex db_ind;
|
DbIndex db_ind;
|
||||||
};
|
};
|
||||||
|
|
||||||
constexpr inline unsigned long long operator""_MB(unsigned long long x) {
|
|
||||||
return 1024L * 1024L * x;
|
|
||||||
}
|
|
||||||
|
|
||||||
constexpr inline unsigned long long operator""_KB(unsigned long long x) {
|
|
||||||
return 1024L * x;
|
|
||||||
}
|
|
||||||
|
|
||||||
inline std::string_view ArgS(CmdArgList args, size_t i) {
|
|
||||||
auto arg = args[i];
|
|
||||||
return std::string_view(arg.data(), arg.size());
|
|
||||||
}
|
|
||||||
|
|
||||||
inline MutableSlice ToMSS(absl::Span<uint8_t> span) {
|
|
||||||
return MutableSlice{reinterpret_cast<char*>(span.data()), span.size()};
|
|
||||||
}
|
|
||||||
|
|
||||||
inline void ToUpper(const MutableSlice* val) {
|
inline void ToUpper(const MutableSlice* val) {
|
||||||
for (auto& c : *val) {
|
for (auto& c : *val) {
|
||||||
c = absl::ascii_toupper(c);
|
c = absl::ascii_toupper(c);
|
||||||
|
@ -99,8 +67,3 @@ inline void ToLower(const MutableSlice* val) {
|
||||||
}
|
}
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace dfly
|
||||||
|
|
||||||
namespace std {
|
|
||||||
ostream& operator<<(ostream& os, dfly::CmdArgList args);
|
|
||||||
|
|
||||||
} // namespace std
|
|
||||||
|
|
|
@ -1,33 +0,0 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
|
||||||
// See LICENSE for licensing terms.
|
|
||||||
//
|
|
||||||
|
|
||||||
#include "server/conn_context.h"
|
|
||||||
|
|
||||||
#include "base/logging.h"
|
|
||||||
#include "server/dragonfly_connection.h"
|
|
||||||
|
|
||||||
namespace dfly {
|
|
||||||
|
|
||||||
ConnectionContext::ConnectionContext(::io::Sink* stream, Connection* owner) : owner_(owner) {
|
|
||||||
switch (owner->protocol()) {
|
|
||||||
case Protocol::REDIS:
|
|
||||||
rbuilder_.reset(new RedisReplyBuilder(stream));
|
|
||||||
break;
|
|
||||||
case Protocol::MEMCACHE:
|
|
||||||
rbuilder_.reset(new MCReplyBuilder(stream));
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
Protocol ConnectionContext::protocol() const {
|
|
||||||
return owner_->protocol();
|
|
||||||
}
|
|
||||||
|
|
||||||
RedisReplyBuilder* ConnectionContext::operator->() {
|
|
||||||
CHECK(Protocol::REDIS == protocol());
|
|
||||||
|
|
||||||
return static_cast<RedisReplyBuilder*>(rbuilder_.get());
|
|
||||||
}
|
|
||||||
|
|
||||||
} // namespace dfly
|
|
|
@ -1,4 +1,4 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
|
@ -6,12 +6,11 @@
|
||||||
|
|
||||||
#include <absl/container/flat_hash_set.h>
|
#include <absl/container/flat_hash_set.h>
|
||||||
|
|
||||||
|
#include "facade/conn_context.h"
|
||||||
#include "server/common_types.h"
|
#include "server/common_types.h"
|
||||||
#include "server/reply_builder.h"
|
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
class Connection;
|
|
||||||
class EngineShardSet;
|
class EngineShardSet;
|
||||||
|
|
||||||
struct StoredCmd {
|
struct StoredCmd {
|
||||||
|
@ -30,19 +29,6 @@ struct ConnectionState {
|
||||||
ExecState exec_state = EXEC_INACTIVE;
|
ExecState exec_state = EXEC_INACTIVE;
|
||||||
std::vector<StoredCmd> exec_body;
|
std::vector<StoredCmd> exec_body;
|
||||||
|
|
||||||
enum Mask : uint32_t {
|
|
||||||
ASYNC_DISPATCH = 1, // whether a command is handled via async dispatch.
|
|
||||||
CONN_CLOSING = 2, // could be because of unrecoverable error or planned action.
|
|
||||||
|
|
||||||
// Whether this connection belongs to replica, i.e. a dragonfly slave is connected to this
|
|
||||||
// host (master) via this connection to sync from it.
|
|
||||||
REPL_CONNECTION = 4,
|
|
||||||
REQ_AUTH = 8,
|
|
||||||
AUTHENTICATED = 0x10,
|
|
||||||
};
|
|
||||||
|
|
||||||
uint32_t mask = 0; // A bitmask of Mask values.
|
|
||||||
|
|
||||||
enum MCGetMask {
|
enum MCGetMask {
|
||||||
FETCH_CAS_VER = 1,
|
FETCH_CAS_VER = 1,
|
||||||
};
|
};
|
||||||
|
@ -52,14 +38,6 @@ struct ConnectionState {
|
||||||
// For get op - we use it as a mask of MCGetMask values.
|
// For get op - we use it as a mask of MCGetMask values.
|
||||||
uint32_t memcache_flag = 0;
|
uint32_t memcache_flag = 0;
|
||||||
|
|
||||||
bool IsClosing() const {
|
|
||||||
return mask & CONN_CLOSING;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool IsRunViaDispatch() const {
|
|
||||||
return mask & ASYNC_DISPATCH;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Lua-script related data.
|
// Lua-script related data.
|
||||||
struct Script {
|
struct Script {
|
||||||
bool is_write = true;
|
bool is_write = true;
|
||||||
|
@ -69,10 +47,11 @@ struct ConnectionState {
|
||||||
std::optional<Script> script_info;
|
std::optional<Script> script_info;
|
||||||
};
|
};
|
||||||
|
|
||||||
class ConnectionContext {
|
class ConnectionContext : public facade::ConnectionContext {
|
||||||
public:
|
public:
|
||||||
ConnectionContext(::io::Sink* stream, Connection* owner);
|
ConnectionContext(::io::Sink* stream, facade::Connection* owner)
|
||||||
|
: facade::ConnectionContext(stream, owner) {
|
||||||
|
}
|
||||||
struct DebugInfo {
|
struct DebugInfo {
|
||||||
uint32_t shards_count = 0;
|
uint32_t shards_count = 0;
|
||||||
TxClock clock = 0;
|
TxClock clock = 0;
|
||||||
|
@ -85,36 +64,11 @@ class ConnectionContext {
|
||||||
Transaction* transaction = nullptr;
|
Transaction* transaction = nullptr;
|
||||||
const CommandId* cid = nullptr;
|
const CommandId* cid = nullptr;
|
||||||
EngineShardSet* shard_set = nullptr;
|
EngineShardSet* shard_set = nullptr;
|
||||||
|
ConnectionState conn_state;
|
||||||
Connection* owner() {
|
|
||||||
return owner_;
|
|
||||||
}
|
|
||||||
|
|
||||||
Protocol protocol() const;
|
|
||||||
|
|
||||||
DbIndex db_index() const {
|
DbIndex db_index() const {
|
||||||
return conn_state.db_index;
|
return conn_state.db_index;
|
||||||
}
|
}
|
||||||
|
|
||||||
ConnectionState conn_state;
|
|
||||||
|
|
||||||
// A convenient proxy for redis interface.
|
|
||||||
RedisReplyBuilder* operator->();
|
|
||||||
|
|
||||||
ReplyBuilderInterface* reply_builder() {
|
|
||||||
return rbuilder_.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
// Allows receiving the output data from the commands called from scripts.
|
|
||||||
ReplyBuilderInterface* Inject(ReplyBuilderInterface* new_i) {
|
|
||||||
ReplyBuilderInterface* res = rbuilder_.release();
|
|
||||||
rbuilder_.reset(new_i);
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
|
||||||
Connection* owner_;
|
|
||||||
std::unique_ptr<ReplyBuilderInterface> rbuilder_;
|
|
||||||
};
|
};
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace dfly
|
||||||
|
|
|
@ -1,4 +1,4 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
|
@ -21,6 +21,7 @@ namespace dfly {
|
||||||
using namespace boost;
|
using namespace boost;
|
||||||
using namespace std;
|
using namespace std;
|
||||||
using namespace util;
|
using namespace util;
|
||||||
|
using facade::OpStatus;
|
||||||
|
|
||||||
#define ADD(x) (x) += o.x
|
#define ADD(x) (x) += o.x
|
||||||
|
|
||||||
|
|
|
@ -8,7 +8,7 @@
|
||||||
#include <absl/container/flat_hash_set.h>
|
#include <absl/container/flat_hash_set.h>
|
||||||
|
|
||||||
#include "core/intent_lock.h"
|
#include "core/intent_lock.h"
|
||||||
#include "core/op_status.h"
|
#include "facade/op_status.h"
|
||||||
#include "server/common_types.h"
|
#include "server/common_types.h"
|
||||||
#include "server/table.h"
|
#include "server/table.h"
|
||||||
|
|
||||||
|
@ -18,6 +18,8 @@ class ProactorBase;
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
|
using facade::OpResult;
|
||||||
|
|
||||||
struct DbStats {
|
struct DbStats {
|
||||||
// number of active keys.
|
// number of active keys.
|
||||||
size_t key_count = 0;
|
size_t key_count = 0;
|
||||||
|
|
|
@ -25,6 +25,7 @@ using namespace std;
|
||||||
using namespace util;
|
using namespace util;
|
||||||
namespace this_fiber = ::boost::this_fiber;
|
namespace this_fiber = ::boost::this_fiber;
|
||||||
using boost::fibers::fiber;
|
using boost::fibers::fiber;
|
||||||
|
using facade::kUintErr;
|
||||||
namespace fs = std::filesystem;
|
namespace fs = std::filesystem;
|
||||||
|
|
||||||
struct PopulateBatch {
|
struct PopulateBatch {
|
||||||
|
|
|
@ -1,9 +1,9 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
#include "base/init.h"
|
#include "base/init.h"
|
||||||
#include "server/dragonfly_listener.h"
|
#include "facade/dragonfly_listener.h"
|
||||||
#include "server/main_service.h"
|
#include "server/main_service.h"
|
||||||
#include "util/accept_server.h"
|
#include "util/accept_server.h"
|
||||||
#include "util/uring/uring_pool.h"
|
#include "util/uring/uring_pool.h"
|
||||||
|
@ -14,13 +14,13 @@ DECLARE_uint32(memcache_port);
|
||||||
|
|
||||||
using namespace util;
|
using namespace util;
|
||||||
using namespace std;
|
using namespace std;
|
||||||
|
using namespace facade;
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
void RunEngine(ProactorPool* pool, AcceptServer* acceptor, HttpListener<>* http) {
|
void RunEngine(ProactorPool* pool, AcceptServer* acceptor) {
|
||||||
Service service(pool);
|
Service service(pool);
|
||||||
|
|
||||||
service.RegisterHttp(http);
|
|
||||||
service.Init(acceptor);
|
service.Init(acceptor);
|
||||||
acceptor->AddListener(FLAGS_port, new Listener{Protocol::REDIS, &service});
|
acceptor->AddListener(FLAGS_port, new Listener{Protocol::REDIS, &service});
|
||||||
if (FLAGS_memcache_port > 0) {
|
if (FLAGS_memcache_port > 0) {
|
||||||
|
@ -47,11 +47,8 @@ int main(int argc, char* argv[]) {
|
||||||
pp.Run();
|
pp.Run();
|
||||||
|
|
||||||
AcceptServer acceptor(&pp);
|
AcceptServer acceptor(&pp);
|
||||||
unique_ptr<HttpListener<>> http_listener(new HttpListener<>);
|
|
||||||
|
|
||||||
http_listener->enable_metrics();
|
dfly::RunEngine(&pp, &acceptor);
|
||||||
|
|
||||||
dfly::RunEngine(&pp, &acceptor, http_listener.get());
|
|
||||||
|
|
||||||
pp.Stop();
|
pp.Stop();
|
||||||
|
|
||||||
|
|
|
@ -2,6 +2,11 @@
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
|
extern "C" {
|
||||||
|
#include "redis/sds.h"
|
||||||
|
#include "redis/zmalloc.h"
|
||||||
|
}
|
||||||
|
|
||||||
#include <absl/strings/ascii.h>
|
#include <absl/strings/ascii.h>
|
||||||
#include <absl/strings/str_join.h>
|
#include <absl/strings/str_join.h>
|
||||||
#include <absl/strings/strip.h>
|
#include <absl/strings/strip.h>
|
||||||
|
@ -9,9 +14,9 @@
|
||||||
|
|
||||||
#include "base/gtest.h"
|
#include "base/gtest.h"
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
|
#include "facade/facade_test.h"
|
||||||
#include "server/conn_context.h"
|
#include "server/conn_context.h"
|
||||||
#include "server/main_service.h"
|
#include "server/main_service.h"
|
||||||
#include "server/redis_parser.h"
|
|
||||||
#include "server/test_utils.h"
|
#include "server/test_utils.h"
|
||||||
#include "util/uring/uring_pool.h"
|
#include "util/uring/uring_pool.h"
|
||||||
|
|
||||||
|
@ -43,8 +48,41 @@ class DflyEngineTest : public BaseFamilyTest {
|
||||||
DflyEngineTest() : BaseFamilyTest() {
|
DflyEngineTest() : BaseFamilyTest() {
|
||||||
num_threads_ = kPoolThreadCount;
|
num_threads_ = kPoolThreadCount;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static void SetUpTestSuite() {
|
||||||
|
init_zmalloc_threadlocal();
|
||||||
|
}
|
||||||
|
|
||||||
};
|
};
|
||||||
|
|
||||||
|
// TODO: to implement equivalent parsing in redis parser.
|
||||||
|
TEST_F(DflyEngineTest, Sds) {
|
||||||
|
int argc;
|
||||||
|
sds* argv = sdssplitargs("\r\n", &argc);
|
||||||
|
EXPECT_EQ(0, argc);
|
||||||
|
sdsfreesplitres(argv, argc);
|
||||||
|
|
||||||
|
argv = sdssplitargs("\026 \020 \200 \277 \r\n", &argc);
|
||||||
|
EXPECT_EQ(4, argc);
|
||||||
|
EXPECT_STREQ("\026", argv[0]);
|
||||||
|
sdsfreesplitres(argv, argc);
|
||||||
|
|
||||||
|
argv = sdssplitargs(R"(abc "oops\n" )"
|
||||||
|
"\r\n",
|
||||||
|
&argc);
|
||||||
|
EXPECT_EQ(2, argc);
|
||||||
|
EXPECT_STREQ("oops\n", argv[1]);
|
||||||
|
sdsfreesplitres(argv, argc);
|
||||||
|
|
||||||
|
argv = sdssplitargs(R"( "abc\xf0" )"
|
||||||
|
"\t'oops\n' \r\n",
|
||||||
|
&argc);
|
||||||
|
ASSERT_EQ(2, argc);
|
||||||
|
EXPECT_STREQ("abc\xf0", argv[0]);
|
||||||
|
EXPECT_STREQ("oops\n", argv[1]);
|
||||||
|
sdsfreesplitres(argv, argc);
|
||||||
|
}
|
||||||
|
|
||||||
TEST_F(DflyEngineTest, Multi) {
|
TEST_F(DflyEngineTest, Multi) {
|
||||||
RespVec resp = Run({"multi"});
|
RespVec resp = Run({"multi"});
|
||||||
ASSERT_THAT(resp, RespEq("OK"));
|
ASSERT_THAT(resp, RespEq("OK"));
|
||||||
|
|
|
@ -7,19 +7,15 @@
|
||||||
#include <atomic>
|
#include <atomic>
|
||||||
#include <string>
|
#include <string>
|
||||||
|
|
||||||
|
#include "facade/error.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
std::string WrongNumArgsError(std::string_view cmd);
|
using facade::kWrongTypeErr;
|
||||||
|
using facade::kInvalidIntErr;
|
||||||
extern const char kSyntaxErr[];
|
using facade::kSyntaxErr;
|
||||||
extern const char kWrongTypeErr[];
|
using facade::kInvalidDbIndErr;
|
||||||
extern const char kKeyNotFoundErr[];
|
using facade::kDbIndOutOfRangeErr;
|
||||||
extern const char kInvalidIntErr[];
|
|
||||||
extern const char kUintErr[];
|
|
||||||
extern const char kDbIndOutOfRangeErr[];
|
|
||||||
extern const char kInvalidDbIndErr[];
|
|
||||||
extern const char kScriptNotFound[];
|
|
||||||
extern const char kAuthRejected[];
|
|
||||||
|
|
||||||
#ifndef RETURN_ON_ERR
|
#ifndef RETURN_ON_ERR
|
||||||
|
|
||||||
|
|
|
@ -1,4 +1,4 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
|
@ -20,6 +20,7 @@ DEFINE_uint32(dbnum, 16, "Number of databases");
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
using namespace std;
|
using namespace std;
|
||||||
|
using facade::Protocol;
|
||||||
|
|
||||||
namespace {
|
namespace {
|
||||||
|
|
||||||
|
@ -177,6 +178,7 @@ void GenericFamily::Del(CmdArgList args, ConnectionContext* cntx) {
|
||||||
|
|
||||||
uint32_t del_cnt = result.load(memory_order_relaxed);
|
uint32_t del_cnt = result.load(memory_order_relaxed);
|
||||||
if (is_mc) {
|
if (is_mc) {
|
||||||
|
using facade::MCReplyBuilder;
|
||||||
MCReplyBuilder* mc_builder = static_cast<MCReplyBuilder*>(cntx->reply_builder());
|
MCReplyBuilder* mc_builder = static_cast<MCReplyBuilder*>(cntx->reply_builder());
|
||||||
|
|
||||||
if (del_cnt == 0) {
|
if (del_cnt == 0) {
|
||||||
|
|
|
@ -4,7 +4,7 @@
|
||||||
|
|
||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include "core/op_status.h"
|
#include "facade/op_status.h"
|
||||||
#include "server/common_types.h"
|
#include "server/common_types.h"
|
||||||
|
|
||||||
namespace util {
|
namespace util {
|
||||||
|
@ -13,6 +13,9 @@ class ProactorPool;
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
|
using facade::OpResult;
|
||||||
|
using facade::OpStatus;
|
||||||
|
|
||||||
class ConnectionContext;
|
class ConnectionContext;
|
||||||
class CommandRegistry;
|
class CommandRegistry;
|
||||||
class EngineShard;
|
class EngineShard;
|
||||||
|
|
|
@ -6,6 +6,7 @@
|
||||||
|
|
||||||
#include "base/gtest.h"
|
#include "base/gtest.h"
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
|
#include "facade/facade_test.h"
|
||||||
#include "server/command_registry.h"
|
#include "server/command_registry.h"
|
||||||
#include "server/conn_context.h"
|
#include "server/conn_context.h"
|
||||||
#include "server/engine_shard_set.h"
|
#include "server/engine_shard_set.h"
|
||||||
|
|
|
@ -4,13 +4,14 @@
|
||||||
|
|
||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include "core/op_status.h"
|
#include "facade/op_status.h"
|
||||||
#include "server/common_types.h"
|
#include "server/common_types.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
class ConnectionContext;
|
class ConnectionContext;
|
||||||
class CommandRegistry;
|
class CommandRegistry;
|
||||||
|
using facade::OpResult;
|
||||||
|
|
||||||
class HSetFamily {
|
class HSetFamily {
|
||||||
public:
|
public:
|
||||||
|
|
|
@ -7,11 +7,13 @@
|
||||||
#include "base/gtest.h"
|
#include "base/gtest.h"
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
#include "server/test_utils.h"
|
#include "server/test_utils.h"
|
||||||
|
#include "facade/facade_test.h"
|
||||||
|
|
||||||
using namespace testing;
|
using namespace testing;
|
||||||
using namespace std;
|
using namespace std;
|
||||||
using namespace util;
|
using namespace util;
|
||||||
using namespace boost;
|
using namespace boost;
|
||||||
|
using namespace facade;
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
|
|
|
@ -4,11 +4,13 @@
|
||||||
|
|
||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include "core/op_status.h"
|
#include "facade/op_status.h"
|
||||||
#include "server/common_types.h"
|
#include "server/common_types.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
|
using facade::OpResult;
|
||||||
|
|
||||||
class ConnectionContext;
|
class ConnectionContext;
|
||||||
class CommandRegistry;
|
class CommandRegistry;
|
||||||
class EngineShard;
|
class EngineShard;
|
||||||
|
|
|
@ -8,6 +8,7 @@
|
||||||
|
|
||||||
#include "base/gtest.h"
|
#include "base/gtest.h"
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
|
#include "facade/facade_test.h"
|
||||||
#include "server/command_registry.h"
|
#include "server/command_registry.h"
|
||||||
#include "server/conn_context.h"
|
#include "server/conn_context.h"
|
||||||
#include "server/engine_shard_set.h"
|
#include "server/engine_shard_set.h"
|
||||||
|
|
|
@ -16,6 +16,8 @@ extern "C" {
|
||||||
#include <filesystem>
|
#include <filesystem>
|
||||||
|
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
|
#include "facade/dragonfly_connection.h"
|
||||||
|
#include "facade/error.h"
|
||||||
#include "server/conn_context.h"
|
#include "server/conn_context.h"
|
||||||
#include "server/error.h"
|
#include "server/error.h"
|
||||||
#include "server/generic_family.h"
|
#include "server/generic_family.h"
|
||||||
|
@ -43,6 +45,8 @@ using base::VarzValue;
|
||||||
using ::boost::intrusive_ptr;
|
using ::boost::intrusive_ptr;
|
||||||
namespace fibers = ::boost::fibers;
|
namespace fibers = ::boost::fibers;
|
||||||
namespace this_fiber = ::boost::this_fiber;
|
namespace this_fiber = ::boost::this_fiber;
|
||||||
|
using facade::MCReplyBuilder;
|
||||||
|
using facade::RedisReplyBuilder;
|
||||||
|
|
||||||
namespace {
|
namespace {
|
||||||
|
|
||||||
|
@ -259,7 +263,7 @@ bool EvalValidator(CmdArgList args, ConnectionContext* cntx) {
|
||||||
int32_t num_keys;
|
int32_t num_keys;
|
||||||
|
|
||||||
if (!absl::SimpleAtoi(num_keys_str, &num_keys) || num_keys < 0) {
|
if (!absl::SimpleAtoi(num_keys_str, &num_keys) || num_keys < 0) {
|
||||||
(*cntx)->SendError(kInvalidIntErr);
|
(*cntx)->SendError(facade::kInvalidIntErr);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -329,7 +333,7 @@ void Service::Shutdown() {
|
||||||
shard_set_.RunBlockingInParallel([&](EngineShard*) { EngineShard::DestroyThreadLocal(); });
|
shard_set_.RunBlockingInParallel([&](EngineShard*) { EngineShard::DestroyThreadLocal(); });
|
||||||
}
|
}
|
||||||
|
|
||||||
void Service::DispatchCommand(CmdArgList args, ConnectionContext* cntx) {
|
void Service::DispatchCommand(CmdArgList args, facade::ConnectionContext* cntx) {
|
||||||
CHECK(!args.empty());
|
CHECK(!args.empty());
|
||||||
DCHECK_NE(0u, shard_set_.size()) << "Init was not called";
|
DCHECK_NE(0u, shard_set_.size()) << "Init was not called";
|
||||||
|
|
||||||
|
@ -344,9 +348,10 @@ void Service::DispatchCommand(CmdArgList args, ConnectionContext* cntx) {
|
||||||
|
|
||||||
etl.RecordCmd();
|
etl.RecordCmd();
|
||||||
|
|
||||||
absl::Cleanup multi_error = [cntx] {
|
ConnectionContext* dfly_cntx = static_cast<ConnectionContext*>(cntx);
|
||||||
if (cntx->conn_state.exec_state != ConnectionState::EXEC_INACTIVE) {
|
absl::Cleanup multi_error = [dfly_cntx] {
|
||||||
cntx->conn_state.exec_state = ConnectionState::EXEC_ERROR;
|
if (dfly_cntx->conn_state.exec_state != ConnectionState::EXEC_INACTIVE) {
|
||||||
|
dfly_cntx->conn_state.exec_state = ConnectionState::EXEC_ERROR;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -362,22 +367,22 @@ void Service::DispatchCommand(CmdArgList args, ConnectionContext* cntx) {
|
||||||
|
|
||||||
string_view cmd_name{cid->name()};
|
string_view cmd_name{cid->name()};
|
||||||
|
|
||||||
if ((cntx->conn_state.mask & (ConnectionState::REQ_AUTH | ConnectionState::AUTHENTICATED)) ==
|
if (cntx->req_auth && !cntx->authenticated) {
|
||||||
ConnectionState::REQ_AUTH) {
|
|
||||||
if (cmd_name != "AUTH") {
|
if (cmd_name != "AUTH") {
|
||||||
return (*cntx)->SendError("-NOAUTH Authentication required.");
|
return (*cntx)->SendError("-NOAUTH Authentication required.");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
bool under_script = cntx->conn_state.script_info.has_value();
|
bool under_script = dfly_cntx->conn_state.script_info.has_value();
|
||||||
|
|
||||||
if (under_script && (cid->opt_mask() & CO::NOSCRIPT)) {
|
if (under_script && (cid->opt_mask() & CO::NOSCRIPT)) {
|
||||||
return (*cntx)->SendError("This Redis command is not allowed from script");
|
return (*cntx)->SendError("This Redis command is not allowed from script");
|
||||||
}
|
}
|
||||||
|
|
||||||
bool is_write_cmd =
|
bool is_write_cmd = (cid->opt_mask() & CO::WRITE) ||
|
||||||
(cid->opt_mask() & CO::WRITE) || (under_script && cntx->conn_state.script_info->is_write);
|
(under_script && dfly_cntx->conn_state.script_info->is_write);
|
||||||
bool under_multi = cntx->conn_state.exec_state != ConnectionState::EXEC_INACTIVE && !is_trans_cmd;
|
bool under_multi =
|
||||||
|
dfly_cntx->conn_state.exec_state != ConnectionState::EXEC_INACTIVE && !is_trans_cmd;
|
||||||
|
|
||||||
if (!etl.is_master && is_write_cmd) {
|
if (!etl.is_master && is_write_cmd) {
|
||||||
(*cntx)->SendError("-READONLY You can't write against a read only replica.");
|
(*cntx)->SendError("-READONLY You can't write against a read only replica.");
|
||||||
|
@ -386,15 +391,15 @@ void Service::DispatchCommand(CmdArgList args, ConnectionContext* cntx) {
|
||||||
|
|
||||||
if ((cid->arity() > 0 && args.size() != size_t(cid->arity())) ||
|
if ((cid->arity() > 0 && args.size() != size_t(cid->arity())) ||
|
||||||
(cid->arity() < 0 && args.size() < size_t(-cid->arity()))) {
|
(cid->arity() < 0 && args.size() < size_t(-cid->arity()))) {
|
||||||
return (*cntx)->SendError(WrongNumArgsError(cmd_str));
|
return (*cntx)->SendError(facade::WrongNumArgsError(cmd_str));
|
||||||
}
|
}
|
||||||
|
|
||||||
if (cid->key_arg_step() == 2 && (args.size() % 2) == 0) {
|
if (cid->key_arg_step() == 2 && (args.size() % 2) == 0) {
|
||||||
return (*cntx)->SendError(WrongNumArgsError(cmd_str));
|
return (*cntx)->SendError(facade::WrongNumArgsError(cmd_str));
|
||||||
}
|
}
|
||||||
|
|
||||||
// Validate more complicated cases with custom validators.
|
// Validate more complicated cases with custom validators.
|
||||||
if (!cid->Validate(args, cntx)) {
|
if (!cid->Validate(args, dfly_cntx)) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -412,14 +417,14 @@ void Service::DispatchCommand(CmdArgList args, ConnectionContext* cntx) {
|
||||||
|
|
||||||
std::move(multi_error).Cancel();
|
std::move(multi_error).Cancel();
|
||||||
|
|
||||||
if (cntx->conn_state.exec_state != ConnectionState::EXEC_INACTIVE && !is_trans_cmd) {
|
if (dfly_cntx->conn_state.exec_state != ConnectionState::EXEC_INACTIVE && !is_trans_cmd) {
|
||||||
// TODO: protect against aggregating huge transactions.
|
// TODO: protect against aggregating huge transactions.
|
||||||
StoredCmd stored_cmd{cid};
|
StoredCmd stored_cmd{cid};
|
||||||
stored_cmd.cmd.reserve(args.size());
|
stored_cmd.cmd.reserve(args.size());
|
||||||
for (size_t i = 0; i < args.size(); ++i) {
|
for (size_t i = 0; i < args.size(); ++i) {
|
||||||
stored_cmd.cmd.emplace_back(ArgS(args, i));
|
stored_cmd.cmd.emplace_back(ArgS(args, i));
|
||||||
}
|
}
|
||||||
cntx->conn_state.exec_body.push_back(std::move(stored_cmd));
|
dfly_cntx->conn_state.exec_body.push_back(std::move(stored_cmd));
|
||||||
|
|
||||||
return (*cntx)->SendSimpleString("QUEUED");
|
return (*cntx)->SendSimpleString("QUEUED");
|
||||||
}
|
}
|
||||||
|
@ -430,48 +435,48 @@ void Service::DispatchCommand(CmdArgList args, ConnectionContext* cntx) {
|
||||||
intrusive_ptr<Transaction> dist_trans;
|
intrusive_ptr<Transaction> dist_trans;
|
||||||
|
|
||||||
if (under_script) {
|
if (under_script) {
|
||||||
DCHECK(cntx->transaction);
|
DCHECK(dfly_cntx->transaction);
|
||||||
KeyIndex key_index = DetermineKeys(cid, args);
|
KeyIndex key_index = DetermineKeys(cid, args);
|
||||||
for (unsigned i = key_index.start; i < key_index.end; ++i) {
|
for (unsigned i = key_index.start; i < key_index.end; ++i) {
|
||||||
string_view key = ArgS(args, i);
|
string_view key = ArgS(args, i);
|
||||||
if (!cntx->conn_state.script_info->keys.contains(key)) {
|
if (!dfly_cntx->conn_state.script_info->keys.contains(key)) {
|
||||||
return (*cntx)->SendError("script tried accessing undeclared key");
|
return (*cntx)->SendError("script tried accessing undeclared key");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
cntx->transaction->SetExecCmd(cid);
|
dfly_cntx->transaction->SetExecCmd(cid);
|
||||||
cntx->transaction->InitByArgs(cntx->conn_state.db_index, args);
|
dfly_cntx->transaction->InitByArgs(dfly_cntx->conn_state.db_index, args);
|
||||||
} else {
|
} else {
|
||||||
DCHECK(cntx->transaction == nullptr);
|
DCHECK(dfly_cntx->transaction == nullptr);
|
||||||
|
|
||||||
if (IsTransactional(cid)) {
|
if (IsTransactional(cid)) {
|
||||||
dist_trans.reset(new Transaction{cid, &shard_set_});
|
dist_trans.reset(new Transaction{cid, &shard_set_});
|
||||||
cntx->transaction = dist_trans.get();
|
dfly_cntx->transaction = dist_trans.get();
|
||||||
|
|
||||||
dist_trans->InitByArgs(cntx->conn_state.db_index, args);
|
dist_trans->InitByArgs(dfly_cntx->conn_state.db_index, args);
|
||||||
cntx->last_command_debug.shards_count = cntx->transaction->unique_shard_cnt();
|
dfly_cntx->last_command_debug.shards_count = dfly_cntx->transaction->unique_shard_cnt();
|
||||||
} else {
|
} else {
|
||||||
cntx->transaction = nullptr;
|
dfly_cntx->transaction = nullptr;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
cntx->cid = cid;
|
dfly_cntx->cid = cid;
|
||||||
cmd_req.Inc({cmd_name});
|
cmd_req.Inc({cmd_name});
|
||||||
cid->Invoke(args, cntx);
|
cid->Invoke(args, dfly_cntx);
|
||||||
end_usec = ProactorBase::GetMonotonicTimeNs();
|
end_usec = ProactorBase::GetMonotonicTimeNs();
|
||||||
|
|
||||||
request_latency_usec.IncBy(cmd_str, (end_usec - start_usec) / 1000);
|
request_latency_usec.IncBy(cmd_str, (end_usec - start_usec) / 1000);
|
||||||
if (dist_trans) {
|
if (dist_trans) {
|
||||||
cntx->last_command_debug.clock = dist_trans->txid();
|
dfly_cntx->last_command_debug.clock = dist_trans->txid();
|
||||||
cntx->last_command_debug.is_ooo = dist_trans->IsOOO();
|
dfly_cntx->last_command_debug.is_ooo = dist_trans->IsOOO();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!under_script) {
|
if (!under_script) {
|
||||||
cntx->transaction = nullptr;
|
dfly_cntx->transaction = nullptr;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void Service::DispatchMC(const MemcacheParser::Command& cmd, std::string_view value,
|
void Service::DispatchMC(const MemcacheParser::Command& cmd, std::string_view value,
|
||||||
ConnectionContext* cntx) {
|
facade::ConnectionContext* cntx) {
|
||||||
absl::InlinedVector<MutableSlice, 8> args;
|
absl::InlinedVector<MutableSlice, 8> args;
|
||||||
char cmd_name[16];
|
char cmd_name[16];
|
||||||
char ttl[16];
|
char ttl[16];
|
||||||
|
@ -533,6 +538,8 @@ void Service::DispatchMC(const MemcacheParser::Command& cmd, std::string_view va
|
||||||
args.emplace_back(key, cmd.key.size());
|
args.emplace_back(key, cmd.key.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ConnectionContext* dfly_cntx = static_cast<ConnectionContext*>(cntx);
|
||||||
|
|
||||||
if (MemcacheParser::IsStoreCmd(cmd.type)) {
|
if (MemcacheParser::IsStoreCmd(cmd.type)) {
|
||||||
char* v = const_cast<char*>(value.data());
|
char* v = const_cast<char*>(value.data());
|
||||||
args.emplace_back(v, value.size());
|
args.emplace_back(v, value.size());
|
||||||
|
@ -546,7 +553,7 @@ void Service::DispatchMC(const MemcacheParser::Command& cmd, std::string_view va
|
||||||
args.emplace_back(ttl_op, 2);
|
args.emplace_back(ttl_op, 2);
|
||||||
args.emplace_back(ttl, next - ttl);
|
args.emplace_back(ttl, next - ttl);
|
||||||
}
|
}
|
||||||
cntx->conn_state.memcache_flag = cmd.flags;
|
dfly_cntx->conn_state.memcache_flag = cmd.flags;
|
||||||
} else if (cmd.type < MemcacheParser::QUIT) { // read commands
|
} else if (cmd.type < MemcacheParser::QUIT) { // read commands
|
||||||
for (auto s : cmd.keys_ext) {
|
for (auto s : cmd.keys_ext) {
|
||||||
char* key = const_cast<char*>(s.data());
|
char* key = const_cast<char*>(s.data());
|
||||||
|
@ -561,7 +568,28 @@ void Service::DispatchMC(const MemcacheParser::Command& cmd, std::string_view va
|
||||||
DispatchCommand(CmdArgList{args}, cntx);
|
DispatchCommand(CmdArgList{args}, cntx);
|
||||||
|
|
||||||
// Reset back.
|
// Reset back.
|
||||||
cntx->conn_state.memcache_flag = 0;
|
dfly_cntx->conn_state.memcache_flag = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
facade::ConnectionContext* Service::CreateContext(util::FiberSocketBase* peer,
|
||||||
|
facade::Connection* owner) {
|
||||||
|
ConnectionContext* res = new ConnectionContext{peer, owner};
|
||||||
|
res->shard_set = &shard_set();
|
||||||
|
res->req_auth = IsPassProtected();
|
||||||
|
|
||||||
|
// a bit of a hack. I set up breaker callback here for the owner.
|
||||||
|
// Should work though it's confusing to have it here.
|
||||||
|
owner->RegisterOnBreak([res](uint32_t) {
|
||||||
|
if (res->transaction) {
|
||||||
|
res->transaction->BreakOnClose();
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
facade::ConnectionStats* Service::GetThreadLocalConnectionStats() {
|
||||||
|
return ServerState::tl_connection_stats();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool Service::IsLocked(DbIndex db_index, std::string_view key) const {
|
bool Service::IsLocked(DbIndex db_index, std::string_view key) const {
|
||||||
|
@ -590,14 +618,10 @@ bool Service::IsPassProtected() const {
|
||||||
return !FLAGS_requirepass.empty();
|
return !FLAGS_requirepass.empty();
|
||||||
}
|
}
|
||||||
|
|
||||||
void Service::RegisterHttp(HttpListenerBase* listener) {
|
|
||||||
CHECK_NOTNULL(listener);
|
|
||||||
http_listener_ = listener;
|
|
||||||
}
|
|
||||||
|
|
||||||
void Service::Quit(CmdArgList args, ConnectionContext* cntx) {
|
void Service::Quit(CmdArgList args, ConnectionContext* cntx) {
|
||||||
if (cntx->protocol() == Protocol::REDIS)
|
if (cntx->protocol() == facade::Protocol::REDIS)
|
||||||
(*cntx)->SendOk();
|
(*cntx)->SendOk();
|
||||||
|
using facade::SinkReplyBuilder;
|
||||||
|
|
||||||
SinkReplyBuilder* builder = static_cast<SinkReplyBuilder*>(cntx->reply_builder());
|
SinkReplyBuilder* builder = static_cast<SinkReplyBuilder*>(cntx->reply_builder());
|
||||||
builder->CloseConnection();
|
builder->CloseConnection();
|
||||||
|
@ -615,7 +639,7 @@ void Service::Multi(CmdArgList args, ConnectionContext* cntx) {
|
||||||
void Service::CallFromScript(CmdArgList args, ObjectExplorer* reply, ConnectionContext* cntx) {
|
void Service::CallFromScript(CmdArgList args, ObjectExplorer* reply, ConnectionContext* cntx) {
|
||||||
DCHECK(cntx->transaction);
|
DCHECK(cntx->transaction);
|
||||||
InterpreterReplier replier(reply);
|
InterpreterReplier replier(reply);
|
||||||
ReplyBuilderInterface* orig = cntx->Inject(&replier);
|
facade::ReplyBuilderInterface* orig = cntx->Inject(&replier);
|
||||||
|
|
||||||
DispatchCommand(std::move(args), cntx);
|
DispatchCommand(std::move(args), cntx);
|
||||||
|
|
||||||
|
@ -670,7 +694,7 @@ void Service::EvalSha(CmdArgList args, ConnectionContext* cntx) {
|
||||||
if (!exists) {
|
if (!exists) {
|
||||||
const char* body = (sha.size() == 40) ? server_family_.script_mgr()->Find(sha) : nullptr;
|
const char* body = (sha.size() == 40) ? server_family_.script_mgr()->Find(sha) : nullptr;
|
||||||
if (!body) {
|
if (!body) {
|
||||||
return (*cntx)->SendError(kScriptNotFound);
|
return (*cntx)->SendError(facade::kScriptNotFound);
|
||||||
}
|
}
|
||||||
|
|
||||||
string res;
|
string res;
|
||||||
|
@ -692,7 +716,7 @@ void Service::EvalInternal(const EvalArgs& eval_args, Interpreter* interpreter,
|
||||||
|
|
||||||
// Sanitizing the input to avoid code injection.
|
// Sanitizing the input to avoid code injection.
|
||||||
if (eval_args.sha.size() != 40 || !IsSHA(eval_args.sha)) {
|
if (eval_args.sha.size() != 40 || !IsSHA(eval_args.sha)) {
|
||||||
return (*cntx)->SendError(kScriptNotFound);
|
return (*cntx)->SendError(facade::kScriptNotFound);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool exists = interpreter->Exists(eval_args.sha);
|
bool exists = interpreter->Exists(eval_args.sha);
|
||||||
|
@ -700,7 +724,7 @@ void Service::EvalInternal(const EvalArgs& eval_args, Interpreter* interpreter,
|
||||||
if (!exists) {
|
if (!exists) {
|
||||||
const char* body = server_family_.script_mgr()->Find(eval_args.sha);
|
const char* body = server_family_.script_mgr()->Find(eval_args.sha);
|
||||||
if (!body) {
|
if (!body) {
|
||||||
return (*cntx)->SendError(kScriptNotFound);
|
return (*cntx)->SendError(facade::kScriptNotFound);
|
||||||
}
|
}
|
||||||
|
|
||||||
string res;
|
string res;
|
||||||
|
|
|
@ -5,11 +5,10 @@
|
||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include "base/varz_value.h"
|
#include "base/varz_value.h"
|
||||||
|
#include "facade/service_interface.h"
|
||||||
#include "server/command_registry.h"
|
#include "server/command_registry.h"
|
||||||
#include "server/engine_shard_set.h"
|
#include "server/engine_shard_set.h"
|
||||||
#include "server/memcache_parser.h"
|
|
||||||
#include "server/server_family.h"
|
#include "server/server_family.h"
|
||||||
#include "util/http/http_handler.h"
|
|
||||||
|
|
||||||
namespace util {
|
namespace util {
|
||||||
class AcceptServer;
|
class AcceptServer;
|
||||||
|
@ -19,8 +18,9 @@ namespace dfly {
|
||||||
|
|
||||||
class Interpreter;
|
class Interpreter;
|
||||||
class ObjectExplorer; // for Interpreter
|
class ObjectExplorer; // for Interpreter
|
||||||
|
using facade::MemcacheParser;
|
||||||
|
|
||||||
class Service {
|
class Service : public facade::ServiceInterface {
|
||||||
public:
|
public:
|
||||||
using error_code = std::error_code;
|
using error_code = std::error_code;
|
||||||
|
|
||||||
|
@ -34,15 +34,18 @@ class Service {
|
||||||
explicit Service(util::ProactorPool* pp);
|
explicit Service(util::ProactorPool* pp);
|
||||||
~Service();
|
~Service();
|
||||||
|
|
||||||
void RegisterHttp(util::HttpListenerBase* listener);
|
|
||||||
|
|
||||||
void Init(util::AcceptServer* acceptor, const InitOpts& opts = InitOpts{});
|
void Init(util::AcceptServer* acceptor, const InitOpts& opts = InitOpts{});
|
||||||
|
|
||||||
void Shutdown();
|
void Shutdown();
|
||||||
|
|
||||||
void DispatchCommand(CmdArgList args, ConnectionContext* cntx);
|
void DispatchCommand(CmdArgList args, facade::ConnectionContext* cntx) final;
|
||||||
void DispatchMC(const MemcacheParser::Command& cmd, std::string_view value,
|
void DispatchMC(const MemcacheParser::Command& cmd, std::string_view value,
|
||||||
ConnectionContext* cntx);
|
facade::ConnectionContext* cntx) final;
|
||||||
|
|
||||||
|
facade::ConnectionContext* CreateContext(util::FiberSocketBase* peer,
|
||||||
|
facade::Connection* owner) final;
|
||||||
|
|
||||||
|
facade::ConnectionStats* GetThreadLocalConnectionStats() final;
|
||||||
|
|
||||||
uint32_t shard_count() const {
|
uint32_t shard_count() const {
|
||||||
return shard_set_.size();
|
return shard_set_.size();
|
||||||
|
@ -60,10 +63,6 @@ class Service {
|
||||||
return pp_;
|
return pp_;
|
||||||
}
|
}
|
||||||
|
|
||||||
util::HttpListenerBase* http_listener() {
|
|
||||||
return http_listener_;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool IsPassProtected() const;
|
bool IsPassProtected() const;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
@ -91,8 +90,6 @@ class Service {
|
||||||
EngineShardSet shard_set_;
|
EngineShardSet shard_set_;
|
||||||
ServerFamily server_family_;
|
ServerFamily server_family_;
|
||||||
CommandRegistry registry_;
|
CommandRegistry registry_;
|
||||||
|
|
||||||
util::HttpListenerBase* http_listener_ = nullptr;
|
|
||||||
};
|
};
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace dfly
|
||||||
|
|
|
@ -28,6 +28,7 @@ using base::IoBuf;
|
||||||
using nonstd::make_unexpected;
|
using nonstd::make_unexpected;
|
||||||
using namespace util;
|
using namespace util;
|
||||||
using rdb::errc;
|
using rdb::errc;
|
||||||
|
using facade::operator""_KB;
|
||||||
|
|
||||||
class error_category : public std::error_category {
|
class error_category : public std::error_category {
|
||||||
public:
|
public:
|
||||||
|
|
|
@ -24,6 +24,7 @@ using namespace std;
|
||||||
using base::IoBuf;
|
using base::IoBuf;
|
||||||
using io::Bytes;
|
using io::Bytes;
|
||||||
using nonstd::make_unexpected;
|
using nonstd::make_unexpected;
|
||||||
|
using facade::operator""_KB;
|
||||||
|
|
||||||
namespace {
|
namespace {
|
||||||
|
|
||||||
|
|
|
@ -14,10 +14,10 @@ extern "C" {
|
||||||
#include <boost/asio/ip/tcp.hpp>
|
#include <boost/asio/ip/tcp.hpp>
|
||||||
|
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
|
#include "facade/redis_parser.h"
|
||||||
#include "server/error.h"
|
#include "server/error.h"
|
||||||
#include "server/main_service.h"
|
#include "server/main_service.h"
|
||||||
#include "server/rdb_load.h"
|
#include "server/rdb_load.h"
|
||||||
#include "server/redis_parser.h"
|
|
||||||
#include "util/proactor_base.h"
|
#include "util/proactor_base.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
@ -25,6 +25,7 @@ namespace dfly {
|
||||||
using namespace std;
|
using namespace std;
|
||||||
using namespace util;
|
using namespace util;
|
||||||
using namespace boost::asio;
|
using namespace boost::asio;
|
||||||
|
using namespace facade;
|
||||||
namespace this_fiber = ::boost::this_fiber;
|
namespace this_fiber = ::boost::this_fiber;
|
||||||
|
|
||||||
namespace {
|
namespace {
|
||||||
|
@ -74,7 +75,6 @@ error_code Recv(FiberSocketBase* input, base::IoBuf* dest) {
|
||||||
return error_code{};
|
return error_code{};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
// TODO: to remove usages of this macro and make code crash-less.
|
// TODO: to remove usages of this macro and make code crash-less.
|
||||||
#define CHECK_EC(x) \
|
#define CHECK_EC(x) \
|
||||||
do { \
|
do { \
|
||||||
|
|
|
@ -7,13 +7,12 @@
|
||||||
#include <variant>
|
#include <variant>
|
||||||
|
|
||||||
#include "base/io_buf.h"
|
#include "base/io_buf.h"
|
||||||
#include "core/resp_expr.h"
|
#include "facade/redis_parser.h"
|
||||||
#include "server/conn_context.h"
|
#include "server/conn_context.h"
|
||||||
#include "util/fiber_socket_base.h"
|
#include "util/fiber_socket_base.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
class RedisParser;
|
|
||||||
class Service;
|
class Service;
|
||||||
|
|
||||||
class Replica {
|
class Replica {
|
||||||
|
@ -76,7 +75,7 @@ class Replica {
|
||||||
|
|
||||||
// Where the sock_ is handled.
|
// Where the sock_ is handled.
|
||||||
util::ProactorBase* sock_thread_ = nullptr;
|
util::ProactorBase* sock_thread_ = nullptr;
|
||||||
std::unique_ptr<RedisParser> parser_;
|
std::unique_ptr<facade::RedisParser> parser_;
|
||||||
|
|
||||||
// repl_offs - till what offset we've already read from the master.
|
// repl_offs - till what offset we've already read from the master.
|
||||||
// ack_offs_ last acknowledged offset.
|
// ack_offs_ last acknowledged offset.
|
||||||
|
|
|
@ -46,6 +46,7 @@ using namespace util;
|
||||||
namespace fibers = ::boost::fibers;
|
namespace fibers = ::boost::fibers;
|
||||||
namespace fs = std::filesystem;
|
namespace fs = std::filesystem;
|
||||||
using strings::HumanReadableNumBytes;
|
using strings::HumanReadableNumBytes;
|
||||||
|
using facade::MCReplyBuilder;
|
||||||
|
|
||||||
namespace {
|
namespace {
|
||||||
|
|
||||||
|
@ -97,7 +98,7 @@ void ServerFamily::Shutdown() {
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
void ServerFamily::StatsMC(std::string_view section, ConnectionContext* cntx) {
|
void ServerFamily::StatsMC(std::string_view section, facade::ConnectionContext* cntx) {
|
||||||
if (!section.empty()) {
|
if (!section.empty()) {
|
||||||
return cntx->reply_builder()->SendError("");
|
return cntx->reply_builder()->SendError("");
|
||||||
}
|
}
|
||||||
|
@ -200,7 +201,7 @@ void ServerFamily::Auth(CmdArgList args, ConnectionContext* cntx) {
|
||||||
return (*cntx)->SendError("ACL is not supported yet");
|
return (*cntx)->SendError("ACL is not supported yet");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!(cntx->conn_state.mask & ConnectionState::REQ_AUTH)) {
|
if (!cntx->req_auth) {
|
||||||
return (*cntx)->SendError(
|
return (*cntx)->SendError(
|
||||||
"AUTH <password> called without any password configured for the "
|
"AUTH <password> called without any password configured for the "
|
||||||
"default user. Are you sure your configuration is correct?");
|
"default user. Are you sure your configuration is correct?");
|
||||||
|
@ -208,10 +209,10 @@ void ServerFamily::Auth(CmdArgList args, ConnectionContext* cntx) {
|
||||||
|
|
||||||
string_view pass = ArgS(args, 1);
|
string_view pass = ArgS(args, 1);
|
||||||
if (pass == FLAGS_requirepass) {
|
if (pass == FLAGS_requirepass) {
|
||||||
cntx->conn_state.mask |= ConnectionState::AUTHENTICATED;
|
cntx->authenticated = true;
|
||||||
(*cntx)->SendOk();
|
(*cntx)->SendOk();
|
||||||
} else {
|
} else {
|
||||||
(*cntx)->SendError(kAuthRejected);
|
(*cntx)->SendError(facade::kAuthRejected);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -544,14 +545,14 @@ void ServerFamily::_Shutdown(CmdArgList args, ConnectionContext* cntx) {
|
||||||
|
|
||||||
void ServerFamily::SyncGeneric(std::string_view repl_master_id, uint64_t offs,
|
void ServerFamily::SyncGeneric(std::string_view repl_master_id, uint64_t offs,
|
||||||
ConnectionContext* cntx) {
|
ConnectionContext* cntx) {
|
||||||
if (cntx->conn_state.mask & ConnectionState::ASYNC_DISPATCH) {
|
if (cntx->async_dispatch) {
|
||||||
// SYNC is a special command that should not be sent in batch with other commands.
|
// SYNC is a special command that should not be sent in batch with other commands.
|
||||||
// It should be the last command since afterwards the server just dumps the replication data.
|
// It should be the last command since afterwards the server just dumps the replication data.
|
||||||
(*cntx)->SendError("Can not sync in pipeline mode");
|
(*cntx)->SendError("Can not sync in pipeline mode");
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
cntx->conn_state.mask |= ConnectionState::REPL_CONNECTION;
|
cntx->replica_conn = true;
|
||||||
ServerState::tl_connection_stats()->num_replicas += 1;
|
ServerState::tl_connection_stats()->num_replicas += 1;
|
||||||
// TBD.
|
// TBD.
|
||||||
}
|
}
|
||||||
|
|
|
@ -4,6 +4,8 @@
|
||||||
|
|
||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include "facade/conn_context.h"
|
||||||
|
#include "facade/redis_parser.h"
|
||||||
#include "server/engine_shard_set.h"
|
#include "server/engine_shard_set.h"
|
||||||
#include "server/global_state.h"
|
#include "server/global_state.h"
|
||||||
#include "util/proactor_pool.h"
|
#include "util/proactor_pool.h"
|
||||||
|
@ -28,7 +30,7 @@ struct Metrics {
|
||||||
size_t heap_used_bytes = 0;
|
size_t heap_used_bytes = 0;
|
||||||
size_t heap_comitted_bytes = 0;
|
size_t heap_comitted_bytes = 0;
|
||||||
|
|
||||||
ConnectionStats conn_stats;
|
facade::ConnectionStats conn_stats;
|
||||||
};
|
};
|
||||||
|
|
||||||
class ServerFamily {
|
class ServerFamily {
|
||||||
|
@ -50,7 +52,7 @@ class ServerFamily {
|
||||||
return script_mgr_.get();
|
return script_mgr_.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
void StatsMC(std::string_view section, ConnectionContext* cntx);
|
void StatsMC(std::string_view section, facade::ConnectionContext* cntx);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
uint32_t shard_count() const {
|
uint32_t shard_count() const {
|
||||||
|
|
|
@ -28,7 +28,7 @@ class ServerState { // public struct - to allow initialization.
|
||||||
return &state_;
|
return &state_;
|
||||||
}
|
}
|
||||||
|
|
||||||
static ConnectionStats* tl_connection_stats() {
|
static facade::ConnectionStats* tl_connection_stats() {
|
||||||
return &state_.connection_stats;
|
return &state_.connection_stats;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -40,7 +40,7 @@ class ServerState { // public struct - to allow initialization.
|
||||||
|
|
||||||
bool is_master = true;
|
bool is_master = true;
|
||||||
|
|
||||||
ConnectionStats connection_stats;
|
facade::ConnectionStats connection_stats;
|
||||||
|
|
||||||
void TxCountInc() {
|
void TxCountInc() {
|
||||||
++live_transactions_;
|
++live_transactions_;
|
||||||
|
|
|
@ -20,6 +20,7 @@ extern "C" {
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
using namespace std;
|
using namespace std;
|
||||||
|
|
||||||
using ResultStringVec = vector<OpResult<vector<string>>>;
|
using ResultStringVec = vector<OpResult<vector<string>>>;
|
||||||
using ResultSetView = OpResult<absl::flat_hash_set<std::string_view>>;
|
using ResultSetView = OpResult<absl::flat_hash_set<std::string_view>>;
|
||||||
using SvArray = vector<std::string_view>;
|
using SvArray = vector<std::string_view>;
|
||||||
|
|
|
@ -4,11 +4,13 @@
|
||||||
|
|
||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include "core/op_status.h"
|
#include "facade/op_status.h"
|
||||||
#include "server/common_types.h"
|
#include "server/common_types.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
|
using facade::OpResult;
|
||||||
|
|
||||||
class ConnectionContext;
|
class ConnectionContext;
|
||||||
class CommandRegistry;
|
class CommandRegistry;
|
||||||
class EngineShard;
|
class EngineShard;
|
||||||
|
|
|
@ -6,6 +6,7 @@
|
||||||
|
|
||||||
#include "base/gtest.h"
|
#include "base/gtest.h"
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
|
#include "facade/facade_test.h"
|
||||||
#include "server/command_registry.h"
|
#include "server/command_registry.h"
|
||||||
#include "server/test_utils.h"
|
#include "server/test_utils.h"
|
||||||
|
|
||||||
|
|
|
@ -1,4 +1,4 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
|
@ -23,6 +23,8 @@ namespace dfly {
|
||||||
namespace {
|
namespace {
|
||||||
|
|
||||||
using namespace std;
|
using namespace std;
|
||||||
|
using facade::Protocol;
|
||||||
|
using facade::ReplyBuilderInterface;
|
||||||
|
|
||||||
using CI = CommandId;
|
using CI = CommandId;
|
||||||
DEFINE_VARZ(VarzQps, set_qps);
|
DEFINE_VARZ(VarzQps, set_qps);
|
||||||
|
@ -284,7 +286,7 @@ void StringFamily::IncrByGeneric(std::string_view key, int64_t val, ConnectionCo
|
||||||
case OpStatus::WRONG_TYPE:
|
case OpStatus::WRONG_TYPE:
|
||||||
return builder->SendError(kWrongTypeErr);
|
return builder->SendError(kWrongTypeErr);
|
||||||
case OpStatus::KEY_NOTFOUND: // Relevant only for MC
|
case OpStatus::KEY_NOTFOUND: // Relevant only for MC
|
||||||
return reinterpret_cast<MCReplyBuilder*>(builder)->SendNotFound();
|
return reinterpret_cast<facade::MCReplyBuilder*>(builder)->SendNotFound();
|
||||||
default:;
|
default:;
|
||||||
}
|
}
|
||||||
__builtin_unreachable();
|
__builtin_unreachable();
|
||||||
|
@ -327,8 +329,10 @@ void StringFamily::MGet(CmdArgList args, ConnectionContext* cntx) {
|
||||||
unsigned shard_count = transaction->shard_set()->size();
|
unsigned shard_count = transaction->shard_set()->size();
|
||||||
std::vector<MGetResponse> mget_resp(shard_count);
|
std::vector<MGetResponse> mget_resp(shard_count);
|
||||||
|
|
||||||
|
ConnectionContext* dfly_cntx = static_cast<ConnectionContext*>(cntx);
|
||||||
bool fetch_mcflag = cntx->protocol() == Protocol::MEMCACHE;
|
bool fetch_mcflag = cntx->protocol() == Protocol::MEMCACHE;
|
||||||
bool fetch_mcver = fetch_mcflag && (cntx->conn_state.mask & ConnectionState::FETCH_CAS_VER);
|
bool fetch_mcver =
|
||||||
|
fetch_mcflag && (dfly_cntx->conn_state.memcache_flag & ConnectionState::FETCH_CAS_VER);
|
||||||
|
|
||||||
auto cb = [&](Transaction* t, EngineShard* shard) {
|
auto cb = [&](Transaction* t, EngineShard* shard) {
|
||||||
ShardId sid = shard->shard_id();
|
ShardId sid = shard->shard_id();
|
||||||
|
@ -436,8 +440,8 @@ OpStatus StringFamily::OpMSet(const Transaction* t, EngineShard* es) {
|
||||||
return OpStatus::OK;
|
return OpStatus::OK;
|
||||||
}
|
}
|
||||||
|
|
||||||
OpResult<int64_t> StringFamily::OpIncrBy(const OpArgs& op_args, std::string_view key,
|
OpResult<int64_t> StringFamily::OpIncrBy(const OpArgs& op_args, std::string_view key, int64_t incr,
|
||||||
int64_t incr, bool skip_on_missing) {
|
bool skip_on_missing) {
|
||||||
auto& db_slice = op_args.shard->db_slice();
|
auto& db_slice = op_args.shard->db_slice();
|
||||||
auto [it, expire_it] = db_slice.FindExt(op_args.db_ind, key);
|
auto [it, expire_it] = db_slice.FindExt(op_args.db_ind, key);
|
||||||
|
|
||||||
|
|
|
@ -12,6 +12,8 @@ namespace dfly {
|
||||||
|
|
||||||
class ConnectionContext;
|
class ConnectionContext;
|
||||||
class CommandRegistry;
|
class CommandRegistry;
|
||||||
|
using facade::OpStatus;
|
||||||
|
using facade::OpResult;
|
||||||
|
|
||||||
class SetCmd {
|
class SetCmd {
|
||||||
DbSlice* db_slice_;
|
DbSlice* db_slice_;
|
||||||
|
|
|
@ -6,6 +6,7 @@
|
||||||
|
|
||||||
#include "base/gtest.h"
|
#include "base/gtest.h"
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
|
#include "facade/facade_test.h"
|
||||||
#include "server/command_registry.h"
|
#include "server/command_registry.h"
|
||||||
#include "server/conn_context.h"
|
#include "server/conn_context.h"
|
||||||
#include "server/engine_shard_set.h"
|
#include "server/engine_shard_set.h"
|
||||||
|
|
|
@ -1,4 +1,4 @@
|
||||||
// Copyright 2021, Roman Gershman. All rights reserved.
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
// See LICENSE for licensing terms.
|
// See LICENSE for licensing terms.
|
||||||
//
|
//
|
||||||
|
|
||||||
|
@ -9,15 +9,14 @@
|
||||||
|
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
#include "base/stl_util.h"
|
#include "base/stl_util.h"
|
||||||
#include "server/dragonfly_connection.h"
|
#include "facade/dragonfly_connection.h"
|
||||||
#include "util/uring/uring_pool.h"
|
#include "util/uring/uring_pool.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
using namespace testing;
|
|
||||||
using namespace util;
|
|
||||||
using namespace std;
|
|
||||||
using MP = MemcacheParser;
|
using MP = MemcacheParser;
|
||||||
|
using namespace std;
|
||||||
|
using namespace util;
|
||||||
|
using namespace testing;
|
||||||
|
|
||||||
static vector<string> SplitLines(const std::string& src) {
|
static vector<string> SplitLines(const std::string& src) {
|
||||||
vector<string> res = absl::StrSplit(src, "\r\n");
|
vector<string> res = absl::StrSplit(src, "\r\n");
|
||||||
|
@ -29,90 +28,6 @@ static vector<string> SplitLines(const std::string& src) {
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool RespMatcher::MatchAndExplain(const RespExpr& e, MatchResultListener* listener) const {
|
|
||||||
if (e.type != type_) {
|
|
||||||
*listener << "\nWrong type: " << RespExpr::TypeName(e.type);
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (type_ == RespExpr::STRING || type_ == RespExpr::ERROR) {
|
|
||||||
RespExpr::Buffer ebuf = e.GetBuf();
|
|
||||||
std::string_view actual{reinterpret_cast<char*>(ebuf.data()), ebuf.size()};
|
|
||||||
|
|
||||||
if (type_ == RespExpr::ERROR && !absl::StrContains(actual, exp_str_)) {
|
|
||||||
*listener << "Actual does not contain '" << exp_str_ << "'";
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
if (type_ == RespExpr::STRING && exp_str_ != actual) {
|
|
||||||
*listener << "\nActual string: " << actual;
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
} else if (type_ == RespExpr::INT64) {
|
|
||||||
auto actual = get<int64_t>(e.u);
|
|
||||||
if (exp_int_ != actual) {
|
|
||||||
*listener << "\nActual : " << actual << " expected: " << exp_int_;
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
} else if (type_ == RespExpr::ARRAY) {
|
|
||||||
size_t len = get<RespVec*>(e.u)->size();
|
|
||||||
if (len != size_t(exp_int_)) {
|
|
||||||
*listener << "Actual length " << len << ", expected: " << exp_int_;
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
void RespMatcher::DescribeTo(std::ostream* os) const {
|
|
||||||
*os << "is ";
|
|
||||||
switch (type_) {
|
|
||||||
case RespExpr::STRING:
|
|
||||||
case RespExpr::ERROR:
|
|
||||||
*os << exp_str_;
|
|
||||||
break;
|
|
||||||
|
|
||||||
case RespExpr::INT64:
|
|
||||||
*os << exp_str_;
|
|
||||||
break;
|
|
||||||
default:
|
|
||||||
*os << "TBD";
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
void RespMatcher::DescribeNegationTo(std::ostream* os) const {
|
|
||||||
*os << "is not ";
|
|
||||||
}
|
|
||||||
|
|
||||||
bool RespTypeMatcher::MatchAndExplain(const RespExpr& e, MatchResultListener* listener) const {
|
|
||||||
if (e.type != type_) {
|
|
||||||
*listener << "\nWrong type: " << RespExpr::TypeName(e.type);
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
void RespTypeMatcher::DescribeTo(std::ostream* os) const {
|
|
||||||
*os << "is " << RespExpr::TypeName(type_);
|
|
||||||
}
|
|
||||||
|
|
||||||
void RespTypeMatcher::DescribeNegationTo(std::ostream* os) const {
|
|
||||||
*os << "is not " << RespExpr::TypeName(type_);
|
|
||||||
}
|
|
||||||
|
|
||||||
void PrintTo(const RespExpr::Vec& vec, std::ostream* os) {
|
|
||||||
*os << "Vec: [";
|
|
||||||
if (!vec.empty()) {
|
|
||||||
for (size_t i = 0; i < vec.size() - 1; ++i) {
|
|
||||||
*os << vec[i] << ",";
|
|
||||||
}
|
|
||||||
*os << vec.back();
|
|
||||||
}
|
|
||||||
*os << "]\n";
|
|
||||||
}
|
|
||||||
|
|
||||||
vector<int64_t> ToIntArr(const RespVec& vec) {
|
vector<int64_t> ToIntArr(const RespVec& vec) {
|
||||||
vector<int64_t> res;
|
vector<int64_t> res;
|
||||||
for (auto a : vec) {
|
for (auto a : vec) {
|
||||||
|
@ -126,7 +41,8 @@ vector<int64_t> ToIntArr(const RespVec& vec) {
|
||||||
}
|
}
|
||||||
|
|
||||||
BaseFamilyTest::TestConnWrapper::TestConnWrapper(Protocol proto)
|
BaseFamilyTest::TestConnWrapper::TestConnWrapper(Protocol proto)
|
||||||
: dummy_conn(new Connection(proto, nullptr, nullptr)), cmd_cntx(&sink, dummy_conn.get()) {
|
: dummy_conn(new facade::Connection(proto, nullptr, nullptr, nullptr)),
|
||||||
|
cmd_cntx(&sink, dummy_conn.get()) {
|
||||||
}
|
}
|
||||||
|
|
||||||
BaseFamilyTest::TestConnWrapper::~TestConnWrapper() {
|
BaseFamilyTest::TestConnWrapper::~TestConnWrapper() {
|
||||||
|
|
|
@ -6,84 +6,15 @@
|
||||||
|
|
||||||
#include <gmock/gmock.h>
|
#include <gmock/gmock.h>
|
||||||
|
|
||||||
|
#include "facade/memcache_parser.h"
|
||||||
|
#include "facade/redis_parser.h"
|
||||||
#include "io/io.h"
|
#include "io/io.h"
|
||||||
#include "server/conn_context.h"
|
#include "server/conn_context.h"
|
||||||
#include "server/main_service.h"
|
#include "server/main_service.h"
|
||||||
#include "server/memcache_parser.h"
|
|
||||||
#include "server/redis_parser.h"
|
|
||||||
#include "util/proactor_pool.h"
|
#include "util/proactor_pool.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
using namespace facade;
|
||||||
class RespMatcher {
|
|
||||||
public:
|
|
||||||
RespMatcher(std::string_view val, RespExpr::Type t = RespExpr::STRING) : type_(t), exp_str_(val) {
|
|
||||||
}
|
|
||||||
|
|
||||||
RespMatcher(int64_t val, RespExpr::Type t = RespExpr::INT64) : type_(t), exp_int_(val) {
|
|
||||||
}
|
|
||||||
|
|
||||||
using is_gtest_matcher = void;
|
|
||||||
|
|
||||||
bool MatchAndExplain(const RespExpr& e, testing::MatchResultListener*) const;
|
|
||||||
|
|
||||||
void DescribeTo(std::ostream* os) const;
|
|
||||||
|
|
||||||
void DescribeNegationTo(std::ostream* os) const;
|
|
||||||
|
|
||||||
private:
|
|
||||||
RespExpr::Type type_;
|
|
||||||
|
|
||||||
std::string exp_str_;
|
|
||||||
int64_t exp_int_;
|
|
||||||
};
|
|
||||||
|
|
||||||
class RespTypeMatcher {
|
|
||||||
public:
|
|
||||||
RespTypeMatcher(RespExpr::Type type) : type_(type) {
|
|
||||||
}
|
|
||||||
|
|
||||||
using is_gtest_matcher = void;
|
|
||||||
|
|
||||||
bool MatchAndExplain(const RespExpr& e, testing::MatchResultListener*) const;
|
|
||||||
|
|
||||||
void DescribeTo(std::ostream* os) const;
|
|
||||||
|
|
||||||
void DescribeNegationTo(std::ostream* os) const;
|
|
||||||
|
|
||||||
private:
|
|
||||||
RespExpr::Type type_;
|
|
||||||
};
|
|
||||||
|
|
||||||
inline ::testing::PolymorphicMatcher<RespMatcher> StrArg(std::string_view str) {
|
|
||||||
return ::testing::MakePolymorphicMatcher(RespMatcher(str));
|
|
||||||
}
|
|
||||||
|
|
||||||
inline ::testing::PolymorphicMatcher<RespMatcher> ErrArg(std::string_view str) {
|
|
||||||
return ::testing::MakePolymorphicMatcher(RespMatcher(str, RespExpr::ERROR));
|
|
||||||
}
|
|
||||||
|
|
||||||
inline ::testing::PolymorphicMatcher<RespMatcher> IntArg(int64_t ival) {
|
|
||||||
return ::testing::MakePolymorphicMatcher(RespMatcher(ival));
|
|
||||||
}
|
|
||||||
|
|
||||||
inline ::testing::PolymorphicMatcher<RespMatcher> ArrLen(size_t len) {
|
|
||||||
return ::testing::MakePolymorphicMatcher(RespMatcher(len, RespExpr::ARRAY));
|
|
||||||
}
|
|
||||||
|
|
||||||
inline ::testing::PolymorphicMatcher<RespTypeMatcher> ArgType(RespExpr::Type t) {
|
|
||||||
return ::testing::MakePolymorphicMatcher(RespTypeMatcher(t));
|
|
||||||
}
|
|
||||||
|
|
||||||
inline bool operator==(const RespExpr& left, const char* s) {
|
|
||||||
return left.type == RespExpr::STRING && ToSV(left.GetBuf()) == s;
|
|
||||||
}
|
|
||||||
|
|
||||||
void PrintTo(const RespExpr::Vec& vec, std::ostream* os);
|
|
||||||
|
|
||||||
MATCHER_P(RespEq, val, "") {
|
|
||||||
return ::testing::ExplainMatchResult(::testing::ElementsAre(StrArg(val)), arg, result_listener);
|
|
||||||
}
|
|
||||||
|
|
||||||
std::vector<int64_t> ToIntArr(const RespVec& vec);
|
std::vector<int64_t> ToIntArr(const RespVec& vec);
|
||||||
|
|
||||||
|
@ -99,7 +30,7 @@ class BaseFamilyTest : public ::testing::Test {
|
||||||
struct TestConnWrapper {
|
struct TestConnWrapper {
|
||||||
::io::StringSink sink; // holds the response blob
|
::io::StringSink sink; // holds the response blob
|
||||||
|
|
||||||
std::unique_ptr<Connection> dummy_conn;
|
std::unique_ptr<facade::Connection> dummy_conn;
|
||||||
|
|
||||||
ConnectionContext cmd_cntx;
|
ConnectionContext cmd_cntx;
|
||||||
std::vector<std::unique_ptr<std::string>> tmp_str_vec;
|
std::vector<std::unique_ptr<std::string>> tmp_str_vec;
|
||||||
|
|
|
@ -306,6 +306,7 @@ void Transaction::SetExecCmd(const CommandId* cid) {
|
||||||
}
|
}
|
||||||
|
|
||||||
unique_shard_cnt_ = 0;
|
unique_shard_cnt_ = 0;
|
||||||
|
args_.clear();
|
||||||
cid_ = cid;
|
cid_ = cid;
|
||||||
cb_ = nullptr;
|
cb_ = nullptr;
|
||||||
}
|
}
|
||||||
|
|
|
@ -13,8 +13,8 @@
|
||||||
#include <vector>
|
#include <vector>
|
||||||
|
|
||||||
#include "core/intent_lock.h"
|
#include "core/intent_lock.h"
|
||||||
#include "core/op_status.h"
|
|
||||||
#include "core/tx_queue.h"
|
#include "core/tx_queue.h"
|
||||||
|
#include "facade/op_status.h"
|
||||||
#include "server/common_types.h"
|
#include "server/common_types.h"
|
||||||
#include "server/table.h"
|
#include "server/table.h"
|
||||||
#include "util/fibers/fibers_ext.h"
|
#include "util/fibers/fibers_ext.h"
|
||||||
|
@ -25,6 +25,9 @@ class DbSlice;
|
||||||
class EngineShardSet;
|
class EngineShardSet;
|
||||||
class EngineShard;
|
class EngineShard;
|
||||||
|
|
||||||
|
using facade::OpStatus;
|
||||||
|
using facade::OpResult;
|
||||||
|
|
||||||
class Transaction {
|
class Transaction {
|
||||||
Transaction(const Transaction&);
|
Transaction(const Transaction&);
|
||||||
void operator=(const Transaction&) = delete;
|
void operator=(const Transaction&) = delete;
|
||||||
|
|
|
@ -6,7 +6,7 @@
|
||||||
|
|
||||||
#include <variant>
|
#include <variant>
|
||||||
|
|
||||||
#include "core/op_status.h"
|
#include "facade/op_status.h"
|
||||||
#include "server/common_types.h"
|
#include "server/common_types.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
Loading…
Reference in New Issue