1
0
Fork 0
mirror of https://github.com/dragonflydb/dragonfly.git synced 2024-12-14 11:58:02 +00:00

chore: get rid of MutableSlice (#3952)

* chore: get rid of MutableSlice

Signed-off-by: Roman Gershman <roman@dragonflydb.io>

* chore: comments

---------

Signed-off-by: Roman Gershman <roman@dragonflydb.io>
This commit is contained in:
Roman Gershman 2024-10-23 21:50:39 +03:00 committed by GitHub
parent 0ebc1a11e1
commit 4aa0ca1ef7
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
33 changed files with 95 additions and 147 deletions

View file

@ -1,13 +0,0 @@
// Copyright 2022, DragonflyDB authors. All rights reserved.
// See LICENSE for licensing terms.
//
#pragma once
#include <absl/types/span.h>
namespace dfly {
using MutableSlice = absl::Span<char>;
using MutSliceSpan = absl::Span<MutableSlice>;
} // namespace dfly

View file

@ -244,7 +244,7 @@ optional<int> FetchKey(lua_State* lua, const char* key) {
return type;
}
void SetGlobalArrayInternal(lua_State* lua, const char* name, MutSliceSpan args) {
void SetGlobalArrayInternal(lua_State* lua, const char* name, Interpreter::SliceSpan args) {
lua_createtable(lua, args.size(), 0);
for (size_t j = 0; j < args.size(); j++) {
lua_pushlstring(lua, args[j].data(), args[j].size());
@ -652,7 +652,7 @@ auto Interpreter::RunFunction(string_view sha, std::string* error) -> RunResult
return err == 0 ? RUN_OK : RUN_ERR;
}
void Interpreter::SetGlobalArray(const char* name, MutSliceSpan args) {
void Interpreter::SetGlobalArray(const char* name, SliceSpan args) {
SetGlobalArrayInternal(lua_, name, args);
}
@ -952,7 +952,7 @@ int Interpreter::RedisGenericCommand(bool raise_error, bool async, ObjectExplore
}
char name_buffer[32]; // backing storage for cmd name
absl::FixedArray<absl::Span<char>, 4> args(argc);
absl::FixedArray<string_view, 4> args(argc);
// Copy command name to name_buffer and set it as first arg.
unsigned name_len = lua_rawlen(lua_, 1);
@ -1004,7 +1004,7 @@ int Interpreter::RedisGenericCommand(bool raise_error, bool async, ObjectExplore
explorer = &*translator;
}
redis_func_(CallArgs{MutSliceSpan{args}, &buffer_, explorer, async, raise_error, &raise_error});
redis_func_(CallArgs{SliceSpan{args}, &buffer_, explorer, async, raise_error, &raise_error});
cmd_depth_--;
// Shrink reusable buffer if it's too big.

View file

@ -4,11 +4,12 @@
#pragma once
#include <absl/types/span.h>
#include <functional>
#include <optional>
#include <string_view>
#include "core/core_types.h"
#include "util/fibers/synchronization.h"
typedef struct lua_State lua_State;
@ -40,10 +41,12 @@ class ObjectExplorer {
class Interpreter {
public:
using SliceSpan = absl::Span<const std::string_view>;
// Arguments received from redis.call
struct CallArgs {
// Full arguments, including cmd name.
MutSliceSpan args;
SliceSpan args;
// Pointer to backing storage for args (excluding cmd name).
// Moving can invalidate arg slice pointers. Moved by async to re-use buffer.
@ -93,7 +96,7 @@ class Interpreter {
RUN_ERR = 2,
};
void SetGlobalArray(const char* name, MutSliceSpan args);
void SetGlobalArray(const char* name, SliceSpan args);
// Runs already added function sha returned by a successful call to AddFunction().
// Returns: true if the call succeeded, otherwise fills error and returns false.

View file

@ -73,6 +73,7 @@ class TestSerializer : public ObjectExplorer {
}
};
using SliceSpan = Interpreter::SliceSpan;
class InterpreterTest : public ::testing::Test {
protected:
InterpreterTest() {
@ -99,12 +100,12 @@ class InterpreterTest : public ::testing::Test {
};
void InterpreterTest::SetGlobalArray(const char* name, const vector<string_view>& vec) {
vector<MutableSlice> slices(vec.size());
vector<string_view> slices(vec.size());
for (size_t i = 0; i < vec.size(); ++i) {
strings_.emplace_back(new string(vec[i]));
slices[i] = MutableSlice{*strings_.back()};
slices[i] = string_view{*strings_.back()};
}
intptr_.SetGlobalArray(name, MutSliceSpan{slices});
intptr_.SetGlobalArray(name, SliceSpan{slices});
}
bool InterpreterTest::Execute(string_view script) {
@ -329,7 +330,7 @@ TEST_F(InterpreterTest, CallArray) {
TEST_F(InterpreterTest, ArgKeys) {
vector<string> vec_arr{};
vector<MutableSlice> slices;
vector<string_view> slices;
SetGlobalArray("ARGV", {"foo", "bar"});
SetGlobalArray("KEYS", {"key1", "key2"});
EXPECT_TRUE(Execute("return {ARGV[1], KEYS[1], KEYS[2]}"));

View file

@ -14,7 +14,6 @@
#include <vector>
#include "base/pmr/memory_resource.h"
#include "core/core_types.h"
#include "core/string_map.h"
namespace dfly::search {

View file

@ -6,8 +6,6 @@
#include <cstdint>
#include <string_view>
#include "core/core_types.h"
namespace dfly {
// blob strings of upto ~256B. Small sizes are probably predominant

View file

@ -1,19 +0,0 @@
// Copyright 2023, Roman Gershman. All rights reserved.
// See LICENSE for licensing terms.
//
#pragma once
#ifdef __linux__
#include "util/fibers/uring_file.h"
#include "util/fibers/uring_proactor.h"
namespace dfly {
using util::fb2::FiberCall;
using util::fb2::LinuxFile;
using util::fb2::OpenLinux;
using util::fb2::OpenRead;
} // namespace dfly
#endif

View file

@ -17,7 +17,6 @@
#include "base/io_buf.h"
#include "base/logging.h"
#include "core/heap_size.h"
#include "core/uring.h"
#include "facade/conn_context.h"
#include "facade/dragonfly_listener.h"
#include "facade/memcache_parser.h"
@ -30,6 +29,10 @@
#include "util/tls/tls_socket.h"
#endif
#ifdef __linux__
#include "util/fibers/uring_file.h"
#endif
using namespace std;
using facade::operator""_MB;
@ -1341,7 +1344,7 @@ bool Connection::ShouldEndDispatchFiber(const MessageHandle& msg) {
void Connection::SquashPipeline() {
DCHECK_EQ(dispatch_q_.size(), pending_pipeline_cmd_cnt_);
vector<CmdArgList> squash_cmds;
vector<ArgSlice> squash_cmds;
squash_cmds.reserve(dispatch_q_.size());
for (auto& msg : dispatch_q_) {

View file

@ -91,7 +91,7 @@ class Connection : public util::Connection {
// The capacity is chosen so that we allocate a fully utilized (256 bytes) block.
using StorageType = absl::InlinedVector<char, kReqStorageSize, mi_stl_allocator<char>>;
absl::InlinedVector<MutableSlice, 6> args;
absl::InlinedVector<std::string_view, 6> args;
StorageType storage;
};

View file

@ -35,16 +35,12 @@ constexpr size_t kSanitizerOverhead = 0u;
enum class Protocol : uint8_t { MEMCACHE = 1, REDIS = 2 };
using MutableSlice = absl::Span<char>;
using CmdArgList = absl::Span<MutableSlice>;
using CmdArgVec = std::vector<MutableSlice>;
using MutableSlice = std::string_view;
using CmdArgList = absl::Span<const std::string_view>;
using CmdArgVec = std::vector<std::string_view>;
using ArgSlice = absl::Span<const std::string_view>;
using OwnedArgSlice = absl::Span<const std::string>;
inline std::string_view ToSV(MutableSlice slice) {
return std::string_view{slice.data(), slice.size()};
}
inline std::string_view ToSV(std::string_view slice) {
return slice;
}
@ -57,13 +53,8 @@ inline std::string_view ToSV(std::string&& slice) = delete;
constexpr auto kToSV = [](auto&& v) { return ToSV(std::forward<decltype(v)>(v)); };
inline std::string_view ArgS(CmdArgList args, size_t i) {
auto arg = args[i];
return {arg.data(), arg.size()};
}
inline auto ArgS(CmdArgList args) {
return base::it::Transform(kToSV, base::it::Range{args.begin(), args.end()});
inline std::string_view ArgS(ArgSlice args, size_t i) {
return args[i];
}
struct ArgRange {
@ -95,7 +86,7 @@ struct ArgRange {
return std::visit([idx](const auto& span) { return facade::ToSV(span[idx]); }, span);
}
std::variant<CmdArgList, ArgSlice, OwnedArgSlice> span;
std::variant<ArgSlice, OwnedArgSlice> span;
};
struct ConnectionStats {
size_t read_buf_capacity = 0; // total capacity of input buffers
@ -179,10 +170,6 @@ struct ErrorReply {
std::optional<OpStatus> status{std::nullopt};
};
inline MutableSlice ToMSS(absl::Span<uint8_t> span) {
return MutableSlice{reinterpret_cast<char*>(span.data()), span.size()};
}
constexpr inline unsigned long long operator""_MB(unsigned long long x) {
return 1024L * 1024L * x;
}

View file

@ -21,11 +21,11 @@ namespace {
class OkService : public ServiceInterface {
public:
void DispatchCommand(CmdArgList args, ConnectionContext* cntx) final {
void DispatchCommand(ArgSlice args, ConnectionContext* cntx) final {
cntx->SendOk();
}
size_t DispatchManyCommands(absl::Span<CmdArgList> args_lists, ConnectionContext* cntx) final {
size_t DispatchManyCommands(absl::Span<ArgSlice> args_lists, ConnectionContext* cntx) final {
for (auto args : args_lists)
DispatchCommand(args, cntx);
return args_lists.size();

View file

@ -13,7 +13,7 @@ void RespExpr::VecToArgList(const Vec& src, CmdArgVec* dest) {
for (size_t i = 0; i < src.size(); ++i) {
DCHECK(src[i].type == RespExpr::STRING);
(*dest)[i] = ToMSS(src[i].GetBuf());
(*dest)[i] = ToSV(src[i].GetBuf());
}
}

View file

@ -25,11 +25,10 @@ class ServiceInterface {
virtual ~ServiceInterface() {
}
virtual void DispatchCommand(CmdArgList args, ConnectionContext* cntx) = 0;
virtual void DispatchCommand(ArgSlice args, ConnectionContext* cntx) = 0;
// Returns number of processed commands
virtual size_t DispatchManyCommands(absl::Span<CmdArgList> args_list,
ConnectionContext* cntx) = 0;
virtual size_t DispatchManyCommands(absl::Span<ArgSlice> args_list, ConnectionContext* cntx) = 0;
virtual void DispatchMC(const MemcacheParser::Command& cmd, std::string_view value,
ConnectionContext* cntx) = 0;

View file

@ -18,7 +18,7 @@ extern "C" {
namespace dfly::acl {
[[nodiscard]] bool IsUserAllowedToInvokeCommand(const ConnectionContext& cntx, const CommandId& id,
CmdArgList tail_args) {
ArgSlice tail_args) {
if (cntx.skip_acl_validation) {
return true;
}

View file

@ -189,8 +189,8 @@ CommandRegistry::FamiliesVec CommandRegistry::GetFamilies() {
return std::move(family_of_commands_);
}
std::pair<const CommandId*, CmdArgList> CommandRegistry::FindExtended(string_view cmd,
CmdArgList tail_args) const {
std::pair<const CommandId*, ArgSlice> CommandRegistry::FindExtended(string_view cmd,
ArgSlice tail_args) const {
if (cmd == RenamedOrOriginal("ACL"sv)) {
if (tail_args.empty()) {
return {Find(cmd), {}};

View file

@ -203,8 +203,8 @@ class CommandRegistry {
using FamiliesVec = std::vector<std::vector<std::string>>;
FamiliesVec GetFamilies();
std::pair<const CommandId*, facade::CmdArgList> FindExtended(std::string_view cmd,
facade::CmdArgList tail_args) const;
std::pair<const CommandId*, facade::ArgSlice> FindExtended(std::string_view cmd,
facade::ArgSlice tail_args) const;
private:
absl::flat_hash_map<std::string, CommandId> cmd_map_;

View file

@ -21,7 +21,7 @@ namespace dfly {
using namespace std;
using namespace facade;
StoredCmd::StoredCmd(const CommandId* cid, CmdArgList args, facade::ReplyMode mode)
StoredCmd::StoredCmd(const CommandId* cid, ArgSlice args, facade::ReplyMode mode)
: cid_{cid}, buffer_{}, sizes_(args.size()), reply_mode_{mode} {
size_t total_size = 0;
for (auto args : args) {
@ -38,7 +38,7 @@ StoredCmd::StoredCmd(const CommandId* cid, CmdArgList args, facade::ReplyMode mo
}
}
StoredCmd::StoredCmd(string&& buffer, const CommandId* cid, CmdArgList args, facade::ReplyMode mode)
StoredCmd::StoredCmd(string&& buffer, const CommandId* cid, ArgSlice args, facade::ReplyMode mode)
: cid_{cid}, buffer_{std::move(buffer)}, sizes_(args.size()), reply_mode_{mode} {
for (unsigned i = 0; i < args.size(); i++) {
// Assume tightly packed list.
@ -47,7 +47,7 @@ StoredCmd::StoredCmd(string&& buffer, const CommandId* cid, CmdArgList args, fac
}
}
void StoredCmd::Fill(CmdArgList args) {
void StoredCmd::Fill(absl::Span<std::string_view> args) {
DCHECK_GE(args.size(), sizes_.size());
unsigned offset = 0;
@ -162,7 +162,7 @@ vector<unsigned> ChangeSubscriptions(bool pattern, CmdArgList args, bool to_add,
// Gather all the channels we need to subscribe to / remove.
size_t i = 0;
for (string_view channel : ArgS(args)) {
for (string_view channel : args) {
if (to_add && local_store.emplace(channel).second)
csu.Record(channel);
else if (!to_add && local_store.erase(channel) > 0)

View file

@ -27,11 +27,10 @@ struct FlowInfo;
// Used for storing MULTI/EXEC commands.
class StoredCmd {
public:
StoredCmd(const CommandId* cid, CmdArgList args,
facade::ReplyMode mode = facade::ReplyMode::FULL);
StoredCmd(const CommandId* cid, ArgSlice args, facade::ReplyMode mode = facade::ReplyMode::FULL);
// Create on top of already filled tightly-packed buffer.
StoredCmd(std::string&& buffer, const CommandId* cid, CmdArgList args,
StoredCmd(std::string&& buffer, const CommandId* cid, ArgSlice args,
facade::ReplyMode mode = facade::ReplyMode::FULL);
size_t NumArgs() const;
@ -40,7 +39,7 @@ class StoredCmd {
// Fill the arg list with stored arguments, it should be at least of size NumArgs().
// Between filling and invocation, cmd should NOT be moved.
void Fill(CmdArgList args);
void Fill(absl::Span<std::string_view> args);
void Fill(CmdArgVec* dest) {
dest->resize(sizes_.size());

View file

@ -145,7 +145,7 @@ void DoPopulateBatch(string_view type, string_view prefix, size_t val_size, bool
local_tx->StartMultiNonAtomic();
boost::intrusive_ptr<Transaction> stub_tx =
new Transaction{local_tx.get(), EngineShard::tlocal()->shard_id(), nullopt};
absl::InlinedVector<MutableSlice, 5> args_view;
absl::InlinedVector<string_view, 5> args_view;
facade::CapturingReplyBuilder crb;
ConnectionContext local_cntx{cntx, stub_tx.get(), &crb};
@ -162,7 +162,7 @@ void DoPopulateBatch(string_view type, string_view prefix, size_t val_size, bool
args_view.clear();
for (auto& arg : args) {
args_view.push_back(absl::MakeSpan(arg));
args_view.push_back(arg);
}
auto args_span = absl::MakeSpan(args_view);

View file

@ -219,9 +219,9 @@ void HttpAPI(const http::QueryArgs& args, HttpRequest&& req, Service* service,
for (size_t i = 0; i < vec.size(); ++i) {
cmd_args.push_back(vec[i].AsString().c_str());
}
vector<facade::MutableSlice> cmd_slices(cmd_args.size());
vector<string_view> cmd_slices(cmd_args.size());
for (size_t i = 0; i < cmd_args.size(); ++i) {
cmd_slices[i] = absl::MakeSpan(cmd_args[i]);
cmd_slices[i] = cmd_args[i];
}
facade::ConnectionContext* context = (facade::ConnectionContext*)http_cntx->user_data();

View file

@ -23,13 +23,13 @@ template <typename... Ts> journal::ParsedEntry::CmdData BuildFromParts(Ts... par
vector<string> raw_parts{absl::StrCat(std::forward<Ts>(parts))...};
auto cmd_str = accumulate(raw_parts.begin(), raw_parts.end(), std::string{});
auto buf = make_unique<char[]>(cmd_str.size());
auto buf = make_unique<uint8_t[]>(cmd_str.size());
memcpy(buf.get(), cmd_str.data(), cmd_str.size());
CmdArgVec slice_parts{};
CmdArgVec slice_parts;
size_t start = 0;
for (const auto& part : raw_parts) {
slice_parts.emplace_back(buf.get() + start, part.size());
slice_parts.emplace_back(reinterpret_cast<char*>(buf.get()) + start, part.size());
start += part.size();
}

View file

@ -140,7 +140,7 @@ template io::Result<uint16_t> JournalReader::ReadUInt<uint16_t>();
template io::Result<uint32_t> JournalReader::ReadUInt<uint32_t>();
template io::Result<uint64_t> JournalReader::ReadUInt<uint64_t>();
io::Result<size_t> JournalReader::ReadString(MutableSlice buffer) {
io::Result<size_t> JournalReader::ReadString(io::MutableBytes buffer) {
size_t size = 0;
SET_OR_UNEXPECT(ReadUInt<uint64_t>(), size);
@ -164,17 +164,17 @@ std::error_code JournalReader::ReadCommand(journal::ParsedEntry::CmdData* data)
SET_OR_RETURN(ReadUInt<uint64_t>(), cmd_size);
// Read all strings consecutively.
data->command_buf = make_unique<char[]>(cmd_size);
char* ptr = data->command_buf.get();
data->command_buf = make_unique<uint8_t[]>(cmd_size);
uint8_t* ptr = data->command_buf.get();
for (auto& span : data->cmd_args) {
size_t size;
SET_OR_RETURN(ReadString({ptr, cmd_size}), size);
DCHECK(size <= cmd_size);
span = MutableSlice{ptr, size};
span = string_view{reinterpret_cast<char*>(ptr), size};
ptr += size;
cmd_size -= size;
}
return std::error_code{};
return {};
}
io::Result<journal::ParsedEntry> JournalReader::ReadEntry() {

View file

@ -26,10 +26,6 @@ class JournalWriter {
private:
void Write(std::string_view sv); // Write string.
void Write(facade::MutableSlice slice) {
Write(facade::ToSV(slice));
}
void Write(const journal::Entry::Payload& payload);
private:
@ -58,7 +54,7 @@ struct JournalReader {
template <typename UT> io::Result<UT> ReadUInt();
// Read and copy to buffer, return size.
io::Result<size_t> ReadString(MutableSlice buffer);
io::Result<size_t> ReadString(io::MutableBytes buffer);
// Read argument array into string buffer.
std::error_code ReadCommand(journal::ParsedEntry::CmdData* entry);

View file

@ -39,14 +39,12 @@ struct Entry : public EntryBase {
// Payload represents a non-owning view into a command executed on the shard.
struct Payload {
std::string_view cmd;
std::variant<CmdArgList, // Parts of a full command.
ShardArgs, // Shard parts.
ArgSlice>
std::variant<ShardArgs, // Shard parts.
ArgSlice> // Parts of a full command.
args;
Payload() = default;
Payload(std::string_view c, CmdArgList a) : cmd(c), args(a) {
}
Payload(std::string_view c, const ShardArgs& a) : cmd(c), args(a) {
}
Payload(std::string_view c, ArgSlice a) : cmd(c), args(a) {
@ -81,7 +79,7 @@ struct Entry : public EntryBase {
struct ParsedEntry : public EntryBase {
struct CmdData {
std::unique_ptr<char[]> command_buf;
std::unique_ptr<uint8_t[]> command_buf;
CmdArgVec cmd_args; // represents the parsed command.
};
CmdData cmd;

View file

@ -1092,7 +1092,7 @@ optional<ErrorReply> CheckKeysDeclared(const ConnectionState::ScriptInfo& eval_i
static optional<ErrorReply> VerifyConnectionAclStatus(const CommandId* cid,
const ConnectionContext* cntx,
string_view error_msg, CmdArgList tail_args) {
string_view error_msg, ArgSlice tail_args) {
// If we are on a squashed context we need to use the owner, because the
// context we are operating on is a stub and the acl username is not copied
// See: MultiCommandSquasher::SquashedHopCb
@ -1238,7 +1238,7 @@ std::optional<ErrorReply> Service::VerifyCommandState(const CommandId* cid, CmdA
return VerifyConnectionAclStatus(cid, &dfly_cntx, "has no ACL permissions", tail_args);
}
void Service::DispatchCommand(CmdArgList args, facade::ConnectionContext* cntx) {
void Service::DispatchCommand(ArgSlice args, facade::ConnectionContext* cntx) {
absl::Cleanup clear_last_error(
[cntx]() { std::ignore = cntx->reply_builder()->ConsumeLastError(); });
DCHECK(!args.empty());
@ -1246,7 +1246,7 @@ void Service::DispatchCommand(CmdArgList args, facade::ConnectionContext* cntx)
ServerState& etl = *ServerState::tlocal();
string cmd = absl::AsciiStrToUpper(ArgS(args, 0));
string cmd = absl::AsciiStrToUpper(args[0]);
const auto [cid, args_no_cmd] = registry_.FindExtended(cmd, args.subspan(1));
if (cid == nullptr) {
@ -1793,7 +1793,7 @@ void Service::Watch(CmdArgList args, ConnectionContext* cntx) {
// Duplicate keys are stored to keep correct count.
exec_info.watched_existed += keys_existed.load(memory_order_relaxed);
for (std::string_view key : ArgS(args)) {
for (string_view key : args) {
exec_info.watched_keys.emplace_back(cntx->db_index(), key);
}
@ -1840,7 +1840,7 @@ optional<CapturingReplyBuilder::Payload> Service::FlushEvalAsyncCmds(ConnectionC
void Service::CallFromScript(ConnectionContext* cntx, Interpreter::CallArgs& ca) {
DCHECK(cntx->transaction);
DVLOG(2) << "CallFromScript " << ArgS(ca.args, 0);
DVLOG(2) << "CallFromScript " << ca.args[0];
InterpreterReplier replier(ca.translator);
facade::SinkReplyBuilder* orig = cntx->Inject(&replier);
@ -1850,7 +1850,7 @@ void Service::CallFromScript(ConnectionContext* cntx, Interpreter::CallArgs& ca)
if (ca.async) {
auto& info = cntx->conn_state.script_info;
string cmd = absl::AsciiStrToUpper(ArgS(ca.args, 0));
string cmd = absl::AsciiStrToUpper(ca.args[0]);
// Full command verification happens during squashed execution
if (auto* cid = registry_.Find(cmd); cid != nullptr) {
@ -1858,7 +1858,7 @@ void Service::CallFromScript(ConnectionContext* cntx, Interpreter::CallArgs& ca)
info->async_cmds.emplace_back(std::move(*ca.buffer), cid, ca.args.subspan(1), replies);
info->async_cmds_heap_mem += info->async_cmds.back().UsedMemory();
} else if (ca.error_abort) { // If we don't abort on errors, we can ignore it completely
findcmd_err = ReportUnknownCmd(ArgS(ca.args, 0));
findcmd_err = ReportUnknownCmd(ca.args[0]);
}
}
@ -2421,7 +2421,7 @@ void Service::PubsubNumSub(CmdArgList args, ConnectionContext* cntx) {
auto* rb = static_cast<RedisReplyBuilder*>(cntx->reply_builder());
rb->StartArray(args.size() * 2);
for (string_view channel : ArgS(args)) {
for (string_view channel : args) {
rb->SendBulkString(channel);
rb->SendLong(ServerState::tlocal()->channel_store()->FetchSubscribers(channel).size());
}

View file

@ -37,10 +37,10 @@ class Service : public facade::ServiceInterface {
void Shutdown();
// Prepare command execution, verify and execute, reply to context
void DispatchCommand(CmdArgList args, facade::ConnectionContext* cntx) final;
void DispatchCommand(ArgSlice args, facade::ConnectionContext* cntx) final;
// Execute multiple consecutive commands, possibly in parallel by squashing
size_t DispatchManyCommands(absl::Span<CmdArgList> args_list,
size_t DispatchManyCommands(absl::Span<ArgSlice> args_list,
facade::ConnectionContext* cntx) final;
// Check VerifyCommandExecution and invoke command with args
@ -55,7 +55,7 @@ class Service : public facade::ServiceInterface {
// Verify command prepares excution in correct state.
// It's usually called before command execution. Only for multi/exec transactions it's checked
// when the command is queued for execution, not before the execution itself.
std::optional<facade::ErrorReply> VerifyCommandState(const CommandId* cid, CmdArgList tail_args,
std::optional<facade::ErrorReply> VerifyCommandState(const CommandId* cid, ArgSlice tail_args,
const ConnectionContext& cntx);
void DispatchMC(const MemcacheParser::Command& cmd, std::string_view value,

View file

@ -246,7 +246,7 @@ struct CmdArgListFormatter {
string UnknownCmd(string cmd, CmdArgList args) {
return absl::StrCat("unknown command '", cmd, "' with args beginning with: ",
StrJoin(args.begin(), args.end(), ", ", CmdArgListFormatter()));
absl::StrJoin(args.begin(), args.end(), ", ", CmdArgListFormatter()));
}
bool IsCloudPath(string_view path) {

View file

@ -43,7 +43,7 @@ using SetType = pair<void*, unsigned>;
namespace {
// Possible sources of new set entries
using NewEntries = std::variant<CmdArgList, ArgSlice, absl::flat_hash_set<std::string_view>>;
using NewEntries = std::variant<ArgSlice, absl::flat_hash_set<std::string_view>>;
auto EntriesRange(const NewEntries& entries) {
return base::it::Wrap(facade::kToSV, entries);

View file

@ -2556,12 +2556,19 @@ void StreamFamily::XPending(CmdArgList args, ConnectionContext* cntx) {
}
void StreamFamily::XRange(CmdArgList args, ConnectionContext* cntx) {
XRangeGeneric(std::move(args), false, cntx);
string_view key = args[0];
string_view start = args[1];
string_view end = args[2];
XRangeGeneric(key, start, end, args.subspan(3), false, cntx);
}
void StreamFamily::XRevRange(CmdArgList args, ConnectionContext* cntx) {
swap(args[1], args[2]);
XRangeGeneric(std::move(args), true, cntx);
string_view key = args[0];
string_view start = args[1];
string_view end = args[2];
XRangeGeneric(key, end, start, args.subspan(3), true, cntx);
}
std::optional<ReadOpts> ParseReadArgsOrReply(CmdArgList args, bool read_group,
@ -3048,10 +3055,8 @@ void StreamFamily::XTrim(CmdArgList args, ConnectionContext* cntx) {
return cntx->SendError(trim_result.status());
}
void StreamFamily::XRangeGeneric(CmdArgList args, bool is_rev, ConnectionContext* cntx) {
string_view key = ArgS(args, 0);
string_view start = ArgS(args, 1);
string_view end = ArgS(args, 2);
void StreamFamily::XRangeGeneric(std::string_view key, std::string_view start, std::string_view end,
CmdArgList args, bool is_rev, ConnectionContext* cntx) {
RangeOpts range_opts;
RangeId rs, re;
if (!ParseRangeId(start, &rs) || !ParseRangeId(end, &re)) {
@ -3066,13 +3071,13 @@ void StreamFamily::XRangeGeneric(CmdArgList args, bool is_rev, ConnectionContext
return cntx->SendError("invalid end ID for the interval", kSyntaxErrType);
}
if (args.size() > 3) {
if (args.size() != 5) {
if (args.size() > 0) {
if (args.size() != 2) {
return cntx->SendError(WrongNumArgsError("XRANGE"), kSyntaxErrType);
}
string opt = absl::AsciiStrToUpper(ArgS(args, 3));
string_view val = ArgS(args, 4);
string opt = absl::AsciiStrToUpper(ArgS(args, 0));
string_view val = ArgS(args, 1);
if (opt != "COUNT" || !absl::SimpleAtoi(val, &range_opts.count)) {
return cntx->SendError(kSyntaxErr);

View file

@ -29,7 +29,8 @@ class StreamFamily {
static void XReadGroup(CmdArgList args, ConnectionContext* cntx);
static void XSetId(CmdArgList args, ConnectionContext* cntx);
static void XTrim(CmdArgList args, ConnectionContext* cntx);
static void XRangeGeneric(CmdArgList args, bool is_rev, ConnectionContext* cntx);
static void XRangeGeneric(std::string_view key, std::string_view start, std::string_view end,
CmdArgList args, bool is_rev, ConnectionContext* cntx);
static void XAck(CmdArgList args, ConnectionContext* cntx);
static void XAutoClaim(CmdArgList args, ConnectionContext* cntx);
};

View file

@ -296,7 +296,7 @@ void Transaction::PrepareMultiFps(CmdArgList keys) {
auto& tag_fps = multi_->tag_fps;
tag_fps.reserve(keys.size());
for (string_view str : ArgS(keys)) {
for (string_view str : keys) {
ShardId sid = Shard(str, shard_set->size());
tag_fps.emplace(sid, LockTag(str).Fingerprint());
}
@ -474,7 +474,7 @@ void Transaction::StartMultiLockedAhead(Namespace* ns, DbIndex dbid, CmdArgList
if (!skip_scheduling)
ScheduleInternal();
full_args_ = {nullptr, 0}; // InitBase set it to temporary keys, now we reset it.
full_args_ = {}; // InitBase set it to temporary keys, now we reset it.
}
void Transaction::StartMultiNonAtomic() {

View file

@ -75,15 +75,6 @@ void TriggerJournalWriteToSink() {
journal->RecordEntry(0, journal::Op::NOOP, 0, 0, nullopt, {}, true);
}
std::ostream& operator<<(std::ostream& os, ArgSlice list) {
os << "[";
if (!list.empty()) {
std::for_each(list.begin(), list.end() - 1, [&os](const auto& val) { os << val << ", "; });
os << (*(list.end() - 1));
}
return os << "]";
}
LockTag::LockTag(std::string_view key) {
if (LockTagOptions::instance().enabled)
str_ = LockTagOptions::instance().Tag(key);

View file

@ -230,6 +230,6 @@ void RecordExpiry(DbIndex dbid, std::string_view key);
// Must be called from shard thread of journal to sink.
void TriggerJournalWriteToSink();
std::ostream& operator<<(std::ostream& os, ArgSlice list);
// std::ostream& operator<<(std::ostream& os, ArgSlice list);
} // namespace dfly