From 62daa73f03fb9675e5cdbcb578f18017d6196864 Mon Sep 17 00:00:00 2001 From: Ruixiang Tan <819464715@qq.com> Date: Wed, 21 Sep 2022 00:14:11 +0800 Subject: [PATCH 01/22] Add the hrange command (#895) --- src/redis_cmd.cc | 55 ++++++++++++-- src/redis_hash.cc | 34 +++++++++ src/redis_hash.h | 2 + tests/cppunit/t_hash_test.cc | 33 +++++++++ tests/gocase/unit/command/command_test.go | 4 +- tests/gocase/unit/type/hash/hash_test.go | 87 +++++++++++++++++++++++ 6 files changed, 206 insertions(+), 9 deletions(-) create mode 100644 tests/gocase/unit/type/hash/hash_test.go diff --git a/src/redis_cmd.cc b/src/redis_cmd.cc index 8943dd0c7cd..89fb81f2f37 100644 --- a/src/redis_cmd.cc +++ b/src/redis_cmd.cc @@ -23,8 +23,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -1427,10 +1429,11 @@ class CommandHVals : public Commander { if (!s.ok()) { return Status(Status::RedisExecErr, s.ToString()); } - *output = "*" + std::to_string(field_values.size()) + CRLF; - for (const auto &fv : field_values) { - *output += Redis::BulkString(fv.value); + std::vector values; + for (const auto &p : field_values) { + values.emplace_back(p.value); } + *output = MultiBulkString(values); return Status::OK(); } }; @@ -1444,13 +1447,50 @@ class CommandHGetAll : public Commander { if (!s.ok()) { return Status(Status::RedisExecErr, s.ToString()); } - *output = "*" + std::to_string(field_values.size() * 2) + CRLF; - for (const auto &fv : field_values) { - *output += Redis::BulkString(fv.field); - *output += Redis::BulkString(fv.value); + std::vector kv_pairs; + for (const auto &p : field_values) { + kv_pairs.emplace_back(p.field); + kv_pairs.emplace_back(p.value); + } + *output = MultiBulkString(kv_pairs); + return Status::OK(); + } +}; + +class CommandHRange : public Commander { + public: + Status Parse(const std::vector &args) override { + if (args.size() != 6 && args.size() != 4) { + return Status(Status::RedisParseErr, errWrongNumOfArguments); + } + if (args.size() == 6 && Util::ToLower(args[4]) != "limit") { + return Status(Status::RedisInvalidCmd, errInvalidSyntax); } + if (args.size() == 6) { + auto parse_result = ParseInt(args_[5], 10); + if (!parse_result)return Status(Status::RedisParseErr, errValueNotInterger); + limit_ = *parse_result; + } + return Commander::Parse(args); + } + Status Execute(Server *svr, Connection *conn, std::string *output) override { + Redis::Hash hash_db(svr->storage_, conn->GetNamespace()); + std::vector field_values; + rocksdb::Status s = hash_db.Range(args_[1], args_[2], args_[3], limit_, &field_values); + if (!s.ok()) { + return Status(Status::RedisExecErr, s.ToString()); + } + std::vector kv_pairs; + for (const auto &p : field_values) { + kv_pairs.emplace_back(p.field); + kv_pairs.emplace_back(p.value); + } + *output = MultiBulkString(kv_pairs); return Status::OK(); } + + private: + int64_t limit_ = LONG_MAX; }; class CommandPush : public Commander { @@ -5823,6 +5863,7 @@ CommandAttributes redisCommandTable[] = { ADD_CMD("hvals", 2, "read-only", 1, 1, 1, CommandHVals), ADD_CMD("hgetall", 2, "read-only", 1, 1, 1, CommandHGetAll), ADD_CMD("hscan", -3, "read-only", 1, 1, 1, CommandHScan), + ADD_CMD("hrange", -4, "read-only", 1, 1, 1, CommandHRange), ADD_CMD("lpush", -3, "write", 1, 1, 1, CommandLPush), ADD_CMD("rpush", -3, "write", 1, 1, 1, CommandRPush), diff --git a/src/redis_hash.cc b/src/redis_hash.cc index 36c1597b4b2..078aa2af62c 100644 --- a/src/redis_hash.cc +++ b/src/redis_hash.cc @@ -20,6 +20,7 @@ #include "redis_hash.h" #include +#include #include #include #include @@ -274,6 +275,39 @@ rocksdb::Status Hash::MSet(const Slice &user_key, const std::vector return storage_->Write(rocksdb::WriteOptions(), &batch); } +rocksdb::Status Hash::Range(const Slice &user_key, const Slice &start, const Slice &stop, + int64_t limit, std::vector *field_values) { + field_values->clear(); + if (start.compare(stop) >= 0 || limit <= 0) { + return rocksdb::Status::OK(); + } + std::string ns_key; + AppendNamespacePrefix(user_key, &ns_key); + HashMetadata metadata(false); + rocksdb::Status s = GetMetadata(ns_key, &metadata); + limit = std::min(static_cast(metadata.size), limit); + std::string start_key, stop_key; + InternalKey(ns_key, start, metadata.version, storage_->IsSlotIdEncoded()).Encode(&start_key); + InternalKey(ns_key, stop, metadata.version, storage_->IsSlotIdEncoded()).Encode(&stop_key); + rocksdb::ReadOptions read_options; + LatestSnapShot ss(db_); + read_options.snapshot = ss.GetSnapShot(); + rocksdb::Slice upper_bound(stop_key); + read_options.iterate_upper_bound = &upper_bound; + read_options.fill_cache = false; + + auto iter = DBUtil::UniqueIterator(db_, read_options); + iter->Seek(start_key); + for (int i = 0; iter->Valid() && i <= limit - 1; i++) { + FieldValue tmp_field_value; + InternalKey ikey(iter->key(), storage_->IsSlotIdEncoded()); + tmp_field_value.field = ikey.GetSubKey().ToString(); + tmp_field_value.value = iter->value().ToString(); + field_values->emplace_back(tmp_field_value); + iter->Next(); + } + return rocksdb::Status::OK(); +} rocksdb::Status Hash::GetAll(const Slice &user_key, std::vector *field_values, HashFetchType type) { field_values->clear(); diff --git a/src/redis_hash.h b/src/redis_hash.h index bd58e42dbfe..9667f783e21 100644 --- a/src/redis_hash.h +++ b/src/redis_hash.h @@ -52,6 +52,8 @@ class Hash : public SubKeyScanner { rocksdb::Status IncrBy(const Slice &user_key, const Slice &field, int64_t increment, int64_t *ret); rocksdb::Status IncrByFloat(const Slice &user_key, const Slice &field, double increment, double *ret); rocksdb::Status MSet(const Slice &user_key, const std::vector &field_values, bool nx, int *ret); + rocksdb::Status Range(const Slice &user_key, const Slice &start, const Slice& stop, + int64_t limit, std::vector *field_values); rocksdb::Status MGet(const Slice &user_key, const std::vector &fields, std::vector *values, diff --git a/tests/cppunit/t_hash_test.cc b/tests/cppunit/t_hash_test.cc index f1153c61bb2..27f612bbb78 100644 --- a/tests/cppunit/t_hash_test.cc +++ b/tests/cppunit/t_hash_test.cc @@ -20,6 +20,9 @@ #include #include +#include +#include +#include #include "test_base.h" #include "redis_hash.h" @@ -153,4 +156,34 @@ TEST_F(RedisHashTest, HIncrByFloat) { value = std::stof(bytes); EXPECT_FLOAT_EQ(32*1.2, value); hash->Del(key_); +} + +TEST_F(RedisHashTest, HRange) { + int ret; + std::vector fvs; + for (size_t i = 0; i < 4; i++) { + fvs.emplace_back(FieldValue{"key" + std::to_string(i), "value" + std::to_string(i)}); + } + for (size_t i = 0; i < 26; i++) { + fvs.emplace_back(FieldValue{std::to_string(char(i + 'a')), std::to_string(char(i + 'a'))}); + } + + std::random_device rd; + std::mt19937 g(rd()); + std::vector tmp(fvs); + for (size_t i =0; i < 100 ; i ++) { + std::shuffle(tmp.begin(), tmp.end(), g); + rocksdb::Status s = hash->MSet(key_, tmp, false, &ret); + EXPECT_TRUE(s.ok() && static_cast(tmp.size()) == ret); + s = hash->MSet(key_, fvs, false, &ret); + EXPECT_EQ(ret ,0); + std::vector result; + s = hash->Range(key_, "key0", "key4", INT_MAX, &result); + EXPECT_TRUE(s.ok()); + EXPECT_EQ(4, result.size()); + EXPECT_EQ("key0", result[0].field); + EXPECT_EQ("key1", result[1].field); + EXPECT_EQ("key2", result[2].field); + hash->Del(key_); + } } \ No newline at end of file diff --git a/tests/gocase/unit/command/command_test.go b/tests/gocase/unit/command/command_test.go index 46c4e4c42d7..b5a87fc92b6 100644 --- a/tests/gocase/unit/command/command_test.go +++ b/tests/gocase/unit/command/command_test.go @@ -35,11 +35,11 @@ func TestCommand(t *testing.T) { rdb := srv.NewClient() defer func() { require.NoError(t, rdb.Close()) }() - t.Run("Kvrocks supports 181 commands currently", func(t *testing.T) { + t.Run("Kvrocks supports 182 commands currently", func(t *testing.T) { r := rdb.Do(ctx, "COMMAND", "COUNT") v, err := r.Int() require.NoError(t, err) - require.Equal(t, 181, v) + require.Equal(t, 182, v) }) t.Run("acquire GET command info by COMMAND INFO", func(t *testing.T) { diff --git a/tests/gocase/unit/type/hash/hash_test.go b/tests/gocase/unit/type/hash/hash_test.go new file mode 100644 index 00000000000..349f88b421a --- /dev/null +++ b/tests/gocase/unit/type/hash/hash_test.go @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package hash + +import ( + "context" + "testing" + + "github.com/apache/incubator-kvrocks/tests/gocase/util" + "github.com/stretchr/testify/require" +) + +func TestHash(t *testing.T) { + srv := util.StartServer(t, map[string]string{}) + defer srv.Close() + ctx := context.Background() + rdb := srv.NewClient() + defer func() { require.NoError(t, rdb.Close()) }() + kvArray := []string{"a", "a", "b", "b", "c", "c", "d", "d", "e", "e", "key1", "value1", "key2", "value2", "key3", "value3", "key10", "value10", "z", "z", "x", "x"} + t.Run("HRange normal situation ", func(t *testing.T) { + require.NoError(t, rdb.Del(ctx, "hashkey").Err()) + require.NoError(t, rdb.HMSet(ctx, "hashkey", kvArray).Err()) + require.EqualValues(t, []interface{}{"key1", "value1", "key10", "value10"}, rdb.Do(ctx, "HRange", "hashkey", "key1", "key2", "limit", 100).Val()) + require.EqualValues(t, []interface{}{"key1", "value1", "key10", "value10", "key2", "value2"}, rdb.Do(ctx, "HRange", "hashkey", "key1", "key3", "limit", 100).Val()) + }) + + t.Run("HRange stop <= start", func(t *testing.T) { + require.NoError(t, rdb.Del(ctx, "hashkey").Err()) + require.NoError(t, rdb.HMSet(ctx, "hashkey", kvArray).Err()) + require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HRange", "hashkey", "key2", "key1", "limit", 100).Val()) + require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HRange", "hashkey", "key1", "key1", "limit", 100).Val()) + }) + + t.Run("HRange limit", func(t *testing.T) { + require.NoError(t, rdb.Del(ctx, "hashkey").Err()) + require.NoError(t, rdb.HMSet(ctx, "hashkey", kvArray).Err()) + require.EqualValues(t, []interface{}{"a", "a", "b", "b"}, rdb.Do(ctx, "HRange", "hashkey", "a", "z", "limit", 2).Val()) + require.EqualValues(t, []interface{}{"a", "a", "b", "b", "c", "c", "d", "d", "e", "e", "key1", "value1", "key10", "value10", "key2", "value2", "key3", "value3", "x", "x", "z", "z"}, rdb.Do(ctx, "HRange", "hashkey", "a", "zzz", "limit", 10000).Val()) + }) + + t.Run("HRange limit is negative", func(t *testing.T) { + require.NoError(t, rdb.Del(ctx, "hashkey").Err()) + require.NoError(t, rdb.HMSet(ctx, "hashkey", kvArray).Err()) + require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HRange", "hashkey", "a", "z", "limit", -100).Val()) + require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HRange", "hashkey", "a", "z", "limit", 0).Val()) + }) + + t.Run("HRange nonexistent key", func(t *testing.T) { + require.NoError(t, rdb.Del(ctx, "hashkey").Err()) + require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HRange", "hashkey", "a", "z", "limit", 10000).Val()) + require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HRange", "hashkey", "a", "z", "limit", 10000).Val()) + }) + + t.Run("HRange limit typo", func(t *testing.T) { + require.NoError(t, rdb.Del(ctx, "hashkey").Err()) + require.NoError(t, rdb.HMSet(ctx, "hashkey", kvArray).Err()) + require.ErrorContains(t, rdb.Do(ctx, "HRange", "hashkey", "a", "z", "limitzz", 10000).Err(), "ERR syntax") + }) + + t.Run("HRange wrong number of arguments", func(t *testing.T) { + require.NoError(t, rdb.Del(ctx, "hashkey").Err()) + require.NoError(t, rdb.HMSet(ctx, "hashkey", kvArray).Err()) + require.ErrorContains(t, rdb.Do(ctx, "HRange", "hashkey", "a", "z", "limit", 10000, "a").Err(), "wrong number of arguments") + require.ErrorContains(t, rdb.Do(ctx, "HRange", "hashkey", "a", "z", "limit").Err(), "wrong number of arguments") + require.ErrorContains(t, rdb.Do(ctx, "HRange", "hashkey", "a").Err(), "wrong number of arguments") + require.ErrorContains(t, rdb.Do(ctx, "HRange", "hashkey").Err(), "wrong number of arguments") + require.ErrorContains(t, rdb.Do(ctx, "HRange").Err(), "wrong number of arguments") + }) + +} From 01c2d87ce0d8e8bcd4cd2f0ff5ae9150040dad8d Mon Sep 17 00:00:00 2001 From: Andrei Vydrin Date: Tue, 20 Sep 2022 23:48:05 +0700 Subject: [PATCH 02/22] Export the rocksdb write options (#885) --- kvrocks.conf | 39 ++++++++++++++++++++++++++++++++++++++ src/config.cc | 8 ++++++++ src/config.h | 10 +++++++++- src/redis_bitmap.cc | 6 +++--- src/redis_bitmap_string.cc | 2 +- src/redis_cmd.cc | 3 +++ src/redis_db.cc | 4 ++-- src/redis_hash.cc | 8 ++++---- src/redis_list.cc | 18 +++++++++--------- src/redis_pubsub.cc | 2 +- src/redis_set.cc | 8 ++++---- src/redis_sortedint.cc | 4 ++-- src/redis_stream.cc | 6 +++--- src/redis_string.cc | 10 +++++----- src/redis_zset.cc | 14 +++++++------- src/server.cc | 2 +- src/storage.cc | 17 +++++++++++++---- src/storage.h | 4 ++++ 18 files changed, 118 insertions(+), 47 deletions(-) diff --git a/kvrocks.conf b/kvrocks.conf index 3847f07c360..2ef247bd698 100644 --- a/kvrocks.conf +++ b/kvrocks.conf @@ -711,5 +711,44 @@ rocksdb.max_bytes_for_level_base 268435456 # Default: 10 rocksdb.max_bytes_for_level_multiplier 10 +# If yes, the write will be flushed from the operating system +# buffer cache before the write is considered complete. +# If this flag is enabled, writes will be slower. +# If this flag is disabled, and the machine crashes, some recent +# rites may be lost. Note that if it is just the process that +# crashes (i.e., the machine does not reboot), no writes will be +# lost even if sync==false. +# +# Default: no +rocksdb.write_options.sync no + +# If yes, writes will not first go to the write ahead log, +# and the write may get lost after a crash. +# +# Deafult: no +rocksdb.write_options.disable_wal no + +# If enabled and we need to wait or sleep for the write request, fails +# immediately. +# +# Default: no +rocksdb.write_options.no_slowdown no + +# If enabled, write requests are of lower priority if compaction is +# behind. In this case, no_slowdown = true, the request will be canceled +# immediately. Otherwise, it will be slowed down. +# The slowdown value is determined by RocksDB to guarantee +# it introduces minimum impacts to high priority writes. +# +# Default: no +rocksdb.write_options.low_pri no + +# If enabled, this writebatch will maintain the last insert positions of each +# memtable as hints in concurrent write. It can improve write performance +# in concurrent writes if keys in one writebatch are sequential. +# +# Default: no +rocksdb.write_options.memtable_insert_hint_per_batch no + ################################ NAMESPACE ##################################### # namespace.test change.me diff --git a/src/config.cc b/src/config.cc index a3cc93acfc0..16c097282fa 100644 --- a/src/config.cc +++ b/src/config.cc @@ -203,6 +203,14 @@ Config::Config() { false, new IntField(&RocksDB.max_bytes_for_level_multiplier, 10, 1, 100)}, {"rocksdb.level_compaction_dynamic_level_bytes", false, new YesNoField(&RocksDB.level_compaction_dynamic_level_bytes, false)}, + + /* rocksdb write options */ + {"rocksdb.write_options.sync", true, new YesNoField(&RocksDB.write_options.sync, false)}, + {"rocksdb.write_options.disable_wal", true, new YesNoField(&RocksDB.write_options.disable_WAL, false)}, + {"rocksdb.write_options.no_slowdown", true, new YesNoField(&RocksDB.write_options.no_slowdown, false)}, + {"rocksdb.write_options.low_pri", true, new YesNoField(&RocksDB.write_options.low_pri, false)}, + {"rocksdb.write_options.memtable_insert_hint_per_batch", + true, new YesNoField(&RocksDB.write_options.memtable_insert_hint_per_batch, false)}, }; for (auto &wrapper : fields) { auto &field = wrapper.field; diff --git a/src/config.h b/src/config.h index 2eba3f47f7b..dec091a9fd5 100644 --- a/src/config.h +++ b/src/config.h @@ -141,7 +141,7 @@ struct Config{ std::set profiling_sample_commands; bool profiling_sample_all_commands = false; - struct { + struct RocksDB { int block_size; bool cache_index_and_filter_blocks; int metadata_block_cache_size; @@ -175,6 +175,14 @@ struct Config{ int max_bytes_for_level_base; int max_bytes_for_level_multiplier; bool level_compaction_dynamic_level_bytes; + + struct WriteOptions { + bool sync; + bool disable_WAL; + bool no_slowdown; + bool low_pri; + bool memtable_insert_hint_per_batch; + } write_options; } RocksDB; public: diff --git a/src/redis_bitmap.cc b/src/redis_bitmap.cc index 4fbcfe4a0ac..02b316b8f31 100644 --- a/src/redis_bitmap.cc +++ b/src/redis_bitmap.cc @@ -232,7 +232,7 @@ rocksdb::Status Bitmap::SetBit(const Slice &user_key, uint32_t offset, bool new_ metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Bitmap::BitCount(const Slice &user_key, int64_t start, int64_t stop, uint32_t *cnt) { @@ -384,7 +384,7 @@ rocksdb::Status Bitmap::BitOp(BitOpFlags op_flag, const std::string &op_name, rocksdb::WriteBatch batch; if (max_size == 0) { batch.Delete(metadata_cf_handle_, ns_key); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } std::vector log_args = {std::to_string(kRedisCmdBitOp), op_name}; for (const auto &op_key : op_keys) { @@ -535,7 +535,7 @@ rocksdb::Status Bitmap::BitOp(BitOpFlags op_flag, const std::string &op_name, res_metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); *len = max_size; - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } bool Bitmap::GetBitFromValueAndOffset(const std::string &value, uint32_t offset) { diff --git a/src/redis_bitmap_string.cc b/src/redis_bitmap_string.cc index 8badca1ed4b..acb2f735a77 100644 --- a/src/redis_bitmap_string.cc +++ b/src/redis_bitmap_string.cc @@ -64,7 +64,7 @@ rocksdb::Status BitmapString::SetBit(const Slice &ns_key, WriteBatchLogData log_data(kRedisString); batch.PutLogData(log_data.Encode()); batch.Put(metadata_cf_handle_, ns_key, *raw_value); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status BitmapString::BitCount(const std::string &raw_value, int64_t start, int64_t stop, uint32_t *cnt) { diff --git a/src/redis_cmd.cc b/src/redis_cmd.cc index 89fb81f2f37..9ece7bf95ea 100644 --- a/src/redis_cmd.cc +++ b/src/redis_cmd.cc @@ -3731,6 +3731,9 @@ class CommandSlaveOf : public Commander { if (svr->GetConfig()->cluster_enabled) { return Status(Status::RedisExecErr, "can't change to slave in cluster mode"); } + if (svr->GetConfig()->RocksDB.write_options.disable_WAL) { + return Status(Status::RedisExecErr, "slaveof doesn't work with disable_wal option"); + } if (!conn->IsAdmin()) { *output = Redis::Error(errAdministorPermissionRequired); return Status::OK(); diff --git a/src/redis_db.cc b/src/redis_db.cc index 8f86c9aa51c..b748146be89 100644 --- a/src/redis_db.cc +++ b/src/redis_db.cc @@ -99,7 +99,7 @@ rocksdb::Status Database::Expire(const Slice &user_key, int timestamp) { WriteBatchLogData log_data(kRedisNone, {std::to_string(kRedisCmdExpire)}); batch.PutLogData(log_data.Encode()); batch.Put(metadata_cf_handle_, ns_key, Slice(buf, value.size())); - s = storage_->Write(rocksdb::WriteOptions(), &batch); + s = storage_->Write(storage_->DefaultWriteOptions(), &batch); delete[]buf; return s; } @@ -117,7 +117,7 @@ rocksdb::Status Database::Del(const Slice &user_key) { if (metadata.Expired()) { return rocksdb::Status::NotFound(kErrMsgKeyExpired); } - return storage_->Delete(rocksdb::WriteOptions(), metadata_cf_handle_, ns_key); + return storage_->Delete(storage_->DefaultWriteOptions(), metadata_cf_handle_, ns_key); } rocksdb::Status Database::Exists(const std::vector &keys, int *ret) { diff --git a/src/redis_hash.cc b/src/redis_hash.cc index 078aa2af62c..0fbf407408c 100644 --- a/src/redis_hash.cc +++ b/src/redis_hash.cc @@ -105,7 +105,7 @@ rocksdb::Status Hash::IncrBy(const Slice &user_key, const Slice &field, int64_t metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Hash::IncrByFloat(const Slice &user_key, const Slice &field, double increment, double *ret) { @@ -155,7 +155,7 @@ rocksdb::Status Hash::IncrByFloat(const Slice &user_key, const Slice &field, dou metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Hash::MGet(const Slice &user_key, @@ -231,7 +231,7 @@ rocksdb::Status Hash::Delete(const Slice &user_key, const std::vector &fi std::string bytes; metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Hash::MSet(const Slice &user_key, const std::vector &field_values, bool nx, int *ret) { @@ -272,7 +272,7 @@ rocksdb::Status Hash::MSet(const Slice &user_key, const std::vector metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Hash::Range(const Slice &user_key, const Slice &start, const Slice &stop, diff --git a/src/redis_list.cc b/src/redis_list.cc index 16fe7791964..a354f377eb8 100644 --- a/src/redis_list.cc +++ b/src/redis_list.cc @@ -88,7 +88,7 @@ rocksdb::Status List::push(const Slice &user_key, metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); *ret = metadata.size; - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status List::Pop(const Slice &user_key, bool left, std::string *elem) { @@ -147,7 +147,7 @@ rocksdb::Status List::PopMulti(const rocksdb::Slice &user_key, bool left, uint32 batch.Put(metadata_cf_handle_, ns_key, bytes); } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } /* @@ -262,7 +262,7 @@ rocksdb::Status List::Rem(const Slice &user_key, int count, const Slice &elem, i } *ret = static_cast(to_delete_indexes.size()); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status List::Insert(const Slice &user_key, const Slice &pivot, const Slice &elem, bool before, int *ret) { @@ -347,7 +347,7 @@ rocksdb::Status List::Insert(const Slice &user_key, const Slice &pivot, const Sl batch.Put(metadata_cf_handle_, ns_key, bytes); *ret = metadata.size; - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status List::Index(const Slice &user_key, int index, std::string *elem) { @@ -447,7 +447,7 @@ rocksdb::Status List::Set(const Slice &user_key, int index, Slice elem) { log_data(kRedisList, {std::to_string(kRedisCmdLSet), std::to_string(index)}); batch.PutLogData(log_data.Encode()); batch.Put(sub_key, elem); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status List::RPopLPush(const Slice &src, const Slice &dst, std::string *elem) { @@ -534,7 +534,7 @@ rocksdb::Status List::lmoveOnSingleList(const rocksdb::Slice &src, bool src_left metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status List::lmoveOnTwoLists(const rocksdb::Slice &src, const rocksdb::Slice &dst, @@ -598,7 +598,7 @@ rocksdb::Status List::lmoveOnTwoLists(const rocksdb::Slice &src, const rocksdb:: dst_metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, dst_ns_key, bytes); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } // Caution: trim the big list may block the server @@ -617,7 +617,7 @@ rocksdb::Status List::Trim(const Slice &user_key, int start, int stop) { // the result will be empty list when start > stop, // or start is larger than the end of list if (start > stop) { - return storage_->Delete(rocksdb::WriteOptions(), metadata_cf_handle_, ns_key); + return storage_->Delete(storage_->DefaultWriteOptions(), metadata_cf_handle_, ns_key); } if (start < 0) start = 0; @@ -655,6 +655,6 @@ rocksdb::Status List::Trim(const Slice &user_key, int start, int stop) { std::string bytes; metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } } // namespace Redis diff --git a/src/redis_pubsub.cc b/src/redis_pubsub.cc index 5be6ff83bf4..78c02976a95 100644 --- a/src/redis_pubsub.cc +++ b/src/redis_pubsub.cc @@ -24,6 +24,6 @@ namespace Redis { rocksdb::Status PubSub::Publish(const Slice &channel, const Slice &value) { rocksdb::WriteBatch batch; batch.Put(pubsub_cf_handle_, channel, value); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } } // namespace Redis diff --git a/src/redis_set.cc b/src/redis_set.cc index 840214989b3..b789ffa652c 100644 --- a/src/redis_set.cc +++ b/src/redis_set.cc @@ -51,7 +51,7 @@ rocksdb::Status Set::Overwrite(Slice user_key, const std::vector &m std::string bytes; metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Set::Add(const Slice &user_key, const std::vector &members, int *ret) { @@ -83,7 +83,7 @@ rocksdb::Status Set::Add(const Slice &user_key, const std::vector &member metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Set::Remove(const Slice &user_key, const std::vector &members, int *ret) { @@ -118,7 +118,7 @@ rocksdb::Status Set::Remove(const Slice &user_key, const std::vector &mem batch.Delete(metadata_cf_handle_, ns_key); } } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Set::Card(const Slice &user_key, int *ret) { @@ -244,7 +244,7 @@ rocksdb::Status Set::Take(const Slice &user_key, std::vector *membe metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Set::Move(const Slice &src, const Slice &dst, const Slice &member, int *ret) { diff --git a/src/redis_sortedint.cc b/src/redis_sortedint.cc index e1b3703cf00..97e5feca95f 100644 --- a/src/redis_sortedint.cc +++ b/src/redis_sortedint.cc @@ -63,7 +63,7 @@ rocksdb::Status Sortedint::Add(const Slice &user_key, std::vector ids, metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Sortedint::Remove(const Slice &user_key, std::vector ids, int *ret) { @@ -95,7 +95,7 @@ rocksdb::Status Sortedint::Remove(const Slice &user_key, std::vector i std::string bytes; metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Sortedint::Card(const Slice &user_key, int *ret) { diff --git a/src/redis_stream.cc b/src/redis_stream.cc index 72c4ea95800..184e8f6dc9c 100644 --- a/src/redis_stream.cc +++ b/src/redis_stream.cc @@ -155,7 +155,7 @@ rocksdb::Status Stream::Add(const Slice &stream_name, const StreamAddOptions& op *id = next_entry_id; - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Stream::getNextEntryID(const StreamMetadata &metadata, const StreamAddOptions &options, @@ -296,7 +296,7 @@ rocksdb::Status Stream::DeleteEntries(const rocksdb::Slice &stream_name, batch.Put(metadata_cf_handle_, ns_key, bytes); } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status Stream::Len(const rocksdb::Slice &stream_name, uint64_t *ret) { @@ -525,7 +525,7 @@ rocksdb::Status Stream::Trim(const rocksdb::Slice &stream_name, const StreamTrim metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } return rocksdb::Status::OK(); diff --git a/src/redis_string.cc b/src/redis_string.cc index 20ea14c6dc4..67e5e0d88b0 100644 --- a/src/redis_string.cc +++ b/src/redis_string.cc @@ -103,7 +103,7 @@ rocksdb::Status String::updateRawValue(const std::string &ns_key, const std::str WriteBatchLogData log_data(kRedisString); batch.PutLogData(log_data.Encode()); batch.Put(metadata_cf_handle_, ns_key, raw_value); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status String::Append(const std::string &user_key, const std::string &value, int *ret) { @@ -173,7 +173,7 @@ rocksdb::Status String::GetDel(const std::string &user_key, std::string *value) rocksdb::Status s = getValue(ns_key, value); if (!s.ok()) return s; - return storage_->Delete(rocksdb::WriteOptions(), metadata_cf_handle_, ns_key); + return storage_->Delete(storage_->DefaultWriteOptions(), metadata_cf_handle_, ns_key); } rocksdb::Status String::Set(const std::string &user_key, const std::string &value) { @@ -357,7 +357,7 @@ rocksdb::Status String::MSet(const std::vector &pairs, int ttl) { AppendNamespacePrefix(pair.key, &ns_key); batch.Put(metadata_cf_handle_, ns_key, bytes); LockGuard guard(storage_->GetLockManager(), ns_key); - auto s = storage_->Write(rocksdb::WriteOptions(), &batch); + auto s = storage_->Write(storage_->DefaultWriteOptions(), &batch); if (!s.ok()) return s; } return rocksdb::Status::OK(); @@ -398,7 +398,7 @@ rocksdb::Status String::MSetNX(const std::vector &pairs, int ttl, in WriteBatchLogData log_data(kRedisString); batch.PutLogData(log_data.Encode()); batch.Put(metadata_cf_handle_, ns_key, bytes); - auto s = storage_->Write(rocksdb::WriteOptions(), &batch); + auto s = storage_->Write(storage_->DefaultWriteOptions(), &batch); if (!s.ok()) return s; } *ret = 1; @@ -472,7 +472,7 @@ rocksdb::Status String::CAD(const std::string &user_key, const std::string &valu } if (value == current_value) { - auto delete_status = storage_->Delete(rocksdb::WriteOptions(), + auto delete_status = storage_->Delete(storage_->DefaultWriteOptions(), storage_->GetCFHandle(Engine::kMetadataColumnFamilyName), ns_key); if (!delete_status.ok()) { diff --git a/src/redis_zset.cc b/src/redis_zset.cc index f29e4dc7089..a1fa0e31899 100644 --- a/src/redis_zset.cc +++ b/src/redis_zset.cc @@ -113,7 +113,7 @@ rocksdb::Status ZSet::Add(const Slice &user_key, uint8_t flags, std::vectorWrite(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status ZSet::Card(const Slice &user_key, int *ret) { @@ -204,7 +204,7 @@ rocksdb::Status ZSet::Pop(const Slice &user_key, int count, bool min, std::vecto metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status ZSet::Range(const Slice &user_key, int start, int stop, uint8_t flags, std::vector @@ -280,7 +280,7 @@ rocksdb::Status ZSet::Range(const Slice &user_key, int start, int stop, uint8_t std::string bytes; metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } return rocksdb::Status::OK(); } @@ -400,7 +400,7 @@ rocksdb::Status ZSet::RangeByScore(const Slice &user_key, std::string bytes; metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } return rocksdb::Status::OK(); } @@ -494,7 +494,7 @@ rocksdb::Status ZSet::RangeByLex(const Slice &user_key, std::string bytes; metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } return rocksdb::Status::OK(); } @@ -552,7 +552,7 @@ rocksdb::Status ZSet::Remove(const Slice &user_key, const std::vector &me metadata.Encode(&bytes); batch.Put(metadata_cf_handle_, ns_key, bytes); } - return storage_->Write(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status ZSet::RemoveRangeByScore(const Slice &user_key, ZRangeSpec spec, int *ret) { @@ -648,7 +648,7 @@ rocksdb::Status ZSet::Overwrite(const Slice &user_key, const std::vectorWrite(rocksdb::WriteOptions(), &batch); + return storage_->Write(storage_->DefaultWriteOptions(), &batch); } rocksdb::Status ZSet::InterStore(const Slice &dst, diff --git a/src/server.cc b/src/server.cc index 7f4ba93d3c9..6818d6da49b 100644 --- a/src/server.cc +++ b/src/server.cc @@ -1444,7 +1444,7 @@ void Server::ScriptReset() { void Server::ScriptFlush() { auto cf = storage_->GetCFHandle(Engine::kPropagateColumnFamilyName); - storage_->FlushScripts(rocksdb::WriteOptions(), cf); + storage_->FlushScripts(storage_->DefaultWriteOptions(), cf); ScriptReset(); } diff --git a/src/storage.cc b/src/storage.cc index 6ae93238d92..73da1e02648 100644 --- a/src/storage.cc +++ b/src/storage.cc @@ -73,6 +73,7 @@ Storage::Storage(Config *config) SetCheckpointCreateTime(0); SetCheckpointAccessTime(0); backup_creating_time_ = std::time(nullptr); + SetWriteOptions(config->RocksDB.write_options); } Storage::~Storage() { @@ -94,6 +95,14 @@ void Storage::CloseDB() { db_ = nullptr; } +void Storage::SetWriteOptions(const Config::RocksDB::WriteOptions& config) { + write_opts_.sync = config.sync; + write_opts_.disableWAL = config.disable_WAL; + write_opts_.no_slowdown = config.no_slowdown; + write_opts_.low_pri = config.low_pri; + write_opts_.memtable_insert_hint_per_batch = config.memtable_insert_hint_per_batch; +} + rocksdb::BlockBasedTableOptions Storage::InitTableOptions() { rocksdb::BlockBasedTableOptions table_options; table_options.format_version = 5; @@ -533,7 +542,7 @@ rocksdb::Status Storage::DeleteRange(const std::string &first_key, const std::st if (!s.ok()) { return s; } - return Write(rocksdb::WriteOptions(), &batch); + return Write(write_opts_, &batch); } rocksdb::Status Storage::FlushScripts(const rocksdb::WriteOptions &options, rocksdb::ColumnFamilyHandle *cf_handle) { @@ -547,7 +556,7 @@ rocksdb::Status Storage::FlushScripts(const rocksdb::WriteOptions &options, rock if (!s.ok()) { return s; } - return Write(rocksdb::WriteOptions(), &batch); + return Write(options, &batch); } Status Storage::ReplicaApplyWriteBatch(std::string &&raw_batch) { @@ -555,7 +564,7 @@ Status Storage::ReplicaApplyWriteBatch(std::string &&raw_batch) { return Status(Status::NotOK, "reach space limit"); } auto bat = rocksdb::WriteBatch(std::move(raw_batch)); - auto s = db_->Write(rocksdb::WriteOptions(), &bat); + auto s = db_->Write(write_opts_, &bat); if (!s.ok()) { return Status(Status::NotOK, s.ToString()); } @@ -647,7 +656,7 @@ Status Storage::WriteToPropagateCF(const std::string &key, const std::string &va auto cf = GetCFHandle(kPropagateColumnFamilyName); batch.Put(cf, key, value); - auto s = Write(rocksdb::WriteOptions(), &batch); + auto s = Write(write_opts_, &batch); if (!s.ok()) { return Status(Status::NotOK, s.ToString()); } diff --git a/src/storage.h b/src/storage.h index 848ddd2be08..1ba5eee1f56 100644 --- a/src/storage.h +++ b/src/storage.h @@ -65,6 +65,7 @@ class Storage { explicit Storage(Config *config); ~Storage(); + void SetWriteOptions(const Config::RocksDB::WriteOptions& config); Status Open(bool read_only); Status Open(); Status OpenForReadOnly(); @@ -86,6 +87,7 @@ class Storage { Status ReplicaApplyWriteBatch(std::string &&raw_batch); rocksdb::SequenceNumber LatestSeq(); rocksdb::Status Write(const rocksdb::WriteOptions& options, rocksdb::WriteBatch* updates); + const rocksdb::WriteOptions& DefaultWriteOptions() { return write_opts_; } rocksdb::Status Delete(const rocksdb::WriteOptions &options, rocksdb::ColumnFamilyHandle *cf_handle, const rocksdb::Slice &key); @@ -187,6 +189,8 @@ class Storage { bool db_closing_ = true; std::atomic db_in_retryable_io_error_{false}; + + rocksdb::WriteOptions write_opts_ = rocksdb::WriteOptions(); }; } // namespace Engine From 04f25c878f8ae80265f1071b674ded0d008e6715 Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 21 Sep 2022 10:44:46 +0800 Subject: [PATCH 03/22] test: harden quit_test and list_test (#899) Signed-off-by: tison --- tests/gocase/unit/quit/quit_test.go | 2 +- tests/gocase/unit/type/list/list_test.go | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/gocase/unit/quit/quit_test.go b/tests/gocase/unit/quit/quit_test.go index c2cddf893a1..fa2aa828aa2 100644 --- a/tests/gocase/unit/quit/quit_test.go +++ b/tests/gocase/unit/quit/quit_test.go @@ -45,7 +45,7 @@ func TestPipeQuit(t *testing.T) { require.NoError(t, c.WriteArgs("PING")) r, err = c.ReadLine() - require.EqualError(t, err, "EOF") + require.Error(t, err) require.Empty(t, r) }) diff --git a/tests/gocase/unit/type/list/list_test.go b/tests/gocase/unit/type/list/list_test.go index 26285e3e71a..579bfabe1c4 100644 --- a/tests/gocase/unit/type/list/list_test.go +++ b/tests/gocase/unit/type/list/list_test.go @@ -26,6 +26,7 @@ import ( "strconv" "strings" "testing" + "time" "github.com/apache/incubator-kvrocks/tests/gocase/util" "github.com/go-redis/redis/v9" @@ -93,7 +94,7 @@ func TestZipList(t *testing.T) { defer srv.Close() ctx := context.Background() rdb := srv.NewClientWithOption(&redis.Options{ - MaxRetries: -1, + ReadTimeout: 10 * time.Second, }) defer func() { require.NoError(t, rdb.Close()) }() From 87644ab5672fdcb8fe7d27e811ab115a88eb897f Mon Sep 17 00:00:00 2001 From: xiaobiaozhao <52393536+xiaobiaozhao@users.noreply.github.com> Date: Wed, 21 Sep 2022 10:59:14 +0800 Subject: [PATCH 04/22] Add EVAL_RO (#782) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * โœจ feat: update lua vm * โœจ feat: update eval_ro * ๐Ÿงช test: add test case * ๐Ÿž fix: fix not set connection * ๐Ÿงช test: add *sha test case * ๐ŸŒˆ style: rm ; * ๐Ÿงช test: command 180 -> 182 * ๐Ÿž fix: fix exclusive issue * Update command_test.go Co-authored-by: tison Co-authored-by: hulk --- src/redis_cmd.cc | 26 ++++++++++++++++- src/redis_connection.cc | 20 +++++++++---- src/scripting.cc | 34 ++++++++++++++++++----- src/scripting.h | 12 ++++---- src/worker.cc | 5 +++- src/worker.h | 4 +++ tests/gocase/unit/command/command_test.go | 7 ----- tests/tcl/tests/unit/scripting.tcl | 22 +++++++++++++++ 8 files changed, 103 insertions(+), 27 deletions(-) diff --git a/src/redis_cmd.cc b/src/redis_cmd.cc index 9ece7bf95ea..553c77f88b2 100644 --- a/src/redis_cmd.cc +++ b/src/redis_cmd.cc @@ -4875,6 +4875,28 @@ class CommandEvalSHA : public Commander { } }; +class CommandEvalRO : public Commander { + public: + Status Execute(Server *svr, Connection *conn, std::string *output) override { + return Lua::evalGenericCommand(conn, args_, false, output, true); + } +}; + +class CommandEvalSHARO : public Commander { + public: + Status Parse(const std::vector &args) override { + if (args[1].size() != 40) { + return Status(Status::NotOK, + "NOSCRIPT No matching script. Please use EVAL"); + } + return Status::OK(); + } + + Status Execute(Server *svr, Connection *conn, std::string *output) override { + return Lua::evalGenericCommand(conn, args_, true, output, true); + } +}; + class CommandScript : public Commander { public: Status Parse(const std::vector &args) override { @@ -4904,7 +4926,7 @@ class CommandScript : public Commander { } } else if (args_.size() == 3 && subcommand_ == "load") { std::string sha; - auto s = Lua::createFunction(svr, args_[2], &sha); + auto s = Lua::createFunction(svr, args_[2], &sha, svr->Lua()); if (!s.IsOK()) { return s; } @@ -5962,6 +5984,8 @@ CommandAttributes redisCommandTable[] = { ADD_CMD("eval", -3, "exclusive write no-script", 0, 0, 0, CommandEval), ADD_CMD("evalsha", -3, "exclusive write no-script", 0, 0, 0, CommandEvalSHA), + ADD_CMD("eval_ro", -3, "read-only no-script", 0, 0, 0, CommandEvalRO), + ADD_CMD("evalsha_ro", -3, "read-only no-script", 0, 0, 0, CommandEvalSHARO), ADD_CMD("script", -2, "exclusive no-script", 0, 0, 0, CommandScript), ADD_CMD("compact", 1, "read-only no-script", 0, 0, 0, CommandCompact), diff --git a/src/redis_connection.cc b/src/redis_connection.cc index db37b28e50c..bf686b2a127 100644 --- a/src/redis_connection.cc +++ b/src/redis_connection.cc @@ -347,19 +347,29 @@ void Connection::ExecuteCommands(std::deque *to_process_cmds) { if (IsFlagEnabled(Connection::kMultiExec) && attributes->name != "exec") { // No lock guard, because 'exec' command has acquired 'WorkExclusivityGuard' } else if (attributes->is_exclusive() || - (cmd_name == "config" && cmd_tokens.size() == 2 && !strcasecmp(cmd_tokens[1].c_str(), "set")) || - (config->cluster_enabled && (cmd_name == "clusterx" || cmd_name == "cluster") - && cmd_tokens.size() >= 2 && Cluster::SubCommandIsExecExclusive(cmd_tokens[1]))) { + (cmd_name == "config" && cmd_tokens.size() == 2 && + !strcasecmp(cmd_tokens[1].c_str(), "set")) || + (config->cluster_enabled && + (cmd_name == "clusterx" || cmd_name == "cluster") && + cmd_tokens.size() >= 2 && + Cluster::SubCommandIsExecExclusive(cmd_tokens[1]))) { exclusivity = svr_->WorkExclusivityGuard(); // When executing lua script commands that have "exclusive" attribute, - // we need to know current connection, but we should set current connection - // after acquiring the WorkExclusivityGuard to make it thread-safe + // we need to know current connection, but we should set current + // connection after acquiring the WorkExclusivityGuard to make it + // thread-safe svr_->SetCurrentConnection(this); } else { concurrency = svr_->WorkConcurrencyGuard(); } + if (cmd_name == "eval_ro" || cmd_name == "evalsha_ro") { + // if executing read only lua script commands, set current + // connection. + svr_->SetCurrentConnection(this); + } + if (svr_->IsLoading() && attributes->is_ok_loading() == false) { Reply(Redis::Error("LOADING kvrocks is restoring the db from backup")); if (IsFlagEnabled(Connection::kMultiExec)) multi_error_ = true; diff --git a/src/scripting.cc b/src/scripting.cc index 19d1d04aa64..652f6062ecf 100644 --- a/src/scripting.cc +++ b/src/scripting.cc @@ -75,11 +75,11 @@ enum { }; namespace Lua { - lua_State* CreateState() { + lua_State *CreateState(bool read_only) { lua_State *lua = lua_open(); loadLibraries(lua); removeUnsupportedFunctions(lua); - loadFuncs(lua); + loadFuncs(lua, read_only); enableGlobalsProtection(lua); return lua; } @@ -89,7 +89,7 @@ namespace Lua { lua_close(lua); } - void loadFuncs(lua_State *lua) { + void loadFuncs(lua_State *lua, bool read_only) { lua_newtable(lua); /* redis.call */ @@ -136,6 +136,13 @@ namespace Lua { lua_pushcfunction(lua, redisStatusReplyCommand); lua_settable(lua, -3); + if (read_only) { + /* redis.read_only */ + lua_pushstring(lua, "read_only"); + lua_pushboolean(lua, 1); + lua_settable(lua, -3); + } + lua_setglobal(lua, "redis"); /* Replace math.random and math.randomseed with our implementations. */ @@ -233,11 +240,16 @@ namespace Lua { Status evalGenericCommand(Redis::Connection *conn, const std::vector &args, bool evalsha, - std::string *output) { + std::string *output, + bool read_only) { int64_t numkeys = 0; char funcname[43]; Server *srv = conn->GetServer(); lua_State *lua = srv->Lua(); + if (read_only) { + // Use the worker's private Lua VM when entering the read-only mode + lua = conn->Owner()->Lua(); + } auto s = Util::DecimalStringToNum(args[2], &numkeys); if (!s.IsOK()) { @@ -281,7 +293,7 @@ namespace Lua { body = args[1]; } std::string sha; - s = createFunction(srv, body, &sha); + s = createFunction(srv, body, &sha, lua); if (!s.IsOK()) { lua_pop(lua, 1); /* remove the error handler from the stack. */ return s; @@ -333,6 +345,10 @@ namespace Lua { int redisGenericCommand(lua_State *lua, int raise_error) { int j, argc = lua_gettop(lua); std::vector args; + lua_getglobal(lua, "redis"); + lua_getfield(lua, -1, "read_only"); + int read_only = lua_toboolean(lua, -1); + lua_pop(lua, 2); if (argc == 0) { pushError(lua, "Please specify at least one argument for redis.call()"); @@ -364,6 +380,10 @@ namespace Lua { return raise_error ? raiseError(lua) : 1; } auto redisCmd = cmd_iter->second; + if (read_only && redisCmd->is_write()) { + pushError(lua, "Write commands are not allowed from read-only scripts"); + return raise_error ? raiseError(lua) : 1; + } auto cmd = redisCmd->factory(); cmd->SetAttributes(redisCmd); cmd->SetArgs(args); @@ -883,7 +903,8 @@ int redisMathRandomSeed(lua_State *L) { * * If 'c' is not NULL, on error the client is informed with an appropriate * error describing the nature of the problem and the Lua interpreter error. */ -Status createFunction(Server *srv, const std::string &body, std::string *sha) { +Status createFunction(Server *srv, const std::string &body, std::string *sha, + lua_State *lua) { char funcname[43]; funcname[0] = 'f'; @@ -898,7 +919,6 @@ Status createFunction(Server *srv, const std::string &body, std::string *sha) { funcdef += body; funcdef += "\nend"; - lua_State *lua = srv->Lua(); if (luaL_loadbuffer(lua, funcdef.c_str(), funcdef.size(), "@user_script")) { std::string errMsg = lua_tostring(lua, -1); lua_pop(lua, 1); diff --git a/src/scripting.h b/src/scripting.h index eee1f02473b..459662de9f2 100644 --- a/src/scripting.h +++ b/src/scripting.h @@ -29,10 +29,10 @@ namespace Lua { -lua_State* CreateState(); +lua_State* CreateState(bool read_only = false); void DestroyState(lua_State *lua); -void loadFuncs(lua_State *lua); +void loadFuncs(lua_State *lua, bool read_only = false); void loadLibraries(lua_State *lua); void removeUnsupportedFunctions(lua_State *lua); void enableGlobalsProtection(lua_State *lua); @@ -42,13 +42,13 @@ int redisGenericCommand(lua_State *lua, int raise_error); int redisSha1hexCommand(lua_State *lua); int redisStatusReplyCommand(lua_State *lua); int redisErrorReplyCommand(lua_State *lua); -Status createFunction(Server *srv, const std::string &body, std::string *sha); +Status createFunction(Server *srv, const std::string &body, std::string *sha, + lua_State *lua); int redisLogCommand(lua_State *lua); Status evalGenericCommand(Redis::Connection *conn, - const std::vector &args, - bool evalsha, - std::string *output); + const std::vector &args, bool evalsha, + std::string *output, bool read_only = false); const char *redisProtocolToLuaType(lua_State *lua, const char *reply); const char *redisProtocolToLuaType_Int(lua_State *lua, const char *reply); diff --git a/src/worker.cc b/src/worker.cc index 7b9d9a40680..5657e9e24f4 100644 --- a/src/worker.cc +++ b/src/worker.cc @@ -38,8 +38,9 @@ #include #endif -#include "redis_request.h" #include "redis_connection.h" +#include "redis_request.h" +#include "scripting.h" #include "server.h" #include "util.h" @@ -65,6 +66,7 @@ Worker::Worker(Server *svr, Config *config, bool repl) : svr_(svr) { LOG(INFO) << "[worker] Listening on: " << bind << ":" << *port; } } + lua_ = Lua::CreateState(true); } Worker::~Worker() { @@ -86,6 +88,7 @@ Worker::~Worker() { ev_token_bucket_cfg_free(rate_limit_group_cfg_); } event_base_free(base_); + Lua::DestroyState(lua_); } void Worker::TimerCB(int, int16_t events, void *ctx) { diff --git a/src/worker.h b/src/worker.h index 400fa1b43ab..81c74bbafdf 100644 --- a/src/worker.h +++ b/src/worker.h @@ -33,6 +33,8 @@ #include #include #include "storage.h" + +#include "lua.hpp" #include "redis_connection.h" class Server; @@ -62,6 +64,7 @@ class Worker { Status ListenUnixSocket(const std::string &path, int perm, int backlog); + lua_State *Lua() { return lua_; } Server *svr_; private: @@ -85,6 +88,7 @@ class Worker { struct bufferevent_rate_limit_group *rate_limit_group_ = nullptr; struct ev_token_bucket_cfg *rate_limit_group_cfg_ = nullptr; + lua_State* lua_; }; class WorkerThread { diff --git a/tests/gocase/unit/command/command_test.go b/tests/gocase/unit/command/command_test.go index b5a87fc92b6..a1eb556c9f8 100644 --- a/tests/gocase/unit/command/command_test.go +++ b/tests/gocase/unit/command/command_test.go @@ -35,13 +35,6 @@ func TestCommand(t *testing.T) { rdb := srv.NewClient() defer func() { require.NoError(t, rdb.Close()) }() - t.Run("Kvrocks supports 182 commands currently", func(t *testing.T) { - r := rdb.Do(ctx, "COMMAND", "COUNT") - v, err := r.Int() - require.NoError(t, err) - require.Equal(t, 182, v) - }) - t.Run("acquire GET command info by COMMAND INFO", func(t *testing.T) { r := rdb.Do(ctx, "COMMAND", "INFO", "GET") vs, err := r.Slice() diff --git a/tests/tcl/tests/unit/scripting.tcl b/tests/tcl/tests/unit/scripting.tcl index 4138faf0790..41268be29bb 100644 --- a/tests/tcl/tests/unit/scripting.tcl +++ b/tests/tcl/tests/unit/scripting.tcl @@ -387,6 +387,28 @@ start_server {tags {"scripting"}} { set v [r eval { return redis.log(redis.LOG_WARNING, 'warning level'); } 0] assert_equal "" $v } {} + + test {EVAL_RO - Successful case} { + r set foo bar + assert_equal bar [r eval_ro {return redis.call('get', KEYS[1]);} 1 foo] + } + + test {EVALSHA_RO - Successful case} { + r set foo bar + assert_equal bar [r evalsha_ro 796941151549c416aa77522fb347487236c05e46 1 foo] + } + + test {EVAL_RO - Cannot run write commands} { + r set foo bar + catch {r eval_ro {redis.call('del', KEYS[1]);} 1 foo} e + set e + } {ERR * Write commands are not allowed from read-only scripts} + + test {EVALSHA_RO - Cannot run write commands} { + r set foo bar + catch {r evalsha_ro a1e63e1cd1bd1d5413851949332cfb9da4ee6dc0 1 foo} e + set e + } {ERR * Write commands are not allowed from read-only scripts} } start_server {tags {"repl"}} { From e8721a270d23345f385af6db0617c3549d6e6812 Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 21 Sep 2022 14:19:34 +0800 Subject: [PATCH 05/22] Move TCL test unit/scripting to Go case (#900) --- tests/gocase/unit/info/info_test.go | 34 +- tests/gocase/unit/scripting/scripting_test.go | 505 ++++++++++++++++++ tests/gocase/util/conditions.go | 55 ++ tests/gocase/util/constants.go | 1 + tests/gocase/util/server.go | 10 +- tests/tcl/tests/test_helper.tcl | 1 - tests/tcl/tests/unit/scripting.tcl | 436 --------------- 7 files changed, 582 insertions(+), 460 deletions(-) create mode 100644 tests/gocase/unit/scripting/scripting_test.go create mode 100644 tests/gocase/util/conditions.go delete mode 100644 tests/tcl/tests/unit/scripting.tcl diff --git a/tests/gocase/unit/info/info_test.go b/tests/gocase/unit/info/info_test.go index 1fde8cffead..8aab00053b4 100644 --- a/tests/gocase/unit/info/info_test.go +++ b/tests/gocase/unit/info/info_test.go @@ -22,9 +22,7 @@ package command import ( "context" "fmt" - "regexp" "strconv" - "strings" "testing" "time" @@ -40,14 +38,6 @@ func TestInfo(t *testing.T) { rdb := srv.NewClient() defer func() { require.NoError(t, rdb.Close()) }() - FindInfoEntry := func(t *testing.T, section string, key string) string { - r := rdb.Info(ctx, section) - p := regexp.MustCompile(fmt.Sprintf("%s:(.+)", key)) - ms := p.FindStringSubmatch(r.Val()) - require.Len(t, ms, 2) - return strings.TrimSpace(ms[1]) - } - MustAtoi := func(t *testing.T, s string) int { i, err := strconv.Atoi(s) require.NoError(t, err) @@ -65,21 +55,21 @@ func TestInfo(t *testing.T) { time.Sleep(time.Second) } - r := FindInfoEntry(t, "rocksdb", "put_per_sec") + r := util.FindInfoEntry(t, ctx, rdb, "put_per_sec", "rocksdb") require.Greater(t, MustAtoi(t, r), 0) - r = FindInfoEntry(t, "rocksdb", "get_per_sec") + r = util.FindInfoEntry(t, ctx, rdb, "get_per_sec", "rocksdb") require.Greater(t, MustAtoi(t, r), 0) - r = FindInfoEntry(t, "rocksdb", "seek_per_sec") + r = util.FindInfoEntry(t, ctx, rdb, "seek_per_sec", "rocksdb") require.Greater(t, MustAtoi(t, r), 0) - r = FindInfoEntry(t, "rocksdb", "next_per_sec") + r = util.FindInfoEntry(t, ctx, rdb, "next_per_sec", "rocksdb") require.Greater(t, MustAtoi(t, r), 0) }) t.Run("get bgsave information by INFO", func(t *testing.T) { - require.Equal(t, "0", FindInfoEntry(t, "persistence", "bgsave_in_progress")) - require.Equal(t, "-1", FindInfoEntry(t, "persistence", "last_bgsave_time")) - require.Equal(t, "ok", FindInfoEntry(t, "persistence", "last_bgsave_status")) - require.Equal(t, "-1", FindInfoEntry(t, "persistence", "last_bgsave_time_sec")) + require.Equal(t, "0", util.FindInfoEntry(t, ctx, rdb, "bgsave_in_progress", "persistence")) + require.Equal(t, "-1", util.FindInfoEntry(t, ctx, rdb, "last_bgsave_time", "persistence")) + require.Equal(t, "ok", util.FindInfoEntry(t, ctx, rdb, "last_bgsave_status", "persistence")) + require.Equal(t, "-1", util.FindInfoEntry(t, ctx, rdb, "last_bgsave_time_sec", "persistence")) r := rdb.Do(ctx, "bgsave") v, err := r.Text() @@ -87,14 +77,14 @@ func TestInfo(t *testing.T) { require.Equal(t, "OK", v) require.Eventually(t, func() bool { - e := MustAtoi(t, FindInfoEntry(t, "persistence", "bgsave_in_progress")) + e := MustAtoi(t, util.FindInfoEntry(t, ctx, rdb, "bgsave_in_progress", "persistence")) return e == 0 }, 5*time.Second, 100*time.Millisecond) - lastBgsaveTime := MustAtoi(t, FindInfoEntry(t, "persistence", "last_bgsave_time")) + lastBgsaveTime := MustAtoi(t, util.FindInfoEntry(t, ctx, rdb, "last_bgsave_time", "persistence")) require.Greater(t, lastBgsaveTime, 1640507660) - require.Equal(t, "ok", FindInfoEntry(t, "persistence", "last_bgsave_status")) - lastBgsaveTimeSec := MustAtoi(t, FindInfoEntry(t, "persistence", "last_bgsave_time_sec")) + require.Equal(t, "ok", util.FindInfoEntry(t, ctx, rdb, "last_bgsave_status", "persistence")) + lastBgsaveTimeSec := MustAtoi(t, util.FindInfoEntry(t, ctx, rdb, "last_bgsave_time_sec", "persistence")) require.GreaterOrEqual(t, lastBgsaveTimeSec, 0) require.Less(t, lastBgsaveTimeSec, 3) }) diff --git a/tests/gocase/unit/scripting/scripting_test.go b/tests/gocase/unit/scripting/scripting_test.go new file mode 100644 index 00000000000..c5e7abad241 --- /dev/null +++ b/tests/gocase/unit/scripting/scripting_test.go @@ -0,0 +1,505 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package scripting + +import ( + "context" + "fmt" + "testing" + + "github.com/apache/incubator-kvrocks/tests/gocase/util" + "github.com/stretchr/testify/require" +) + +func TestScripting(t *testing.T) { + srv := util.StartServer(t, map[string]string{}) + defer srv.Close() + + ctx := context.Background() + rdb := srv.NewClient() + defer func() { require.NoError(t, rdb.Close()) }() + + t.Run("EVAL - Does Lua interpreter replies to our requests?", func(t *testing.T) { + r := rdb.Eval(ctx, `return 'hello'`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, "hello", r.Val()) + }) + + t.Run("EVAL - Lua integer -> Redis protocol type conversion", func(t *testing.T) { + r := rdb.Eval(ctx, `return 100.5`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, int64(100), r.Val()) + }) + + t.Run("EVAL - Lua string -> Redis protocol type conversion", func(t *testing.T) { + r := rdb.Eval(ctx, `return 'hello world'`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, "hello world", r.Val()) + }) + + t.Run("EVAL - Lua true boolean -> Redis protocol type conversion", func(t *testing.T) { + r := rdb.Eval(ctx, `return true`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, int64(1), r.Val()) + }) + + t.Run("EVAL - Lua false boolean -> Redis protocol type conversion", func(t *testing.T) { + r := rdb.Eval(ctx, `return false`, []string{}) + require.EqualError(t, r.Err(), util.ErrRedisNil) + require.Nil(t, r.Val()) + }) + + t.Run("EVAL - Lua status code reply -> Redis protocol type conversion", func(t *testing.T) { + r := rdb.Eval(ctx, `return {ok='fine'}`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, "fine", r.Val()) + }) + + t.Run("EVAL - Lua error reply -> Redis protocol type conversion", func(t *testing.T) { + r := rdb.Eval(ctx, `return {err='this is an error'}`, []string{}) + require.EqualError(t, r.Err(), "this is an error") + require.Nil(t, r.Val()) + }) + + t.Run("Script return recursive object", func(t *testing.T) { + c := srv.NewTCPClient() + defer func() { require.NoError(t, c.Close()) }() + require.NoError(t, c.WriteArgs("EVAL", `return "hello"`, "0")) + r, err := c.ReadLine() + require.NoError(t, err) + require.Equal(t, "$5", r) + r, err = c.ReadLine() + require.NoError(t, err) + require.Equal(t, "hello", r) + }) + + t.Run("EVAL - Lua table -> Redis protocol type conversion", func(t *testing.T) { + r := rdb.Eval(ctx, `return {1,2,3,'ciao',{1,2}}`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, "[1 2 3 ciao [1 2]]", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("EVAL - Are the KEYS and ARGV arrays populated correctly?", func(t *testing.T) { + r := rdb.Eval(ctx, `return {KEYS[1],KEYS[2],ARGV[1],ARGV[2]}`, []string{"a", "b"}, "c", "d") + require.NoError(t, r.Err()) + require.Equal(t, "[a b c d]", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("EVAL - is Lua able to call Redis API?", func(t *testing.T) { + require.NoError(t, rdb.Set(ctx, "mykey", "myval", 0).Err()) + r := rdb.Eval(ctx, `return redis.call('get',KEYS[1])`, []string{"mykey"}) + require.NoError(t, r.Err()) + require.Equal(t, "myval", r.Val()) + }) + + t.Run("EVALSHA - Can we call a SHA1 if already defined?", func(t *testing.T) { + r := rdb.EvalSha(ctx, "fd758d1589d044dd850a6f05d52f2eefd27f033f", []string{"mykey"}) + require.NoError(t, r.Err()) + require.Equal(t, "myval", r.Val()) + }) + + t.Run("EVALSHA - Can we call a SHA1 in uppercase?", func(t *testing.T) { + r := rdb.EvalSha(ctx, "FD758D1589D044DD850A6F05D52F2EEFD27F033F", []string{"mykey"}) + require.NoError(t, r.Err()) + require.Equal(t, "myval", r.Val()) + }) + + t.Run("EVALSHA - Do we get an error on invalid SHA1?", func(t *testing.T) { + r := rdb.EvalSha(ctx, "NotValidShaSUM", []string{}) + util.ErrorRegexp(t, r.Err(), "ERR NOSCRIPT.*") + require.Nil(t, r.Val()) + }) + + t.Run("EVALSHA - Do we get an error on non defined SHA1?", func(t *testing.T) { + r := rdb.EvalSha(ctx, "ffd632c7d33e571e9f24556ebed26c3479a87130", []string{}) + util.ErrorRegexp(t, r.Err(), "ERR NOSCRIPT.*") + require.Nil(t, r.Val()) + }) + + t.Run("EVAL - Redis integer -> Lua type conversion", func(t *testing.T) { + require.NoError(t, rdb.Set(ctx, "x", 0, 0).Err()) + r := rdb.Eval(ctx, ` +local foo = redis.pcall('incr',KEYS[1]) +return {type(foo),foo} +`, []string{"x"}) + require.NoError(t, r.Err()) + require.Equal(t, "[number 1]", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("EVAL - Redis bulk -> Lua type conversion", func(t *testing.T) { + require.NoError(t, rdb.Set(ctx, "mykey", "myval", 0).Err()) + r := rdb.Eval(ctx, ` +local foo = redis.pcall('get',KEYS[1]) +return {type(foo),foo} +`, []string{"mykey"}) + require.NoError(t, r.Err()) + require.Equal(t, "[string myval]", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("EVAL - Redis multi bulk -> Lua type conversion", func(t *testing.T) { + require.NoError(t, rdb.Del(ctx, "mylist").Err()) + require.NoError(t, rdb.RPush(ctx, "mylist", "a", "b", "c").Err()) + r := rdb.Eval(ctx, ` +local foo = redis.pcall('lrange',KEYS[1],0,-1) +return {type(foo),foo[1],foo[2],foo[3],# foo} +`, []string{"mylist"}) + require.NoError(t, r.Err()) + require.Equal(t, "[table a b c 3]", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("EVAL - Redis status reply -> Lua type conversion", func(t *testing.T) { + r := rdb.Eval(ctx, ` +local foo = redis.pcall('set',KEYS[1],'myval') +return {type(foo),foo['ok']} +`, []string{"mykey"}) + require.NoError(t, r.Err()) + require.Equal(t, "[table OK]", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("EVAL - Redis error reply -> Lua type conversion", func(t *testing.T) { + require.NoError(t, rdb.Set(ctx, "mykey", "myval", 0).Err()) + r := rdb.Eval(ctx, ` +local foo = redis.pcall('incr',KEYS[1]) +return {type(foo),foo['err']} +`, []string{"mykey"}) + require.NoError(t, r.Err()) + require.Equal(t, "[table Invalid argument: value is not an integer or out of range]", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("EVAL - Redis nil bulk reply -> Lua type conversion", func(t *testing.T) { + require.NoError(t, rdb.Del(ctx, "mykey").Err()) + r := rdb.Eval(ctx, ` +local foo = redis.pcall('get',KEYS[1]) +return {type(foo),foo == false} +`, []string{"mykey"}) + require.NoError(t, r.Err()) + require.Equal(t, "[boolean 1]", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("EVAL - Scripts can't run certain commands", func(t *testing.T) { + r := rdb.Eval(ctx, `return redis.pcall('blpop','x',0)`, []string{}) + require.ErrorContains(t, r.Err(), "not allowed") + }) + + t.Run("EVAL - Scripts can run certain commands", func(t *testing.T) { + r := rdb.Eval(ctx, `redis.pcall('randomkey'); return redis.pcall('set','x','ciao')`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, "OK", r.Val()) + }) + + t.Run("EVAL - No arguments to redis.call/pcall is considered an error", func(t *testing.T) { + r := rdb.Eval(ctx, `return redis.call()`, []string{}) + require.ErrorContains(t, r.Err(), "one argument") + }) + + t.Run("EVAL - redis.call variant raises a Lua error on Redis cmd error", func(t *testing.T) { + r := rdb.Eval(ctx, `redis.call('nosuchcommand')`, []string{}) + require.ErrorContains(t, r.Err(), "Unknown Redis") + r = rdb.Eval(ctx, `redis.call('get','a','b','c')`, []string{}) + require.ErrorContains(t, r.Err(), "number of args") + require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err()) + r = rdb.Eval(ctx, `redis.call('lpush',KEYS[1],'val')`, []string{"foo"}) + require.ErrorContains(t, r.Err(), "against a key") + }) + + t.Run("EVAL - JSON numeric decoding", func(t *testing.T) { + r := rdb.Eval(ctx, ` +return + table.concat( + cjson.decode( + "[0.0, -5e3, -1, 0.3e-3, 1023.2, 0e10]"), " ") +`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, "0 -5000 -1 0.0003 1023.2 0", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("EVAL - JSON string decoding", func(t *testing.T) { + r := rdb.Eval(ctx, ` +local decoded = cjson.decode('{"keya": "a", "keyb": "b"}') +return {decoded.keya, decoded.keyb} +`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, "[a b]", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("EVAL - cmsgpack can pack double?", func(t *testing.T) { + r := rdb.Eval(ctx, ` +local encoded = cmsgpack.pack(0.1) +local h = "" +for i = 1, #encoded do + h = h .. string.format("%02x",string.byte(encoded,i)) +end +return h +`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, "cb3fb999999999999a", r.Val()) + }) + + t.Run("EVAL - cmsgpack can pack negative int64?", func(t *testing.T) { + r := rdb.Eval(ctx, ` +local encoded = cmsgpack.pack(-1099511627776) +local h = "" +for i = 1, #encoded do + h = h .. string.format("%02x",string.byte(encoded,i)) +end +return h +`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, "d3ffffff0000000000", r.Val()) + }) + + t.Run("EVAL - cmsgpack can pack and unpack circular references?", func(t *testing.T) { + r := rdb.Eval(ctx, ` +local a = {x=nil,y=5} +local b = {x=a} +a['x'] = b +local encoded = cmsgpack.pack(a) +local h = "" +-- cmsgpack encodes to a depth of 16, but can't encode +-- references, so the encoded object has a deep copy recursive +-- depth of 16. +for i = 1, #encoded do + h = h .. string.format("%02x",string.byte(encoded,i)) +end +-- when unpacked, re.x.x != re because the unpack creates +-- individual tables down to a depth of 16. +-- (that's why the encoded output is so large) +local re = cmsgpack.unpack(encoded) +assert(re) +assert(re.x) +assert(re.x.x.y == re.y) +assert(re.x.x.x.x.y == re.y) +assert(re.x.x.x.x.x.x.y == re.y) +assert(re.x.x.x.x.x.x.x.x.x.x.y == re.y) +-- maximum working depth: +assert(re.x.x.x.x.x.x.x.x.x.x.x.x.x.x.y == re.y) +-- now the last x would be b above and has no y +assert(re.x.x.x.x.x.x.x.x.x.x.x.x.x.x.x) +-- so, the final x.x is at the depth limit and was assigned nil +assert(re.x.x.x.x.x.x.x.x.x.x.x.x.x.x.x.x == nil) +assert(h == "82a17881a17882a17881a17882a17881a17882a17881a17882a17881a17882a17881a17882a17881a17882a17881a178c0a17905a17905a17905a17905a17905a17905a17905a17905" or h == "82a17905a17881a17882a17905a17881a17882a17905a17881a17882a17905a17881a17882a17905a17881a17882a17905a17881a17882a17905a17881a17882a17905a17881a178c0") +return {re.x.x.x.x.x.x.x.x.y == re.y, re.y == 5} +`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, "[1 1]", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("EVAL - Numerical sanity check from bitop", func(t *testing.T) { + r := rdb.Eval(ctx, ` +assert(0x7fffffff == 2147483647, "broken hex literals"); +assert(0xffffffff == -1 or 0xffffffff == 2^32-1, + "broken hex literals"); +assert(tostring(-1) == "-1", "broken tostring()"); +assert(tostring(0xffffffff) == "-1" or + tostring(0xffffffff) == "4294967295", + "broken tostring()") +`, []string{}) + require.EqualError(t, r.Err(), util.ErrRedisNil) + require.Nil(t, r.Val()) + }) + + t.Run("EVAL - Verify minimal bitop functionality", func(t *testing.T) { + r := rdb.Eval(ctx, ` +assert(bit.tobit(1) == 1); +assert(bit.band(1) == 1); +assert(bit.bxor(1,2) == 3); +assert(bit.bor(1,2,4,8,16,32,64,128) == 255) +`, []string{}) + require.EqualError(t, r.Err(), util.ErrRedisNil) + require.Nil(t, r.Val()) + }) + + t.Run("EVAL - Able to parse trailing comments", func(t *testing.T) { + r := rdb.Eval(ctx, `return 'hello' --trailing comment`, []string{}) + require.NoError(t, r.Err()) + require.Equal(t, "hello", r.Val()) + }) + + t.Run("EVAL does not leak in the Lua stack", func(t *testing.T) { + require.NoError(t, rdb.Set(ctx, "x", 0, 0).Err()) + + // use a non-blocking client to speed up the loop. + c := srv.NewTCPClient() + defer func() { require.NoError(t, c.Close()) }() + + for i := 0; i < 10000; i++ { + require.NoError(t, c.WriteArgs("EVAL", `return redis.call("incr",KEYS[1])`, "1", "x")) + } + for i := 0; i < 10000; i++ { + _, err := c.ReadLine() + require.NoError(t, err) + } + + require.EqualValues(t, "10000", rdb.Get(ctx, "x").Val()) + }) + + t.Run("SCRIPTING FLUSH - is able to clear the scripts cache?", func(t *testing.T) { + require.NoError(t, rdb.Set(ctx, "mykey", "myval", 0).Err()) + r := rdb.EvalSha(ctx, "fd758d1589d044dd850a6f05d52f2eefd27f033f", []string{"mykey"}) + require.NoError(t, r.Err()) + require.Equal(t, "myval", r.Val()) + require.NoError(t, rdb.ScriptFlush(ctx).Err()) + r = rdb.EvalSha(ctx, "fd758d1589d044dd850a6f05d52f2eefd27f033f", []string{"mykey"}) + util.ErrorRegexp(t, r.Err(), "ERR NOSCRIPT.*") + }) + + t.Run("SCRIPT EXISTS - can detect already defined scripts?", func(t *testing.T) { + r1 := rdb.Eval(ctx, "return 1+1", []string{}) + require.NoError(t, r1.Err()) + require.Equal(t, int64(2), r1.Val()) + r2 := rdb.ScriptExists(ctx, "a27e7e8a43702b7046d4f6a7ccf5b60cef6b9bd9", "a27e7e8a43702b7046d4f6a7ccf5b60cef6b9bda") + require.NoError(t, r2.Err()) + require.Equal(t, []bool{true, false}, r2.Val()) + }) + + t.Run("SCRIPT LOAD - is able to register scripts in the scripting cache", func(t *testing.T) { + r1 := rdb.ScriptLoad(ctx, "return 'loaded'") + require.NoError(t, r1.Err()) + require.Equal(t, "b534286061d4b9e4026607613b95c06c06015ae8", r1.Val()) + r2 := rdb.EvalSha(ctx, "b534286061d4b9e4026607613b95c06c06015ae8", []string{}) + require.NoError(t, r2.Err()) + require.Equal(t, "loaded", r2.Val()) + }) + + t.Run("Globals protection reading an undeclared global variable", func(t *testing.T) { + r2 := rdb.Eval(ctx, `return a`, []string{}) + util.ErrorRegexp(t, r2.Err(), ".*ERR.*attempted to access .* global.*") + }) + + t.Run("Globals protection setting an undeclared global variable", func(t *testing.T) { + r2 := rdb.Eval(ctx, `a=10`, []string{}) + util.ErrorRegexp(t, r2.Err(), ".*ERR.*attempted to create global.*") + }) + + t.Run("Test an example script DECR_IF_GT", func(t *testing.T) { + scriptDecrIfGt := ` +local current + +current = redis.call('get',KEYS[1]) +if not current then return nil end +if current > ARGV[1] then + return redis.call('decr',KEYS[1]) +else + return redis.call('get',KEYS[1]) +end +` + require.NoError(t, rdb.Set(ctx, "foo", 5, 0).Err()) + r := rdb.Eval(ctx, scriptDecrIfGt, []string{"foo"}, 2) + require.NoError(t, r.Err()) + require.Equal(t, int64(4), r.Val()) + r = rdb.Eval(ctx, scriptDecrIfGt, []string{"foo"}, 2) + require.NoError(t, r.Err()) + require.Equal(t, int64(3), r.Val()) + r = rdb.Eval(ctx, scriptDecrIfGt, []string{"foo"}, 2) + require.NoError(t, r.Err()) + require.Equal(t, int64(2), r.Val()) + r = rdb.Eval(ctx, scriptDecrIfGt, []string{"foo"}, 2) + require.NoError(t, r.Err()) + require.Equal(t, "2", r.Val()) + r = rdb.Eval(ctx, scriptDecrIfGt, []string{"foo"}, 2) + require.NoError(t, r.Err()) + require.Equal(t, "2", r.Val()) + }) + + t.Run("Scripting engine PRNG can be seeded correctly", func(t *testing.T) { + rand1 := rdb.Eval(ctx, ` +math.randomseed(ARGV[1]); return tostring(math.random()) +`, []string{}, 10).Val() + rand2 := rdb.Eval(ctx, ` +math.randomseed(ARGV[1]); return tostring(math.random()) +`, []string{}, 10).Val() + rand3 := rdb.Eval(ctx, ` +math.randomseed(ARGV[1]); return tostring(math.random()) +`, []string{}, 20).Val() + require.Equal(t, rand1, rand2) + require.NotEqual(t, rand2, rand3) + }) + + t.Run("In the context of Lua the output of random commands gets ordered", func(t *testing.T) { + require.NoError(t, rdb.Del(ctx, "myset").Err()) + require.NoError(t, rdb.SAdd(ctx, "myset", "a", "b", "c", "d", "e", "f", "g", "h", "i", "l", "m", "n", "o", "p", "q", "r", "s", "t", "u", "v", "z", "aa", "aaa", "azz").Err()) + r := rdb.Eval(ctx, `return redis.call('smembers',KEYS[1])`, []string{"myset"}) + require.NoError(t, r.Err()) + require.Equal(t, "[a aa aaa azz b c d e f g h i l m n o p q r s t u v z]", fmt.Sprintf("%v", r.Val())) + }) + + t.Run("Make sure redis.log() works", func(t *testing.T) { + require.EqualError(t, rdb.Eval(ctx, `return redis.log(redis.LOG_DEBUG, 'debug level');`, []string{}).Err(), util.ErrRedisNil) + require.EqualError(t, rdb.Eval(ctx, `return redis.log(redis.LOG_VERBOSE, 'debug level');`, []string{}).Err(), util.ErrRedisNil) + require.EqualError(t, rdb.Eval(ctx, `return redis.log(redis.LOG_NOTICE, 'debug level');`, []string{}).Err(), util.ErrRedisNil) + require.EqualError(t, rdb.Eval(ctx, `return redis.log(redis.LOG_WARNING, 'debug level');`, []string{}).Err(), util.ErrRedisNil) + }) + + t.Run("EVAL_RO - successful case", func(t *testing.T) { + require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err()) + r := rdb.Do(ctx, "EVAL_RO", `return redis.call('get', KEYS[1]);`, "1", "foo") + require.NoError(t, r.Err()) + require.Equal(t, "bar", r.Val()) + }) + + t.Run("EVALSHA_RO - successful case", func(t *testing.T) { + require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err()) + r := rdb.Do(ctx, "EVALSHA_RO", "796941151549c416aa77522fb347487236c05e46", "1", "foo") + require.NoError(t, r.Err()) + require.Equal(t, "bar", r.Val()) + }) + + t.Run("EVAL_RO - cannot run write commands", func(t *testing.T) { + require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err()) + r := rdb.Do(ctx, "EVAL_RO", `redis.call('del', KEYS[1]);`, "1", "foo") + util.ErrorRegexp(t, r.Err(), "ERR .* Write commands are not allowed from read-only scripts") + }) + + t.Run("EVALSHA_RO - cannot run write commands", func(t *testing.T) { + require.NoError(t, rdb.Set(ctx, "foo", "bar", 0).Err()) + r := rdb.Do(ctx, "EVALSHA_RO", "a1e63e1cd1bd1d5413851949332cfb9da4ee6dc0", "1", "foo") + util.ErrorRegexp(t, r.Err(), "ERR .* Write commands are not allowed from read-only scripts") + }) +} + +func TestScriptingMasterSlave(t *testing.T) { + master := util.StartServer(t, map[string]string{}) + defer master.Close() + masterClient := master.NewClient() + defer func() { require.NoError(t, masterClient.Close()) }() + + slave := util.StartServer(t, map[string]string{}) + defer slave.Close() + slaveClient := slave.NewClient() + defer func() { require.NoError(t, slaveClient.Close()) }() + + ctx := context.Background() + + require.NoError(t, slaveClient.SlaveOf(ctx, master.Host(), fmt.Sprintf("%d", master.Port())).Err()) + util.WaitForSync(t, ctx, slaveClient) + + t.Run("SCRIPTING: script load on master, read on slave", func(t *testing.T) { + sha := masterClient.ScriptLoad(ctx, `return 'script loaded'`).Val() + require.Equal(t, "4167ea82ed9c381c7659f7cf93f394219147e8c4", sha) + util.WaitForOffsetSync(t, ctx, masterClient, slaveClient) + require.Equal(t, []bool{true}, masterClient.ScriptExists(ctx, sha).Val()) + require.Equal(t, []bool{true}, slaveClient.ScriptExists(ctx, sha).Val()) + + require.NoError(t, masterClient.ScriptFlush(ctx).Err()) + util.WaitForOffsetSync(t, ctx, masterClient, slaveClient) + require.Equal(t, []bool{false}, masterClient.ScriptExists(ctx, sha).Val()) + require.Equal(t, []bool{false}, slaveClient.ScriptExists(ctx, sha).Val()) + }) +} diff --git a/tests/gocase/util/conditions.go b/tests/gocase/util/conditions.go new file mode 100644 index 00000000000..d6ac7a8fc75 --- /dev/null +++ b/tests/gocase/util/conditions.go @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package util + +import ( + "context" + "fmt" + "regexp" + "strings" + "testing" + "time" + + "github.com/go-redis/redis/v9" + "github.com/stretchr/testify/require" +) + +func FindInfoEntry(t *testing.T, ctx context.Context, rdb *redis.Client, key string, section ...string) string { + r := rdb.Info(ctx, section...) + p := regexp.MustCompile(fmt.Sprintf("%s:(.+)", key)) + ms := p.FindStringSubmatch(r.Val()) + require.Len(t, ms, 2) + return strings.TrimSpace(ms[1]) +} + +func WaitForSync(t *testing.T, ctx context.Context, slave *redis.Client) { + require.Eventually(t, func() bool { + r := FindInfoEntry(t, ctx, slave, "master_link_status") + return r == "up" + }, 5*time.Second, 100*time.Millisecond) +} + +func WaitForOffsetSync(t *testing.T, ctx context.Context, master, slave *redis.Client) { + require.Eventually(t, func() bool { + o1 := FindInfoEntry(t, ctx, master, "master_repl_offset") + o2 := FindInfoEntry(t, ctx, slave, "master_repl_offset") + return o1 == o2 + }, 5*time.Second, 100*time.Millisecond) +} diff --git a/tests/gocase/util/constants.go b/tests/gocase/util/constants.go index 9c90e7ed9a6..cdb8bf429df 100644 --- a/tests/gocase/util/constants.go +++ b/tests/gocase/util/constants.go @@ -20,3 +20,4 @@ package util const DefaultDelta = 0.000001 +const ErrRedisNil = "redis: nil" diff --git a/tests/gocase/util/server.go b/tests/gocase/util/server.go index 136e60a9de7..10cca148c33 100644 --- a/tests/gocase/util/server.go +++ b/tests/gocase/util/server.go @@ -36,11 +36,19 @@ import ( type KvrocksServer struct { t testing.TB cmd *exec.Cmd - addr net.Addr + addr *net.TCPAddr clean func() } +func (s *KvrocksServer) Host() string { + return s.addr.AddrPort().Addr().String() +} + +func (s *KvrocksServer) Port() uint16 { + return s.addr.AddrPort().Port() +} + func (s *KvrocksServer) NewClient() *redis.Client { return s.NewClientWithOption(&redis.Options{Addr: s.addr.String()}) } diff --git a/tests/tcl/tests/test_helper.tcl b/tests/tcl/tests/test_helper.tcl index 24fc3ad549d..d16a59c696e 100644 --- a/tests/tcl/tests/test_helper.tcl +++ b/tests/tcl/tests/test_helper.tcl @@ -45,7 +45,6 @@ set ::all_tests { unit/pubsub unit/introspection unit/geo - unit/scripting integration/slotmigrate integration/slotimport integration/redis-cli diff --git a/tests/tcl/tests/unit/scripting.tcl b/tests/tcl/tests/unit/scripting.tcl deleted file mode 100644 index 41268be29bb..00000000000 --- a/tests/tcl/tests/unit/scripting.tcl +++ /dev/null @@ -1,436 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -# Copyright (c) 2006-2020, Salvatore Sanfilippo -# See bundled license file licenses/LICENSE.redis for details. - -# This file is copied and modified from the Redis project, -# which started out as: https://github.com/redis/redis/blob/dbcc0a8/tests/unit/scripting.tcl - -start_server {tags {"scripting"}} { - test {EVAL - Does Lua interpreter replies to our requests?} { - r eval {return 'hello'} 0 - } {hello} - test {EVAL - Lua integer -> Redis protocol type conversion} { - r eval {return 100.5} 0 - } {100} - - test {EVAL - Lua string -> Redis protocol type conversion} { - r eval {return 'hello world'} 0 - } {hello world} - - test {EVAL - Lua true boolean -> Redis protocol type conversion} { - r eval {return true} 0 - } {1} - - test {EVAL - Lua false boolean -> Redis protocol type conversion} { - r eval {return false} 0 - } {} - - test {EVAL - Lua status code reply -> Redis protocol type conversion} { - r eval {return {ok='fine'}} 0 - } {fine} - - test {EVAL - Lua error reply -> Redis protocol type conversion} { - catch { - r eval {return {err='this is an error'}} 0 - } e - set _ $e - } {this is an error} - - test {Script return recursive object} { - r readraw 1 - set bulk_len [r eval {return "hello"} 0] - set bulk [r read] - r readraw 0 - assert_equal $bulk_len {$5} - assert_equal $bulk hello - } {} - - test {EVAL - Lua table -> Redis protocol type conversion} { - r eval {return {1,2,3,'ciao',{1,2}}} 0 - } {1 2 3 ciao {1 2}} - - test {EVAL - Are the KEYS and ARGV arrays populated correctly?} { - r eval {return {KEYS[1],KEYS[2],ARGV[1],ARGV[2]}} 2 a b c d - } {a b c d} - - test {EVAL - is Lua able to call Redis API?} { - r set mykey myval - r eval {return redis.call('get',KEYS[1])} 1 mykey - } {myval} - - test {EVALSHA - Can we call a SHA1 if already defined?} { - r evalsha fd758d1589d044dd850a6f05d52f2eefd27f033f 1 mykey - } {myval} - - test {EVALSHA - Can we call a SHA1 in uppercase?} { - r evalsha FD758D1589D044DD850A6F05D52F2EEFD27F033F 1 mykey - } {myval} - - test {EVALSHA - Do we get an error on invalid SHA1?} { - catch {r evalsha NotValidShaSUM 0} e - set _ $e - } {ERR NOSCRIPT*} - - test {EVALSHA - Do we get an error on non defined SHA1?} { - catch {r evalsha ffd632c7d33e571e9f24556ebed26c3479a87130 0} e - set _ $e - } {ERR NOSCRIPT*} - - test {EVAL - Redis integer -> Lua type conversion} { - r set x 0 - r eval { - local foo = redis.pcall('incr',KEYS[1]) - return {type(foo),foo} - } 1 x - } {number 1} - - test {EVAL - Redis bulk -> Lua type conversion} { - r set mykey myval - r eval { - local foo = redis.pcall('get',KEYS[1]) - return {type(foo),foo} - } 1 mykey - } {string myval} - - test {EVAL - Redis integer -> Lua type conversion} { - r set x 0 - r eval { - local foo = redis.pcall('incr',KEYS[1]) - return {type(foo),foo} - } 1 x - } {number 1} - - test {EVAL - Redis bulk -> Lua type conversion} { - r set mykey myval - r eval { - local foo = redis.pcall('get',KEYS[1]) - return {type(foo),foo} - } 1 mykey - } {string myval} - test {EVAL - Redis multi bulk -> Lua type conversion} { - r del mylist - r rpush mylist a - r rpush mylist b - r rpush mylist c - r eval { - local foo = redis.pcall('lrange',KEYS[1],0,-1) - return {type(foo),foo[1],foo[2],foo[3],# foo} - } 1 mylist - } {table a b c 3} - - test {EVAL - Redis status reply -> Lua type conversion} { - r eval { - local foo = redis.pcall('set',KEYS[1],'myval') - return {type(foo),foo['ok']} - } 1 mykey - } {table OK} - - test {EVAL - Redis error reply -> Lua type conversion} { - r set mykey myval - r eval { - local foo = redis.pcall('incr',KEYS[1]) - return {type(foo),foo['err']} - } 1 mykey - } {table {Invalid argument: value is not an integer or out of range}} - test {EVAL - Redis nil bulk reply -> Lua type conversion} { - r del mykey - r eval { - local foo = redis.pcall('get',KEYS[1]) - return {type(foo),foo == false} - } 1 mykey - } {boolean 1} - test {EVAL - Scripts can't run certain commands} { - set e {} - catch {r eval {return redis.pcall('blpop','x',0)} 0} e - set e - } {*not allowed*} - - test {EVAL - Scripts can't run certain commands} { - set e {} - catch { - r eval "redis.pcall('randomkey'); return redis.pcall('set','x','ciao')" 0 - } e - set e - } {OK} - - test {EVAL - No arguments to redis.call/pcall is considered an error} { - set e {} - catch {r eval {return redis.call()} 0} e - set e - } {*one argument*} - - test {EVAL - redis.call variant raises a Lua error on Redis cmd error (1)} { - set e {} - catch { - r eval "redis.call('nosuchcommand')" 0 - } e - set e - } {*Unknown Redis*} - test {EVAL - redis.call variant raises a Lua error on Redis cmd error (1)} { - set e {} - catch { - r eval "redis.call('get','a','b','c')" 0 - } e - set e - } {*number of args*} - - test {EVAL - redis.call variant raises a Lua error on Redis cmd error (1)} { - set e {} - r set foo bar - catch { - r eval {redis.call('lpush',KEYS[1],'val')} 1 foo - } e - set e - } {*against a key*} - - test {EVAL - JSON numeric decoding} { - # We must return the table as a string because otherwise - # Redis converts floats to ints and we get 0 and 1023 instead - # of 0.0003 and 1023.2 as the parsed output. - r eval {return - table.concat( - cjson.decode( - "[0.0, -5e3, -1, 0.3e-3, 1023.2, 0e10]"), " ") - } 0 - } {0 -5000 -1 0.0003 1023.2 0} - - test {EVAL - JSON string decoding} { - r eval {local decoded = cjson.decode('{"keya": "a", "keyb": "b"}') - return {decoded.keya, decoded.keyb} - } 0 - } {a b} - - test {EVAL - cmsgpack can pack double?} { - r eval {local encoded = cmsgpack.pack(0.1) - local h = "" - for i = 1, #encoded do - h = h .. string.format("%02x",string.byte(encoded,i)) - end - return h - } 0 - } {cb3fb999999999999a} - - test {EVAL - cmsgpack can pack negative int64?} { - r eval {local encoded = cmsgpack.pack(-1099511627776) - local h = "" - for i = 1, #encoded do - h = h .. string.format("%02x",string.byte(encoded,i)) - end - return h - } 0 - } {d3ffffff0000000000} - test {EVAL - cmsgpack can pack and unpack circular references?} { - r eval {local a = {x=nil,y=5} - local b = {x=a} - a['x'] = b - local encoded = cmsgpack.pack(a) - local h = "" - -- cmsgpack encodes to a depth of 16, but can't encode - -- references, so the encoded object has a deep copy recursive - -- depth of 16. - for i = 1, #encoded do - h = h .. string.format("%02x",string.byte(encoded,i)) - end - -- when unpacked, re.x.x != re because the unpack creates - -- individual tables down to a depth of 16. - -- (that's why the encoded output is so large) - local re = cmsgpack.unpack(encoded) - assert(re) - assert(re.x) - assert(re.x.x.y == re.y) - assert(re.x.x.x.x.y == re.y) - assert(re.x.x.x.x.x.x.y == re.y) - assert(re.x.x.x.x.x.x.x.x.x.x.y == re.y) - -- maximum working depth: - assert(re.x.x.x.x.x.x.x.x.x.x.x.x.x.x.y == re.y) - -- now the last x would be b above and has no y - assert(re.x.x.x.x.x.x.x.x.x.x.x.x.x.x.x) - -- so, the final x.x is at the depth limit and was assigned nil - assert(re.x.x.x.x.x.x.x.x.x.x.x.x.x.x.x.x == nil) - assert(h == "82a17881a17882a17881a17882a17881a17882a17881a17882a17881a17882a17881a17882a17881a17882a17881a178c0a17905a17905a17905a17905a17905a17905a17905a17905" or h == "82a17905a17881a17882a17905a17881a17882a17905a17881a17882a17905a17881a17882a17905a17881a17882a17905a17881a17882a17905a17881a17882a17905a17881a178c0") - return {re.x.x.x.x.x.x.x.x.y == re.y, re.y == 5} - } 0 - } {1 1} - test {EVAL - Numerical sanity check from bitop} { - r eval {assert(0x7fffffff == 2147483647, "broken hex literals"); - assert(0xffffffff == -1 or 0xffffffff == 2^32-1, - "broken hex literals"); - assert(tostring(-1) == "-1", "broken tostring()"); - assert(tostring(0xffffffff) == "-1" or - tostring(0xffffffff) == "4294967295", - "broken tostring()") - } 0 - } {} - - test {EVAL - Verify minimal bitop functionality} { - r eval {assert(bit.tobit(1) == 1); - assert(bit.band(1) == 1); - assert(bit.bxor(1,2) == 3); - assert(bit.bor(1,2,4,8,16,32,64,128) == 255) - } 0 - } {} - - test {EVAL - Able to parse trailing comments} { - r eval {return 'hello' --trailing comment} 0 - } {hello} - - test {EVAL does not leak in the Lua stack} { - r set x 0 - # Use a non blocking client to speedup the loop. - set rd [redis_deferring_client] - for {set j 0} {$j < 10000} {incr j} { - $rd eval {return redis.call("incr",KEYS[1])} 1 x - } - for {set j 0} {$j < 10000} {incr j} { - $rd read - } - assert {[s used_memory_lua] < 1024*100} - $rd close - r get x - } {10000} - - test {SCRIPTING FLUSH - is able to clear the scripts cache?} { - r set mykey myval - set v [r evalsha fd758d1589d044dd850a6f05d52f2eefd27f033f 1 mykey] - assert_equal $v myval - set e "" - r script flush - catch {r evalsha fd758d1589d044dd850a6f05d52f2eefd27f033f 1 mykey} e - set e - } {ERR NOSCRIPT*} - - test {SCRIPT EXISTS - can detect already defined scripts?} { - r eval "return 1+1" 0 - r script exists a27e7e8a43702b7046d4f6a7ccf5b60cef6b9bd9 a27e7e8a43702b7046d4f6a7ccf5b60cef6b9bda - } {1 0} - - test {SCRIPT LOAD - is able to register scripts in the scripting cache} { - list \ - [r script load "return 'loaded'"] \ - [r evalsha b534286061d4b9e4026607613b95c06c06015ae8 0] - } {b534286061d4b9e4026607613b95c06c06015ae8 loaded} - - test {Globals protection reading an undeclared global variable} { - catch {r eval {return a} 0} e - set e - } {*ERR*attempted to access * global*} - - test {Globals protection setting an undeclared global*} { - catch {r eval {a=10} 0} e - set e - } {*ERR*attempted to create global*} - - test {Test an example script DECR_IF_GT} { - set decr_if_gt { - local current - - current = redis.call('get',KEYS[1]) - if not current then return nil end - if current > ARGV[1] then - return redis.call('decr',KEYS[1]) - else - return redis.call('get',KEYS[1]) - end - } - r set foo 5 - set res {} - lappend res [r eval $decr_if_gt 1 foo 2] - lappend res [r eval $decr_if_gt 1 foo 2] - lappend res [r eval $decr_if_gt 1 foo 2] - lappend res [r eval $decr_if_gt 1 foo 2] - lappend res [r eval $decr_if_gt 1 foo 2] - set res - } {4 3 2 2 2} - - test {Scripting engine PRNG can be seeded correctly} { - set rand1 [r eval { - math.randomseed(ARGV[1]); return tostring(math.random()) - } 0 10] - set rand2 [r eval { - math.randomseed(ARGV[1]); return tostring(math.random()) - } 0 10] - set rand3 [r eval { - math.randomseed(ARGV[1]); return tostring(math.random()) - } 0 20] - assert_equal $rand1 $rand2 - assert {$rand2 ne $rand3} - } - test "In the context of Lua the output of random commands gets ordered" { - r del myset - r sadd myset a b c d e f g h i l m n o p q r s t u v z aa aaa azz - r eval {return redis.call('smembers',KEYS[1])} 1 myset - } {a aa aaa azz b c d e f g h i l m n o p q r s t u v z} - - test "Make sure redis.log() works" { - set v [r eval { return redis.log(redis.LOG_DEBUG, 'debug level'); } 0] - assert_equal "" $v - set v [r eval { return redis.log(redis.LOG_VERBOSE, 'verbose level'); } 0] - assert_equal "" $v - set v [r eval { return redis.log(redis.LOG_NOTICE, 'notice level'); } 0] - assert_equal "" $v - set v [r eval { return redis.log(redis.LOG_WARNING, 'warning level'); } 0] - assert_equal "" $v - } {} - - test {EVAL_RO - Successful case} { - r set foo bar - assert_equal bar [r eval_ro {return redis.call('get', KEYS[1]);} 1 foo] - } - - test {EVALSHA_RO - Successful case} { - r set foo bar - assert_equal bar [r evalsha_ro 796941151549c416aa77522fb347487236c05e46 1 foo] - } - - test {EVAL_RO - Cannot run write commands} { - r set foo bar - catch {r eval_ro {redis.call('del', KEYS[1]);} 1 foo} e - set e - } {ERR * Write commands are not allowed from read-only scripts} - - test {EVALSHA_RO - Cannot run write commands} { - r set foo bar - catch {r evalsha_ro a1e63e1cd1bd1d5413851949332cfb9da4ee6dc0 1 foo} e - set e - } {ERR * Write commands are not allowed from read-only scripts} -} - -start_server {tags {"repl"}} { - start_server {} { - set master [srv -1 client] - set master_host [srv -1 host] - set master_port [srv -1 port] - set slave [srv 0 client] - - $slave slaveof $master_host $master_port - wait_for_sync $slave - - test {SCRIPTING: script load on master, read on slave} { - set sha [$master script load "return 'script loaded'"] - assert_equal 4167ea82ed9c381c7659f7cf93f394219147e8c4 $sha - wait_for_ofs_sync $master $slave - assert_equal 1 [$master script exists $sha] - assert_equal 1 [$slave script exists $sha] - - $master script flush - wait_for_ofs_sync $master $slave - assert_equal 0 [$slave script exists $sha] - } - } -} From 399a33d06af9542b3758347a5ee0c5844a032e3c Mon Sep 17 00:00:00 2001 From: Twice Date: Thu, 22 Sep 2022 17:58:58 +0800 Subject: [PATCH 06/22] Support quoted string and inline comment in config (#849) --- src/config.cc | 43 ++++---- src/config.h | 2 +- src/config_util.cc | 195 +++++++++++++++++++++++++++++++++++ src/config_util.h | 40 +++++++ tests/cppunit/config_test.cc | 34 ++++++ 5 files changed, 290 insertions(+), 24 deletions(-) create mode 100644 src/config_util.cc create mode 100644 src/config_util.h diff --git a/src/config.cc b/src/config.cc index 16c097282fa..a98fdf49496 100644 --- a/src/config.cc +++ b/src/config.cc @@ -27,6 +27,8 @@ #include #include #include +#include +#include #include #include @@ -38,6 +40,7 @@ #include "cron.h" #include "server.h" #include "log_collector.h" +#include "config_util.h" const char *kDefaultNamespace = "__namespace"; @@ -558,28 +561,27 @@ void Config::ClearMaster() { } } -Status Config::parseConfigFromString(std::string input, int line_number) { - std::vector kv = Util::Split2KV(input, " \t"); +Status Config::parseConfigFromString(const std::string &input, int line_number) { + auto parsed = ParseConfigLine(input); + if (!parsed) return parsed.ToStatus(); - // skip the comment and empty line - if (kv.empty() || kv[0].front() == '#') return Status::OK(); + auto kv = std::move(*parsed); - if (kv.size() != 2) return Status(Status::NotOK, "wrong number of arguments"); - if (kv[1] == "\"\"") return Status::OK(); + if (kv.first.empty() || kv.second.empty()) return Status::OK(); - std::string field_key = Util::ToLower(kv[0]); + std::string field_key = Util::ToLower(kv.first); const char ns_str[] = "namespace."; size_t ns_str_size = sizeof(ns_str) - 1; - if (!strncasecmp(kv[0].data(), ns_str, ns_str_size)) { + if (!strncasecmp(kv.first.data(), ns_str, ns_str_size)) { // namespace should keep key case-sensitive - field_key = kv[0]; - tokens[kv[1]] = kv[0].substr(ns_str_size); + field_key = kv.first; + tokens[kv.second] = kv.first.substr(ns_str_size); } auto iter = fields_.find(field_key); if (iter != fields_.end()) { auto& field = iter->second; field->line_number = line_number; - auto s = field->Set(kv[1]); + auto s = field->Set(kv.second); if (!s.IsOK()) return s; } return Status::OK(); @@ -711,27 +713,22 @@ Status Config::Rewrite() { std::ifstream file(path_); if (file.is_open()) { - std::string raw_line, trim_line, new_value; - std::vector kv; + std::string raw_line; while (!file.eof()) { std::getline(file, raw_line); - trim_line = Util::Trim(raw_line, " \t\r\n"); - if (trim_line.empty() || trim_line.front() == '#') { + auto parsed = ParseConfigLine(raw_line); + if (!parsed || parsed->first.empty()) { lines.emplace_back(raw_line); continue; } - kv = Util::Split2KV(trim_line, " \t"); - if (kv.size() != 2) { - lines.emplace_back(raw_line); - continue; - } - if (Util::HasPrefix(kv[0], namespacePrefix)) { + auto kv = std::move(*parsed); + if (Util::HasPrefix(kv.first, namespacePrefix)) { // Ignore namespace fields here since we would always rewrite them continue; } - auto iter = new_config.find(Util::ToLower(kv[0])); + auto iter = new_config.find(Util::ToLower(kv.first)); if (iter != new_config.end()) { - if (!iter->second.empty()) lines.emplace_back(iter->first + " " + iter->second); + if (!iter->second.empty()) lines.emplace_back(DumpConfigLine({iter->first, iter->second})); new_config.erase(iter); } else { lines.emplace_back(raw_line); diff --git a/src/config.h b/src/config.h index dec091a9fd5..5c5ec3e63c0 100644 --- a/src/config.h +++ b/src/config.h @@ -210,7 +210,7 @@ struct Config{ void initFieldValidator(); void initFieldCallback(); - Status parseConfigFromString(std::string input, int line_number); + Status parseConfigFromString(const std::string &input, int line_number); Status finish(); Status isNamespaceLegal(const std::string &ns); }; diff --git a/src/config_util.cc b/src/config_util.cc new file mode 100644 index 00000000000..6f22ef877fc --- /dev/null +++ b/src/config_util.cc @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +#include "config_util.h" +#include "util.h" + + +StatusOr ParseConfigLine(const std::string& line) { + enum { + KEY, // in (unquoted) key string + NORMAL, // in unquoted value string + QUOTED, // in quoted value string + PRE_KEY_SPACE, // in whitespace characters before key + AFTER_KEY_SPACE, // in whitespace characters after key and before value + AFTER_VAL_SPACE, // in whitespace characters after value + ESCAPE, // in escape character of quoted string + ERROR // error state, e.g. encounter more than one value + } state = PRE_KEY_SPACE; + + char quote; // single or double quote + std::string current_str; + ConfigKV res; + + for (auto i = line.begin(); i != line.end();) { + switch (state) { + case PRE_KEY_SPACE: + if (!std::isspace(*i)) { + if (*i == '#') { + i = line.end(); + } else { + state = KEY; + } + } else { + i++; + } + break; + case KEY: + if (std::isspace(*i)) { + res.first = current_str; + current_str = ""; + state = AFTER_KEY_SPACE; + } else if (*i == '#') { + res.first = current_str; + i = line.end(); + } else { + current_str.push_back(*i); + i++; + } + break; + case AFTER_KEY_SPACE: + if (!std::isspace(*i)) { + if (*i == '"' || *i == '\'') { + state = QUOTED; + quote = *i; + i++; + } else if (*i == '#') { + i = line.end(); + } else { + state = NORMAL; + } + } else { + i++; + } + break; + case NORMAL: + if (*i == '#') { + res.second = current_str; + i = line.end(); + } else { + current_str.push_back(*i); + i++; + } + break; + case QUOTED: + if (*i == '\\') { + state = ESCAPE; + } else if (*i == quote) { + res.second = current_str; + state = AFTER_VAL_SPACE; + } else { + current_str.push_back(*i); + } + i++; + break; + case ESCAPE: + if (*i == '\'' || *i == '"' || *i == '\\') { + current_str.push_back(*i); + } else if (*i == 't') { + current_str.push_back('\t'); + } else if (*i == 'r') { + current_str.push_back('\r'); + } else if (*i == 'n') { + current_str.push_back('\n'); + } else if (*i == 'v') { + current_str.push_back('\v'); + } else if (*i == 'f') { + current_str.push_back('\f'); + } else if (*i == 'b') { + current_str.push_back('\b'); + } + state = QUOTED; + i++; + break; + case AFTER_VAL_SPACE: + if (!std::isspace(*i)) { + if (*i == '#') { + i = line.end(); + } else { + state = ERROR; + } + } else { + i++; + } + break; + case ERROR: + i = line.end(); + break; + } + } + + + if (state == KEY) { + res.first = current_str; + state = AFTER_KEY_SPACE; + } else if (state == NORMAL) { + res.second = Util::Trim(current_str, " \t\r\n\v\f\b"); + state = AFTER_VAL_SPACE; + } else if (state == QUOTED || state == ESCAPE) { + return {Status::NotOK, "config line ends unexpectedly in quoted string"}; + } else if (state == ERROR) { + return {Status::NotOK, "more than 2 item in config line"}; + } + + return res; +} + +std::string DumpConfigLine(const ConfigKV &config) { + std::string res; + + res += config.first; + res += " "; + + if (std::any_of(config.second.begin(), config.second.end(), [](char c) { + return std::isspace(c) || c == '"' || c == '\'' || c == '#'; + })) { + res += '"'; + for (char c : config.second) { + if (c == '\\') { + res += "\\\\"; + } else if (c == '\'') { + res += "\\'"; + } else if (c == '"') { + res += "\\\""; + } else if (c == '\t') { + res += "\\t"; + } else if (c == '\r') { + res += "\\r"; + } else if (c == '\n') { + res += "\\n"; + } else if (c == '\v') { + res += "\\v"; + } else if (c == '\f') { + res += "\\f"; + } else if (c == '\b') { + res += "\\b"; + } else { + res += c; + } + } + res += '"'; + } else if (config.second.empty()) { + res += "\"\""; + } else { + res += config.second; + } + + return res; +} diff --git a/src/config_util.h b/src/config_util.h new file mode 100644 index 00000000000..9d781ddfbe1 --- /dev/null +++ b/src/config_util.h @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ + +#pragma once + +#include +#include +#include + +#include "status.h" + +using ConfigKV = std::pair; + +// refer to https://redis.io/docs/manual/config +// format: key value +// inline comment: key value # comment +// quoted string: key "hello world" +// e.g. `key "hello # world\"" # hi` -> key: hello # world" +StatusOr ParseConfigLine(const std::string &line); + +// dump a config item to a string line +// e.g. {'a', 'b c'} -> "a 'b c'" +std::string DumpConfigLine(const ConfigKV &config); diff --git a/tests/cppunit/config_test.cc b/tests/cppunit/config_test.cc index 7efc0564e14..add990dad9c 100644 --- a/tests/cppunit/config_test.cc +++ b/tests/cppunit/config_test.cc @@ -25,6 +25,7 @@ #include #include #include +#include TEST(Config, GetAndSet) { const char *path = "test.conf"; @@ -276,3 +277,36 @@ TEST(Namespace, RewriteNamespaces) { EXPECT_FALSE(new_config.GetNamespace("to-be-deleted-ns", &token).IsOK()); unlink(path); } + +TEST(Config, ParseConfigLine) { + ASSERT_EQ(*ParseConfigLine(""), ConfigKV{}); + ASSERT_EQ(*ParseConfigLine("# hello"), ConfigKV{}); + ASSERT_EQ(*ParseConfigLine(" #x y z "), ConfigKV{}); + ASSERT_EQ(*ParseConfigLine("key value "), (ConfigKV{"key", "value"})); + ASSERT_EQ(*ParseConfigLine("key value#x"), (ConfigKV{"key", "value"})); + ASSERT_EQ(*ParseConfigLine("key"), (ConfigKV{"key", ""})); + ASSERT_EQ(*ParseConfigLine(" key value1 value2 "), (ConfigKV{"key", "value1 value2"})); + ASSERT_EQ(*ParseConfigLine(" #"), ConfigKV{}); + ASSERT_EQ(*ParseConfigLine(" key val ue #h e l l o"), (ConfigKV{"key", "val ue"})); + ASSERT_EQ(*ParseConfigLine("key 'val ue'"), (ConfigKV{"key", "val ue"})); + ASSERT_EQ(*ParseConfigLine(R"(key ' value\'\'v a l ')"), (ConfigKV{"key", " value''v a l "})); + ASSERT_EQ(*ParseConfigLine(R"( key "val # hi" # hello!)"), (ConfigKV{"key", "val # hi"})); + ASSERT_EQ(*ParseConfigLine(R"(key "\n \r \t ")"), (ConfigKV{"key", "\n \r \t "})); + ASSERT_EQ(*ParseConfigLine("key ''"), (ConfigKV{"key", ""})); + ASSERT_FALSE(ParseConfigLine("key \"hello ")); + ASSERT_FALSE(ParseConfigLine("key \'\\")); + ASSERT_FALSE(ParseConfigLine("key \"hello'")); + ASSERT_FALSE(ParseConfigLine("key \"")); + ASSERT_FALSE(ParseConfigLine("key '' ''")); + ASSERT_FALSE(ParseConfigLine("key '' x")); +} + +TEST(Config, DumpConfigLine) { + ASSERT_EQ(DumpConfigLine({"key", "value"}), "key value"); + ASSERT_EQ(DumpConfigLine({"key", " v a l "}), R"(key " v a l ")"); + ASSERT_EQ(DumpConfigLine({"a", "'b"}), "a \"\\'b\""); + ASSERT_EQ(DumpConfigLine({"a", "x#y"}), "a \"x#y\""); + ASSERT_EQ(DumpConfigLine({"a", "x y"}), "a \"x y\""); + ASSERT_EQ(DumpConfigLine({"a", "xy"}), "a xy"); + ASSERT_EQ(DumpConfigLine({"a", "x\n"}), "a \"x\\n\""); +} From d469155d7493ef2d61559eb525e84b3758dd7cbc Mon Sep 17 00:00:00 2001 From: Twice Date: Fri, 23 Sep 2022 08:36:25 +0800 Subject: [PATCH 07/22] Fix sanitizer report checking in CI (#912) --- .github/workflows/kvrocks.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/kvrocks.yaml b/.github/workflows/kvrocks.yaml index 46c83fbdf06..15dc6838f14 100644 --- a/.github/workflows/kvrocks.yaml +++ b/.github/workflows/kvrocks.yaml @@ -208,11 +208,11 @@ jobs: cp minica.pem tests/tcl/tests/tls/ca.crt ./x.py test tcl build --tls --single unit/tls --dont-clean fi - SANITIZER_OUTPUT=$(grep "Sanitizer:" tests/tmp -r || true) + SANITIZER_OUTPUT=$(grep "Sanitizer:" tests/tcl/tests/tmp -r || true) if [[ $SANITIZER_OUTPUT ]]; then echo "$SANITIZER_OUTPUT" echo "\ndetail reports:\n" - cat $(find tests/tmp -iname stderr) + cat $(find tests/tcl/tests/tmp -iname stderr) echo "sanitizer error was reported, exiting..." exit 1 fi From 671cab35bbbaf57aeb969025c3c688d8e0dd7e6b Mon Sep 17 00:00:00 2001 From: Ruixiang Tan <819464715@qq.com> Date: Fri, 23 Sep 2022 09:45:03 +0800 Subject: [PATCH 08/22] Remove unnecessary Tostring (#909) --- src/redis_bitmap.cc | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/redis_bitmap.cc b/src/redis_bitmap.cc index 02b316b8f31..1e592c1acb7 100644 --- a/src/redis_bitmap.cc +++ b/src/redis_bitmap.cc @@ -549,7 +549,6 @@ bool Bitmap::GetBitFromValueAndOffset(const std::string &value, uint32_t offset) bool Bitmap::IsEmptySegment(const Slice &segment) { static const char zero_byte_segment[kBitmapSegmentBytes] = {0}; - std::string value = segment.ToString(); - return !memcmp(zero_byte_segment, value.c_str(), value.size()); + return !memcmp(zero_byte_segment, segment.data(), segment.size()); } } // namespace Redis From 114af54a57aa5949910d53d81c8d67002c9d3b41 Mon Sep 17 00:00:00 2001 From: Andrei Vydrin Date: Fri, 23 Sep 2022 08:46:19 +0700 Subject: [PATCH 09/22] Add the FilterBlobByKey for SubKeyFilter (#902) --- src/compact_filter.cc | 79 +++++++++++++++++++++++++++++-------------- src/compact_filter.h | 6 +++- 2 files changed, 58 insertions(+), 27 deletions(-) diff --git a/src/compact_filter.cc b/src/compact_filter.cc index e76440165b3..60e57403abd 100644 --- a/src/compact_filter.cc +++ b/src/compact_filter.cc @@ -50,14 +50,13 @@ bool MetadataFilter::Filter(int level, return metadata.Expired(); } -bool SubKeyFilter::IsKeyExpired(const InternalKey &ikey, const Slice &value) const { +Status SubKeyFilter::GetMetadata(const InternalKey &ikey, Metadata* metadata) const { std::string metadata_key; auto db = stor_->GetDB(); const auto cf_handles = stor_->GetCFHandles(); // storage close the would delete the column family handler and DB - if (!db || cf_handles->size() < 2) return false; - + if (!db || cf_handles->size() < 2) return Status(Status::NotOK, "storage is closed"); ComposeNamespaceKey(ikey.GetNamespace(), ikey.GetKey(), &metadata_key, stor_->IsSlotIdEncoded()); if (cached_key_.empty() || metadata_key != cached_key_) { @@ -70,36 +69,56 @@ bool SubKeyFilter::IsKeyExpired(const InternalKey &ikey, const Slice &value) con // metadata was deleted(perhaps compaction or manual) // clear the metadata cached_metadata_.clear(); - return true; + return Status(Status::NotFound, "metadata is not found"); } else { - LOG(ERROR) << "[compact_filter/subkey] Failed to fetch metadata" - << ", namespace: " << ikey.GetNamespace().ToString() - << ", key: " << ikey.GetKey().ToString() - << ", err: " << s.ToString(); cached_key_.clear(); cached_metadata_.clear(); - return false; + return Status(Status::NotOK, "fetch error: " + s.ToString()); } } // the metadata was not found - if (cached_metadata_.empty()) return true; + if (cached_metadata_.empty()) return Status(Status::NotFound, "metadata is not found"); // the metadata is cached - Metadata metadata(kRedisNone, false); - rocksdb::Status s = metadata.Decode(cached_metadata_); + rocksdb::Status s = metadata->Decode(cached_metadata_); if (!s.ok()) { cached_key_.clear(); - LOG(ERROR) << "[compact_filter/subkey] Failed to decode metadata" - << ", namespace: " << ikey.GetNamespace().ToString() - << ", key: " << ikey.GetKey().ToString() - << ", err: " << s.ToString(); - return false; + return Status(Status::NotOK, "decode error: " + s.ToString());; } + return Status::OK(); +} + +bool SubKeyFilter::IsMetadataExpired(const InternalKey &ikey, const Metadata& metadata) const { if (metadata.Type() == kRedisString // metadata key was overwrite by set command || metadata.Expired() || ikey.GetVersion() != metadata.version) { return true; } - return metadata.Type() == kRedisBitmap && Redis::Bitmap::IsEmptySegment(value); + return false; +} + +rocksdb::CompactionFilter::Decision SubKeyFilter::FilterBlobByKey(int level, const Slice &key, + std::string *new_value, + std::string *skip_until) const { + InternalKey ikey(key, stor_->IsSlotIdEncoded()); + Metadata metadata(kRedisNone, false); + Status s = GetMetadata(ikey, &metadata); + if (s.Is()) { + return rocksdb::CompactionFilter::Decision::kRemove; + } + if (!s.IsOK()) { + LOG(ERROR) << "[compact_filter/subkey] Failed to get metadata" + << ", namespace: " << ikey.GetNamespace().ToString() + << ", key: " << ikey.GetKey().ToString() + << ", err: " << s.Msg(); + return rocksdb::CompactionFilter::Decision::kKeep; + } + // bitmap will be checked in Filter + if (metadata.Type() == kRedisBitmap) { + return rocksdb::CompactionFilter::Decision::kUndetermined; + } + + bool result = IsMetadataExpired(ikey, metadata); + return result ? rocksdb::CompactionFilter::Decision::kRemove : rocksdb::CompactionFilter::Decision::kKeep; } bool SubKeyFilter::Filter(int level, @@ -108,13 +127,21 @@ bool SubKeyFilter::Filter(int level, std::string *new_value, bool *modified) const { InternalKey ikey(key, stor_->IsSlotIdEncoded()); - bool result = IsKeyExpired(ikey, value); - DLOG(INFO) << "[compact_filter/subkey] " - << "namespace: " << ikey.GetNamespace().ToString() - << ", metadata key: " << ikey.GetKey().ToString() - << ", subkey: " << ikey.GetSubKey().ToString() - << ", verison: " << ikey.GetVersion() - << ", result: " << (result ? "deleted" : "reserved"); - return result; + Metadata metadata(kRedisNone, false); + Status s = GetMetadata(ikey, &metadata); + if (s.Is()) { + return true; + } + if (!s.IsOK()) { + LOG(ERROR) << "[compact_filter/subkey] Failed to get metadata" + << ", namespace: " << ikey.GetNamespace().ToString() + << ", key: " << ikey.GetKey().ToString() + << ", err: " << s.Msg(); + return false; + } + + return IsMetadataExpired(ikey, metadata) || + (metadata.Type() == kRedisBitmap && Redis::Bitmap::IsEmptySegment(value)); } + } // namespace Engine diff --git a/src/compact_filter.h b/src/compact_filter.h index bedb4bd5a92..f296252328b 100644 --- a/src/compact_filter.h +++ b/src/compact_filter.h @@ -64,7 +64,11 @@ class SubKeyFilter : public rocksdb::CompactionFilter { stor_(storage) {} const char *Name() const override { return "SubkeyFilter"; } - bool IsKeyExpired(const InternalKey &ikey, const Slice &value) const; + Status GetMetadata(const InternalKey &ikey, Metadata* metadata) const; + bool IsMetadataExpired(const InternalKey &ikey, const Metadata& metadata) const; + rocksdb::CompactionFilter::Decision FilterBlobByKey(int level, const Slice &key, + std::string *new_value, + std::string *skip_until) const override; bool Filter(int level, const Slice &key, const Slice &value, std::string *new_value, bool *modified) const override; From 2c88a86205b2c52ec445e972b485ecd4f42e2b5e Mon Sep 17 00:00:00 2001 From: XiongDa <52746580+shangxiaoxiong@users.noreply.github.com> Date: Fri, 23 Sep 2022 13:46:35 +0800 Subject: [PATCH 10/22] Fix config set rocksdb.blob_garbage_collection_age_cutoff (#914) --- src/config.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/config.cc b/src/config.cc index a98fdf49496..ea72084739d 100644 --- a/src/config.cc +++ b/src/config.cc @@ -483,7 +483,7 @@ void Config::initFieldCallback() { return Status(Status::NotOK, "blob_garbage_collection_age_cutoff must >= 0 and <= 100."); } - double cutoff = val / 100; + double cutoff = val / 100.0; return srv->storage_->SetColumnFamilyOption(trimRocksDBPrefix(k), std::to_string(cutoff)); }}, {"rocksdb.level_compaction_dynamic_level_bytes", [](Server* srv, const std::string &k, From afd8668ed536f96d3efa0551c17819b2c28f5e7e Mon Sep 17 00:00:00 2001 From: Myth Date: Sat, 24 Sep 2022 17:00:48 +0800 Subject: [PATCH 11/22] Avoid using Get when iterating sub-keys during migration (#906) --- src/slot_migrate.cc | 22 ++++------------------ src/slot_migrate.h | 3 +-- 2 files changed, 5 insertions(+), 20 deletions(-) diff --git a/src/slot_migrate.cc b/src/slot_migrate.cc index 14a4b79ca61..a65a3cbb479 100644 --- a/src/slot_migrate.cc +++ b/src/slot_migrate.cc @@ -302,7 +302,7 @@ Status SlotMigrate::SendSnapshot(void) { // Add key's constructed cmd to restore_cmds, send pipeline // or not according to current_pipeline_size_ - auto stat = MigrateOneKey(rocksdb::Slice(user_key), &restore_cmds); + auto stat = MigrateOneKey(user_key, iter->value(), &restore_cmds); if (stat.IsOK()) { if (stat.Msg() == "ok") migratedkey_cnt++; if (stat.Msg() == "expired") expiredkey_cnt++; @@ -543,29 +543,15 @@ bool SlotMigrate::CheckResponseWithCounts(int sock_fd, int total) { return true; // Can't reach here } -bool SlotMigrate::GetSlotKeyMetadata(const rocksdb::Slice &prefix_key, std::string *bytes) { - rocksdb::ReadOptions read_options; - read_options.snapshot = slot_snapshot_; - auto s = storage_->GetDB()->Get(read_options, storage_->GetCFHandle("metadata"), prefix_key, bytes); - if (!s.ok()) { - LOG(ERROR) << "[migrate] Failed to get metadata of key: " << prefix_key.ToString() - << ", Err: " << s.ToString(); - return false; - } - return true; -} - -Status SlotMigrate::MigrateOneKey(rocksdb::Slice key, std::string *restore_cmds) { +Status SlotMigrate::MigrateOneKey(const rocksdb::Slice &key, const rocksdb::Slice &value, std::string *restore_cmds) { std::string prefix_key; AppendNamespacePrefix(key, &prefix_key); - std::string bytes; - bool st = GetSlotKeyMetadata(prefix_key, &bytes); - if (!st) return Status(Status::NotOK, "[migrate] Failed to get key's metadata"); + std::string bytes = value.ToString(); Metadata metadata(kRedisNone, false); metadata.Decode(bytes); if (metadata.Type() != kRedisString && metadata.size == 0) { LOG(INFO) << "[migrate] No elements of key: " << prefix_key; - return Status(Status::cOK, "empty");; + return Status(Status::cOK, "empty"); } // Construct command according to type of the key diff --git a/src/slot_migrate.h b/src/slot_migrate.h index 4eae2636e47..e8c6864a79f 100644 --- a/src/slot_migrate.h +++ b/src/slot_migrate.h @@ -112,8 +112,7 @@ class SlotMigrate : public Redis::Database { bool CheckResponseOnce(int sock_fd); bool CheckResponseWithCounts(int sock_fd, int total); - bool GetSlotKeyMetadata(const rocksdb::Slice &prefix_key, std::string *bytes); - Status MigrateOneKey(rocksdb::Slice key, std::string *restore_cmds); + Status MigrateOneKey(const rocksdb::Slice &key, const rocksdb::Slice &value, std::string *restore_cmds); bool MigrateSimpleKey(const rocksdb::Slice &key, const Metadata &metadata, const std::string &bytes, std::string *restore_cmds); bool MigrateComplexKey(const rocksdb::Slice &key, const Metadata &metadata, std::string *restore_cmds); From 231d48d5b0129ca0ebb5e070bb769264dd0d0ce5 Mon Sep 17 00:00:00 2001 From: ColinChamber <95634254+ColinChamber@users.noreply.github.com> Date: Sat, 24 Sep 2022 22:00:36 +0800 Subject: [PATCH 12/22] Send SIGTERM after finishing the unit test (#898) --- tests/gocase/util/constants.go | 6 ++++++ tests/gocase/util/server.go | 9 +++++++-- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/tests/gocase/util/constants.go b/tests/gocase/util/constants.go index cdb8bf429df..978879625b2 100644 --- a/tests/gocase/util/constants.go +++ b/tests/gocase/util/constants.go @@ -21,3 +21,9 @@ package util const DefaultDelta = 0.000001 const ErrRedisNil = "redis: nil" + +// Kubernetes will send a SIGTERM signal to the containers in the pod after deleting the pod. +// It waits for a specified time, called the termination grace period. By default, this is 30 seconds. +// If the containers are still running after the grace period, +// they are sent the SIGKILL signal and forcibly removed. +const k8sDefaultGracePeriod = 30 diff --git a/tests/gocase/util/server.go b/tests/gocase/util/server.go index 10cca148c33..d6f5285589c 100644 --- a/tests/gocase/util/server.go +++ b/tests/gocase/util/server.go @@ -26,6 +26,7 @@ import ( "os" "os/exec" "path/filepath" + "syscall" "testing" "time" @@ -67,8 +68,12 @@ func (s *KvrocksServer) NewTCPClient() *tcpClient { } func (s *KvrocksServer) Close() { - require.NoError(s.t, s.cmd.Process.Kill()) - require.EqualError(s.t, s.cmd.Wait(), "signal: killed") + require.NoError(s.t, s.cmd.Process.Signal(syscall.SIGTERM)) + timer := time.AfterFunc(k8sDefaultGracePeriod*time.Second, func() { + require.NoError(s.t, s.cmd.Process.Kill()) + }) + defer timer.Stop() + require.NoError(s.t, s.cmd.Wait()) s.clean() } From 91cebf2ccec66c711d64cd207b90ff9e600a565d Mon Sep 17 00:00:00 2001 From: hulk Date: Sat, 24 Sep 2022 23:14:22 +0800 Subject: [PATCH 13/22] Disable the TLS by default (#911) --- .github/workflows/kvrocks.yaml | 20 ++++++++++++++------ CMakeLists.txt | 2 +- Dockerfile | 4 ++-- README.md | 7 +++++++ 4 files changed, 24 insertions(+), 9 deletions(-) diff --git a/.github/workflows/kvrocks.yaml b/.github/workflows/kvrocks.yaml index 15dc6838f14..5bd1bdf1d93 100644 --- a/.github/workflows/kvrocks.yaml +++ b/.github/workflows/kvrocks.yaml @@ -72,24 +72,30 @@ jobs: - name: Darwin Clang os: macos-11 compiler: auto + with_openssl: -DENABLE_OPENSSL=ON - name: Darwin Clang without Jemalloc os: macos-11 compiler: auto without_jemalloc: -DDISABLE_JEMALLOC=ON + with_openssl: -DENABLE_OPENSSL=ON - name: Ubuntu GCC os: ubuntu-18.04 compiler: gcc + with_openssl: -DENABLE_OPENSSL=ON - name: Ubuntu Clang os: ubuntu-18.04 compiler: clang + with_openssl: -DENABLE_OPENSSL=ON - name: Ubuntu GCC ASan os: ubuntu-18.04 without_jemalloc: -DDISABLE_JEMALLOC=ON with_sanitizer: -DENABLE_ASAN=ON + with_openssl: -DENABLE_OPENSSL=ON compiler: gcc - name: Ubuntu GCC TSan os: ubuntu-18.04 without_jemalloc: -DDISABLE_JEMALLOC=ON + with_openssl: -DENABLE_OPENSSL=ON with_sanitizer: -DENABLE_TSAN=ON compiler: gcc runtime_env_vars: TSAN_OPTIONS="suppressions=$(realpath ./tests/tsan-suppressions)" @@ -97,30 +103,34 @@ jobs: os: ubuntu-18.04 with_sanitizer: -DENABLE_ASAN=ON without_jemalloc: -DDISABLE_JEMALLOC=ON + with_openssl: -DENABLE_OPENSSL=ON compiler: clang - name: Ubuntu Clang TSan os: ubuntu-18.04 with_sanitizer: -DENABLE_TSAN=ON without_jemalloc: -DDISABLE_JEMALLOC=ON + with_openssl: -DENABLE_OPENSSL=ON compiler: clang runtime_env_vars: TSAN_OPTIONS="suppressions=$(realpath ./tests/tsan-suppressions)" - name: Ubuntu GCC Ninja os: ubuntu-18.04 with_ninja: --ninja + with_openssl: -DENABLE_OPENSSL=ON compiler: gcc - name: Ubuntu GCC Ninja without Jemalloc os: ubuntu-18.04 with_ninja: --ninja without_jemalloc: -DDISABLE_JEMALLOC=ON + with_openssl: -DENABLE_OPENSSL=ON compiler: gcc - name: Ubuntu GCC without luajit os: ubuntu-18.04 without_luajit: -DUSE_LUAJIT=OFF + with_openssl: -DENABLE_OPENSSL=ON compiler: gcc - name: Ubuntu GCC without OpenSSL os: ubuntu-18.04 compiler: gcc - without_openssl: -DENABLE_OPENSSL=OFF runs-on: ${{ matrix.os }} steps: @@ -164,10 +174,8 @@ jobs: - name: Build Kvrocks run: | - ./x.py build -j$NPROC --unittest --compiler \ - ${{ matrix.compiler }} ${{ matrix.with_ninja }} ${{ matrix.with_sanitizer }} \ - ${{ matrix.without_jemalloc }} ${{ matrix.without_openssl }} ${{ matrix.without_luajit }} \ - ${{ env.CMAKE_EXTRA_DEFS }} + ./x.py build -j$NPROC --unittest --compiler ${{ matrix.compiler }} ${{ matrix.without_jemalloc }} ${{ matrix.without_luajit }} \ + ${{ matrix.with_ninja }} ${{ matrix.with_sanitizer }} ${{ matrix.with_openssl }} ${{ env.CMAKE_EXTRA_DEFS }} - name: Run Unit Test run: | @@ -196,7 +204,7 @@ jobs: run: | export ${{ matrix.runtime_env_vars }} ./x.py test tcl build --cli-path $HOME/local/bin/redis-cli --dont-clean - if [[ -z "${{ matrix.without_openssl }}" ]] && [[ "${{ matrix.os }}" == ubuntu* ]]; then + if [[ -n "${{ matrix.with_openssl }}" ]] && [[ "${{ matrix.os }}" == ubuntu* ]]; then git clone https://github.com/jsha/minica cd minica && go build && cd .. ./minica/minica --domains localhost diff --git a/CMakeLists.txt b/CMakeLists.txt index 0a529bcedd3..e41f1741a0a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -26,7 +26,7 @@ option(ENABLE_TSAN "enable thread santinizer" OFF) option(ASAN_WITH_LSAN "enable leak santinizer while address santinizer is enabled" ON) option(ENABLE_STATIC_LIBSTDCXX "link kvrocks with static library of libstd++ instead of shared library" ON) option(USE_LUAJIT "use luaJIT instead of lua" ON) -option(ENABLE_OPENSSL "enable openssl to support tls connection" ON) +option(ENABLE_OPENSSL "enable openssl to support tls connection" OFF) if (CMAKE_VERSION VERSION_GREATER_EQUAL "3.24.0") cmake_policy(SET CMP0135 NEW) diff --git a/Dockerfile b/Dockerfile index bd32996b9c0..8e806d04e6b 100644 --- a/Dockerfile +++ b/Dockerfile @@ -26,7 +26,7 @@ RUN apt install -y git gcc g++ make cmake autoconf automake libtool python3 libs WORKDIR /kvrocks COPY . . -RUN ./x.py build +RUN ./x.py build -DENABLE_OPENSSL=ON FROM ubuntu:focal @@ -38,5 +38,5 @@ COPY ./kvrocks.conf ./conf/ RUN sed -i -e 's%dir /tmp/kvrocks%dir /var/lib/kvrocks%g' ./conf/kvrocks.conf VOLUME /var/lib/kvrocks -EXPOSE 6666:6666 +EXPOSE 6666:6666 ENTRYPOINT ["./bin/kvrocks", "-c", "./conf/kvrocks.conf"] diff --git a/README.md b/README.md index da91e0b9cb9..72d1eada9b7 100644 --- a/README.md +++ b/README.md @@ -93,6 +93,13 @@ $ ./x.py build # `./x.py build -h` to check more options; # especially, `./x.py build --ghproxy` will fetch dependencies via ghproxy.com. ``` +To build with TLS support, you'll need OpenSSL development libraries (e.g. libssl-dev on Debian/Ubuntu) and run: + +```shell +$ ./x.py build -DENABLE_OPENSSL=ON +``` + + ### Running kvrocks ```shell From 6d8e8f56269ac5e77e3dba88feeae2f1fb6e54db Mon Sep 17 00:00:00 2001 From: Ruixiang Tan <819464715@qq.com> Date: Sun, 25 Sep 2022 16:04:27 +0800 Subject: [PATCH 14/22] Replace `std::stol` with ParseInt (#897) --- src/redis_cmd.cc | 177 ++++++++++++++--------------- src/redis_hash.cc | 13 ++- src/redis_request.cc | 7 +- src/redis_string.cc | 10 +- src/util.cc | 23 ++-- tests/cppunit/t_hash_test.cc | 8 +- tests/tcl/tests/unit/type/hash.tcl | 4 +- 7 files changed, 120 insertions(+), 122 deletions(-) diff --git a/src/redis_cmd.cc b/src/redis_cmd.cc index 553c77f88b2..9e6d1771f8d 100644 --- a/src/redis_cmd.cc +++ b/src/redis_cmd.cc @@ -490,16 +490,12 @@ class CommandSet : public Commander { if (ttl_ <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); } else if (opt == "px" && !ttl_ && !last_arg) { int64_t ttl_ms = 0; - try { - std::string s = args_[++i]; - std::string::size_type sz; - ttl_ms = std::stol(s, &sz); - if (sz != s.size()) { - return Status(Status::RedisParseErr, errValueNotInterger); - } - } catch (std::exception &e) { + std::string s = args_[++i]; + auto parse_result = ParseInt(s, 10); + if (!parse_result) { return Status(Status::RedisParseErr, errValueNotInterger); } + ttl_ms = *parse_result; if (ttl_ms <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); if (ttl_ms > 0 && ttl_ms < 1000) { ttl_ = 1; // round up the pttl to second @@ -566,17 +562,16 @@ class CommandSetEX : public Commander { class CommandPSetEX : public Commander { public: Status Parse(const std::vector &args) override { - try { - auto ttl_ms = std::stol(args[2]); - if (ttl_ms <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); - if (ttl_ms > 0 && ttl_ms < 1000) { - ttl_ = 1; - } else { - ttl_ = ttl_ms / 1000; - } - } catch (std::exception &e) { + auto ttl_ms = ParseInt(args[2], 10); + if (!ttl_ms) { return Status(Status::RedisParseErr, errValueNotInterger); } + if (*ttl_ms <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); + if (*ttl_ms > 0 && *ttl_ms < 1000) { + ttl_ = 1; + } else { + ttl_ = *ttl_ms / 1000; + } return Commander::Parse(args); } @@ -679,11 +674,11 @@ class CommandDecr : public Commander { class CommandIncrBy : public Commander { public: Status Parse(const std::vector &args) override { - try { - increment_ = std::stoll(args[2]); - } catch (std::exception &e) { + auto parse_result = ParseInt(args[2], 10); + if (!parse_result) { return Status(Status::RedisParseErr, errValueNotInterger); } + increment_ = *parse_result; return Commander::Parse(args); } @@ -727,11 +722,11 @@ class CommandIncrByFloat : public Commander { class CommandDecrBy : public Commander { public: Status Parse(const std::vector &args) override { - try { - increment_ = std::stoll(args[2]); - } catch (std::exception &e) { + auto parse_result = ParseInt(args[2], 10); + if (!parse_result) { return Status(Status::RedisParseErr, errValueNotInterger); } + increment_ = *parse_result; return Commander::Parse(args); } @@ -822,16 +817,12 @@ class CommandDel : public Commander { }; Status getBitOffsetFromArgument(std::string arg, uint32_t *offset) { - int64_t offset_arg = 0; - try { - offset_arg = std::stoll(arg); - } catch (std::exception &e) { - return Status(Status::RedisParseErr, errValueNotInterger); - } - if (offset_arg < 0 || offset_arg > UINT_MAX) { - return Status(Status::RedisParseErr, "bit offset is out of range"); + auto parse_result = ParseInt(arg, 10); + if (!parse_result) { + return parse_result.ToStatus(); } - *offset = static_cast(offset_arg); + + *offset = *parse_result; return Status::OK(); } @@ -891,12 +882,16 @@ class CommandBitCount : public Commander { return Status(Status::RedisParseErr, errInvalidSyntax); } if (args.size() == 4) { - try { - start_ = std::stol(args[2]); - stop_ = std::stol(args[3]); - } catch (std::exception &e) { + auto parse_start = ParseInt(args[2], 10); + if (!parse_start) { return Status(Status::RedisParseErr, errValueNotInterger); } + start_ = *parse_start; + auto parse_stop = ParseInt(args[3], 10); + if (!parse_stop) { + return Status(Status::RedisParseErr, errValueNotInterger); + } + stop_ = *parse_stop; } return Commander::Parse(args); } @@ -917,14 +912,20 @@ class CommandBitCount : public Commander { class CommandBitPos: public Commander { public: Status Parse(const std::vector &args) override { - try { - if (args.size() >= 4) start_ = std::stol(args[3]); - if (args.size() >= 5) { - stop_given_ = true; - stop_ = std::stol(args[4]); + if (args.size() >= 4) { + auto parse_start = ParseInt(args[3], 10); + if (!parse_start) { + return Status(Status::RedisParseErr, errValueNotInterger); } - } catch (std::exception &e) { - return Status(Status::RedisParseErr, errValueNotInterger); + start_ = *parse_start; + } + if (args.size() >= 5) { + auto parse_stop = ParseInt(args[4], 10); + if (!parse_stop) { + return Status(Status::RedisParseErr, errValueNotInterger); + } + stop_given_ = true; + stop_ = *parse_stop; } if (args[2] == "0") { bit_ = false; @@ -1106,20 +1107,19 @@ class CommandPExpire : public Commander { Status Parse(const std::vector &args) override { int64_t now; rocksdb::Env::Default()->GetCurrentTime(&now); - try { - auto ttl_ms = std::stol(args[2]); - if (ttl_ms > 0 && ttl_ms < 1000) { - seconds_ = 1; - } else { - seconds_ = ttl_ms / 1000; - if (seconds_ >= INT32_MAX - now) { - return Status(Status::RedisParseErr, "the expire time was overflow"); - } - } - seconds_ += now; - } catch (std::exception &e) { + auto ttl_ms = ParseInt(args[2], 10); + if (!ttl_ms) { return Status(Status::RedisParseErr, errValueNotInterger); } + if (*ttl_ms > 0 && *ttl_ms < 1000) { + seconds_ = 1; + } else { + seconds_ = *ttl_ms / 1000; + if (seconds_ >= INT32_MAX - now) { + return Status(Status::RedisParseErr, "the expire time was overflow"); + } + } + seconds_ += now; return Commander::Parse(args); } @@ -1169,14 +1169,14 @@ class CommandExpireAt : public Commander { class CommandPExpireAt : public Commander { public: Status Parse(const std::vector &args) override { - try { - timestamp_ = static_cast(std::stol(args[2])/1000); - if (timestamp_ >= INT32_MAX) { - return Status(Status::RedisParseErr, "the expire time was overflow"); - } - } catch (std::exception &e) { + auto parse_result = ParseInt(args[2], 10); + if (!parse_result) { return Status(Status::RedisParseErr, errValueNotInterger); } + if (*parse_result/1000 >= INT32_MAX) { + return Status(Status::RedisParseErr, "the expire time was overflow"); + } + timestamp_ = static_cast(*parse_result/1000); return Commander::Parse(args); } Status Execute(Server *svr, Connection *conn, std::string *output) override { @@ -1303,11 +1303,11 @@ class CommandHLen : public Commander { class CommandHIncrBy : public Commander { public: Status Parse(const std::vector &args) override { - try { - increment_ = std::stoll(args[3]); - } catch (std::exception &e) { + auto parse_result = ParseInt(args[3], 10); + if (!parse_result) { return Status(Status::RedisParseErr, errValueNotInterger); } + increment_ = *parse_result; return Commander::Parse(args); } Status Execute(Server *svr, Connection *conn, std::string *output) override { @@ -1557,16 +1557,15 @@ class CommandPop : public Commander { if (args.size() == 2) { return Status::OK(); } - try { - int32_t v = std::stol(args[2]); - if (v < 0) { - return Status(Status::RedisParseErr, errValueMustBePositive); - } - count_ = v; - with_count_ = true; - } catch (const std::exception& ) { + auto v = ParseInt(args[2], 10); + if (!v) { return Status(Status::RedisParseErr, errValueNotInterger); } + if (*v < 0) { + return Status(Status::RedisParseErr, errValueMustBePositive); + } + count_ = *v; + with_count_ = true; return Status::OK(); } @@ -4005,11 +4004,11 @@ class CommandClient : public Commander { if (!strcasecmp(args[i].c_str(), "addr") && moreargs) { addr_ = args[i+1]; } else if (!strcasecmp(args[i].c_str(), "id") && moreargs) { - try { - id_ = std::stoll(args[i+1]); - } catch (std::exception &e) { + auto parse_result = ParseInt(args[i+1], 10); + if (!parse_result) { return Status(Status::RedisParseErr, errValueNotInterger); } + id_ = *parse_result; } else if (!strcasecmp(args[i].c_str(), "skipme") && moreargs) { if (!strcasecmp(args[i+1].c_str(), "yes")) { skipme_ = true; @@ -4201,12 +4200,12 @@ class CommandHello final : public Commander { size_t next_arg = 1; if (args_.size() >= 2) { int64_t protocol; - auto parseResult = ParseInt(args_[next_arg], /* base= */ 10); + auto parse_result = ParseInt(args_[next_arg], 10); ++next_arg; - if (!parseResult.IsOK()) { + if (!parse_result) { return Status(Status::NotOK, "Protocol version is not an integer or out of range"); } - protocol = parseResult.GetValue(); + protocol = *parse_result; // In redis, it will check protocol < 2 or protocol > 3, // kvrocks only supports REPL2 by now, but for supporting some @@ -5428,13 +5427,12 @@ class CommandXRead : public Commander { if (i+1 >= args.size()) { return Status(Status::RedisParseErr, errInvalidSyntax); } - - try { - with_count_ = true; - count_ = static_cast(std::stoll(args[i+1])); - } catch (const std::exception &) { + with_count_ = true; + auto parse_result = ParseInt(args[i+1], 10); + if (!parse_result) { return Status(Status::RedisParseErr, errValueNotInterger); } + count_ = *parse_result; i += 2; continue; } @@ -5445,15 +5443,14 @@ class CommandXRead : public Commander { } block_ = true; - try { - auto v = std::stoll(args[i+1]); - if (v < 0) { - return Status(Status::RedisParseErr, errTimeoutIsNegative); - } - block_timeout_ = v; - } catch (const std::exception &) { + auto parse_result = ParseInt(args[i+1], 10); + if (!parse_result) { return Status(Status::RedisParseErr, errValueNotInterger); } + if (*parse_result < 0) { + return Status(Status::RedisParseErr, errTimeoutIsNegative); + } + block_timeout_ = *parse_result; i += 2; continue; } diff --git a/src/redis_hash.cc b/src/redis_hash.cc index 0fbf407408c..7faadbc07e9 100644 --- a/src/redis_hash.cc +++ b/src/redis_hash.cc @@ -19,6 +19,7 @@ */ #include "redis_hash.h" +#include #include #include #include @@ -26,6 +27,7 @@ #include #include #include "db_util.h" +#include "parse_util.h" namespace Redis { rocksdb::Status Hash::GetMetadata(const Slice &ns_key, HashMetadata *metadata) { @@ -74,18 +76,17 @@ rocksdb::Status Hash::IncrBy(const Slice &user_key, const Slice &field, int64_t InternalKey(ns_key, field, metadata.version, storage_->IsSlotIdEncoded()).Encode(&sub_key); if (s.ok()) { std::string value_bytes; - std::size_t idx = 0; s = db_->Get(rocksdb::ReadOptions(), sub_key, &value_bytes); if (!s.ok() && !s.IsNotFound()) return s; if (s.ok()) { - try { - old_value = std::stoll(value_bytes, &idx); - } catch (std::exception &e) { - return rocksdb::Status::InvalidArgument(e.what()); + auto parse_result = ParseInt(value_bytes, 10); + if (!parse_result) { + return rocksdb::Status::InvalidArgument(parse_result.Msg()); } - if (isspace(value_bytes[0]) || idx != value_bytes.size()) { + if (isspace(value_bytes[0])) { return rocksdb::Status::InvalidArgument("value is not an integer"); } + old_value = *parse_result; exists = true; } } diff --git a/src/redis_request.cc b/src/redis_request.cc index 88a18c42a38..47ff5095c75 100644 --- a/src/redis_request.cc +++ b/src/redis_request.cc @@ -32,6 +32,7 @@ #include "server.h" #include "redis_slot.h" #include "event_util.h" +#include "parse_util.h" namespace Redis { const size_t PROTO_INLINE_MAX_SIZE = 16 * 1024L; @@ -66,11 +67,11 @@ Status Request::Tokenize(evbuffer *input) { pipeline_size++; svr_->stats_.IncrInbondBytes(line.length); if (line[0] == '*') { - try { - multi_bulk_len_ = std::stoll(std::string(line.get() + 1, line.length - 1)); - } catch (std::exception &e) { + auto parse_result = ParseInt(std::string(line.get() + 1, line.length - 1), 10); + if (!parse_result) { return Status(Status::NotOK, "Protocol error: invalid multibulk length"); } + multi_bulk_len_ = *parse_result; if (isOnlyLF || multi_bulk_len_ > (int64_t)PROTO_MULTI_MAX_SIZE) { return Status(Status::NotOK, "Protocol error: invalid multibulk length"); } diff --git a/src/redis_string.cc b/src/redis_string.cc index 67e5e0d88b0..5c882687be9 100644 --- a/src/redis_string.cc +++ b/src/redis_string.cc @@ -23,6 +23,7 @@ #include #include #include +#include "parse_util.h" namespace Redis { @@ -274,16 +275,15 @@ rocksdb::Status String::IncrBy(const std::string &user_key, int64_t increment, i value = raw_value.substr(STRING_HDR_SIZE, raw_value.size()-STRING_HDR_SIZE); int64_t n = 0; - std::size_t idx = 0; if (!value.empty()) { - try { - n = std::stoll(value, &idx); - } catch(std::exception &e) { + auto parse_result = ParseInt(value, 10); + if (!parse_result) { return rocksdb::Status::InvalidArgument("value is not an integer or out of range"); } - if (isspace(value[0]) || idx != value.size()) { + if (isspace(value[0])) { return rocksdb::Status::InvalidArgument("value is not an integer"); } + n = *parse_result; } if ((increment < 0 && n <= 0 && increment < (LLONG_MIN-n)) || (increment > 0 && n >= 0 && increment > (LLONG_MAX-n))) { diff --git a/src/util.cc b/src/util.cc index 31944f027f0..7c80ab9826c 100644 --- a/src/util.cc +++ b/src/util.cc @@ -44,6 +44,7 @@ #include "util.h" #include "status.h" #include "event_util.h" +#include "parse_util.h" #ifndef POLLIN # define POLLIN 0x0001 /* There is data to read */ @@ -341,26 +342,20 @@ int GetLocalPort(int fd) { } Status DecimalStringToNum(const std::string &str, int64_t *n, int64_t min, int64_t max) { - try { - *n = static_cast(std::stoll(str)); - if (max > min && (*n < min || *n > max)) { - return Status(Status::NotOK, "value should between "+std::to_string(min)+" and "+std::to_string(max)); - } - } catch (std::exception &e) { - return Status(Status::NotOK, "value is not an integer or out of range"); + auto parse_result = ParseInt(str, NumericRange{min, max}, 10); + if (!parse_result) { + return parse_result.ToStatus(); } + *n = *parse_result; return Status::OK(); } Status OctalStringToNum(const std::string &str, int64_t *n, int64_t min, int64_t max) { - try { - *n = static_cast(std::stoll(str, nullptr, 8)); - if (max > min && (*n < min || *n > max)) { - return Status(Status::NotOK, "value should between "+std::to_string(min)+" and "+std::to_string(max)); - } - } catch (std::exception &e) { - return Status(Status::NotOK, "value is not an integer or out of range"); + auto parse_result = ParseInt(str, NumericRange{min, max}, 8); + if (!parse_result) { + return parse_result.ToStatus(); } + *n = *parse_result; return Status::OK(); } diff --git a/tests/cppunit/t_hash_test.cc b/tests/cppunit/t_hash_test.cc index 27f612bbb78..9ff9b9fffd1 100644 --- a/tests/cppunit/t_hash_test.cc +++ b/tests/cppunit/t_hash_test.cc @@ -26,6 +26,7 @@ #include "test_base.h" #include "redis_hash.h" +#include "parse_util.h" class RedisHashTest : public TestBase { protected: explicit RedisHashTest() : TestBase() { @@ -117,8 +118,11 @@ TEST_F(RedisHashTest, HIncr) { } std::string bytes; hash->Get(key_, field, &bytes); - value = std::stoll(bytes); - EXPECT_EQ(32, value); + auto parseResult = ParseInt(bytes, 10); + if (!parseResult) { + FAIL(); + } + EXPECT_EQ(32, *parseResult); hash->Del(key_); } diff --git a/tests/tcl/tests/unit/type/hash.tcl b/tests/tcl/tests/unit/type/hash.tcl index e34ece147b7..0352eaeb3ec 100644 --- a/tests/tcl/tests/unit/type/hash.tcl +++ b/tests/tcl/tests/unit/type/hash.tcl @@ -335,8 +335,8 @@ start_server {tags {"hash"}} { catch {r hincrby smallhash str 1} smallerr catch {r hincrby smallhash str 1} bigerr set rv {} - lappend rv [string match "ERR*not an integer*" $smallerr] - lappend rv [string match "ERR*not an integer*" $bigerr] + lappend rv [string match "ERR*non-integer*" $smallerr] + lappend rv [string match "ERR*non-integer*" $bigerr] } {1 1} test {HINCRBY can detect overflows} { From 9b6cc11e4a26c106b906e62f8e968ac682033ba2 Mon Sep 17 00:00:00 2001 From: hulk Date: Mon, 26 Sep 2022 17:17:31 +0800 Subject: [PATCH 15/22] Fix many typos in cluster.cc (#919) --- src/cluster.cc | 82 +++++++++++++------------ tests/cppunit/cluster_test.cc | 10 +-- tests/cppunit/rwlock_test.cc | 2 +- tests/tcl/tests/integration/cluster.tcl | 2 +- 4 files changed, 51 insertions(+), 45 deletions(-) diff --git a/src/cluster.cc b/src/cluster.cc index e6200a1e9ce..5f920d0c3d7 100644 --- a/src/cluster.cc +++ b/src/cluster.cc @@ -29,6 +29,16 @@ #include "redis_cmd.h" #include "replication.h" +const char *errInvalidNodeID = "Invalid cluster node id"; +const char *errInvalidSlotID = "Invalid slot id"; +const char *errSlotOutOfRange = "Slot is out of range"; +const char *errInvalidClusterVersion = "Invalid cluster version"; +const char *errSlotOverlapped = "Slot distribution is overlapped"; +const char *errNoMasterNode = "The node isn't a master"; +const char *errClusterNoInitialized = "CLUSTERDOWN The cluster is not initialized"; +const char *errInvalidClusterNodeInfo = "Invalid cluster nodes info"; +const char *errInvalidImportState = "Invalid import state"; + ClusterNode::ClusterNode(std::string id, std::string host, int port, int role, std::string master_id, std::bitset slots): id_(id), host_(host), port_(port), role_(role), @@ -42,7 +52,7 @@ Cluster::Cluster(Server *svr, std::vector binds, int port) : } // We access cluster without lock, actually we guarantee data-safe by work threads -// ReadWriteLockGuard, CLUSTER command doesn't have 'execlusive' attribute, i.e. +// ReadWriteLockGuard, CLUSTER command doesn't have 'exclusive' attribute, i.e. // CLUSTER command can be executed concurrently, but some subcommand may change // cluster data, so these commands should be executed exclusively, and ReadWriteLock // also can guarantee accessing data is safe. @@ -61,7 +71,7 @@ bool Cluster::SubCommandIsExecExclusive(const std::string &subcommand) { Status Cluster::SetNodeId(std::string node_id) { if (node_id.size() != kClusterNodeIdLen) { - return Status(Status::ClusterInvalidInfo, "Invalid node id"); + return Status(Status::ClusterInvalidInfo, errInvalidNodeID); } myid_ = node_id; @@ -92,13 +102,13 @@ Status Cluster::SetNodeId(std::string node_id) { Status Cluster::SetSlot(int slot, std::string node_id, int64_t new_version) { // Parameters check if (new_version <= 0 || new_version != version_ + 1) { - return Status(Status::NotOK, "Invalid cluster version"); + return Status(Status::NotOK, errInvalidClusterVersion); } if (!IsValidSlot(slot)) { - return Status(Status::NotOK, "Invalid slot id"); + return Status(Status::NotOK, errInvalidSlotID); } if (node_id.size() != kClusterNodeIdLen) { - return Status(Status::NotOK, "Invalid node id"); + return Status(Status::NotOK, errInvalidNodeID); } // Get the node which we want to assign a slot into it @@ -107,7 +117,7 @@ Status Cluster::SetSlot(int slot, std::string node_id, int64_t new_version) { return Status(Status::NotOK, "No this node in the cluster"); } if (to_assign_node->role_ != kClusterMaster) { - return Status(Status::NotOK, "The node is not the master"); + return Status(Status::NotOK, errNoMasterNode); } // Update version @@ -143,12 +153,12 @@ Status Cluster::SetSlot(int slot, std::string node_id, int64_t new_version) { // cluster setnodes $all_nodes_info $version $force // one line of $all_nodes: $node_id $host $port $role $master_node_id $slot_range Status Cluster::SetClusterNodes(const std::string &nodes_str, int64_t version, bool force) { - if (version < 0) return Status(Status::NotOK, "Invalid version"); + if (version < 0) return Status(Status::NotOK, errInvalidClusterVersion); if (force == false) { // Low version wants to reset current version if (version_ > version) { - return Status(Status::NotOK, "Invalid version of cluster"); + return Status(Status::NotOK, errInvalidClusterVersion); } // The same version, it is not needed to update if (version_ == version) return Status::OK(); @@ -245,7 +255,7 @@ bool Cluster::IsNotMaster() { Status Cluster::SetSlotMigrated(int slot, const std::string &ip_port) { if (!IsValidSlot(slot)) { - return Status(Status::NotOK, "Slot is out of range"); + return Status(Status::NotOK, errSlotOutOfRange); } // It is called by slot-migrating thread which is an asynchronous thread. // Therefore, it should be locked when a record is added to 'migrated_slots_' @@ -257,7 +267,7 @@ Status Cluster::SetSlotMigrated(int slot, const std::string &ip_port) { Status Cluster::SetSlotImported(int slot) { if (!IsValidSlot(slot)) { - return Status(Status::NotOK, "Slot is out of range"); + return Status(Status::NotOK, errSlotOutOfRange); } // It is called by command 'cluster import'. When executing the command, the // exclusive lock has been locked. Therefore, it can't be locked again. @@ -270,7 +280,7 @@ Status Cluster::MigrateSlot(int slot, const std::string &dst_node_id) { return Status(Status::NotOK, "Can't find the destination node id"); } if (!IsValidSlot(slot)) { - return Status(Status::NotOK, "Slot is out of range"); + return Status(Status::NotOK, errSlotOutOfRange); } if (slots_nodes_[slot] != myself_) { return Status(Status::NotOK, "Can't migrate slot which doesn't belong to me"); @@ -299,7 +309,7 @@ Status Cluster::ImportSlot(Redis::Connection *conn, int slot, int state) { return Status(Status::NotOK, "Slave can't import slot"); } if (!IsValidSlot(slot)) { - return Status(Status::NotOK, "Slot is out of range"); + return Status(Status::NotOK, errSlotOutOfRange); } switch (state) { @@ -312,14 +322,14 @@ Status Cluster::ImportSlot(Redis::Connection *conn, int slot, int state) { myself_->importing_slot_ = slot; // Set link error callback conn->close_cb_ = std::bind(&SlotImport::StopForLinkError, svr_->slot_import_, conn->GetFD()); - // Stop forbiding writing slot to accept write commands + // Stop forbidding writing slot to accept write commands if (slot == svr_->slot_migrate_->GetForbiddenSlot()) svr_->slot_migrate_->ReleaseForbiddenSlot(); LOG(INFO) << "[import] Start importing slot " << slot; break; case kImportSuccess: if (!svr_->slot_import_->Success(slot)) { LOG(ERROR) << "[import] Failed to set slot importing success, maybe slot is wrong" - << ", recieved slot: " << slot + << ", received slot: " << slot << ", current slot: " << svr_->slot_import_->GetSlot(); return Status(Status::NotOK, "Failed to set slot " + std::to_string(slot) + " importing success"); } @@ -328,23 +338,21 @@ Status Cluster::ImportSlot(Redis::Connection *conn, int slot, int state) { case kImportFailed: if (!svr_->slot_import_->Fail(slot)) { LOG(ERROR) << "[import] Failed to set slot importing error, maybe slot is wrong" - << ", recieved slot: " << slot + << ", received slot: " << slot << ", current slot: " << svr_->slot_import_->GetSlot(); return Status(Status::NotOK, "Failed to set slot " + std::to_string(slot) + " importing error"); } LOG(INFO) << "[import] Failed to import slot " << slot; break; default: - return Status(Status::NotOK, "Invalid import state"); - break; + return Status(Status::NotOK, errInvalidImportState); } return Status::OK(); } Status Cluster::GetClusterInfo(std::string *cluster_infos) { if (version_ < 0) { - return Status(Status::ClusterDown, - "CLUSTERDOWN The cluster is not initialized"); + return Status(Status::ClusterDown, errClusterNoInitialized); } cluster_infos->clear(); @@ -390,8 +398,7 @@ Status Cluster::GetClusterInfo(std::string *cluster_infos) { // ... continued until done Status Cluster::GetSlotsInfo(std::vector *slots_infos) { if (version_ < 0) { - return Status(Status::ClusterDown, - "CLUSTERDOWN The cluster is not initialized"); + return Status(Status::ClusterDown, errClusterNoInitialized); } slots_infos->clear(); @@ -418,7 +425,7 @@ Status Cluster::GetSlotsInfo(std::vector *slots_infos) { SlotInfo Cluster::GenSlotNodeInfo(int start, int end, std::shared_ptr n) { std::vector vn; - vn.push_back({n->host_, n->port_, n->id_}); // Itslef + vn.push_back({n->host_, n->port_, n->id_}); // itself for (const auto &id : n->replicas) { // replicas if (nodes_.find(id) == nodes_.end()) continue; @@ -427,12 +434,11 @@ SlotInfo Cluster::GenSlotNodeInfo(int start, int end, std::shared_ptr *slots_nodes) { std::vector nodes_info = Util::Split(nodes_str, "\n"); if (nodes_info.size() == 0) { - return Status(Status::ClusterInvalidInfo, "Invalid cluster nodes info"); + return Status(Status::ClusterInvalidInfo, errInvalidClusterNodeInfo); } nodes->clear(); @@ -511,12 +517,12 @@ Status Cluster::ParseClusterNodes(const std::string &nodes_str, ClusterNodes *no for (const auto &node_str : nodes_info) { std::vector fields = Util::Split(node_str, " "); if (fields.size() < 5) { - return Status(Status::ClusterInvalidInfo, "Invalid cluster nodes info"); + return Status(Status::ClusterInvalidInfo, errInvalidClusterNodeInfo); } // 1) node id if (fields[0].size() != kClusterNodeIdLen) { - return Status(Status::ClusterInvalidInfo, "Invalid cluster node id"); + return Status(Status::ClusterInvalidInfo, errInvalidNodeID); } std::string id = fields[0]; @@ -526,7 +532,7 @@ Status Cluster::ParseClusterNodes(const std::string &nodes_str, ClusterNodes *no // 3) port int port = std::atoi(fields[2].c_str()); if (port <= 0 || port >= (65535-kClusterPortIncr)) { - return Status(Status::ClusterInvalidInfo, "Invalid cluste node port"); + return Status(Status::ClusterInvalidInfo, "Invalid cluster node port"); } // 4) role @@ -537,20 +543,20 @@ Status Cluster::ParseClusterNodes(const std::string &nodes_str, ClusterNodes *no strcasecmp(fields[3].c_str(), "replica") == 0) { role = kClusterSlave; } else { - return Status(Status::ClusterInvalidInfo, "Invalid cluste node role"); + return Status(Status::ClusterInvalidInfo, "Invalid cluster node role"); } // 5) master id std::string master_id = fields[4]; if ((role == kClusterMaster && master_id != "-") || (role == kClusterSlave && master_id.size() != kClusterNodeIdLen)) { - return Status(Status::ClusterInvalidInfo, "Invalid cluste node master id"); + return Status(Status::ClusterInvalidInfo, errInvalidNodeID); } std::bitset slots; if (role == kClusterSlave) { if (fields.size() != 5) { - return Status(Status::ClusterInvalidInfo, "Invalid cluster nodes info"); + return Status(Status::ClusterInvalidInfo, errInvalidClusterNodeInfo); } else { // Create slave node (*nodes)[id] = Util::MakeShared( @@ -566,12 +572,12 @@ Status Cluster::ParseClusterNodes(const std::string &nodes_str, ClusterNodes *no if (ranges.size() == 1) { start = std::atoi(ranges[0].c_str()); if (IsValidSlot(start) == false) { - return Status(Status::ClusterInvalidInfo, "Invalid cluste slot range"); + return Status(Status::ClusterInvalidInfo, errSlotOutOfRange); } slots.set(start, 1); if (role == kClusterMaster) { if (slots_nodes->find(start) != slots_nodes->end()) { - return Status(Status::ClusterInvalidInfo, "Slot distribution is overlapped"); + return Status(Status::ClusterInvalidInfo, errSlotOverlapped); } else { (*slots_nodes)[start] = id; } @@ -580,20 +586,20 @@ Status Cluster::ParseClusterNodes(const std::string &nodes_str, ClusterNodes *no start = std::atoi(ranges[0].c_str()); stop = std::atoi(ranges[1].c_str()); if (start >= stop || start < 0 || stop >= kClusterSlots) { - return Status(Status::ClusterInvalidInfo, "Invalid cluste slot range"); + return Status(Status::ClusterInvalidInfo, errSlotOutOfRange); } for (int j = start; j <= stop; j++) { slots.set(j, 1); if (role == kClusterMaster) { if (slots_nodes->find(j) != slots_nodes->end()) { - return Status(Status::ClusterInvalidInfo, "Slot distribution is overlapped"); + return Status(Status::ClusterInvalidInfo, errSlotOverlapped); } else { (*slots_nodes)[j] = id; } } } } else { - return Status(Status::ClusterInvalidInfo, "Invalid cluste slot range"); + return Status(Status::ClusterInvalidInfo, errSlotOutOfRange); } } @@ -627,7 +633,7 @@ Status Cluster::CanExecByMySelf(const Redis::CommandAttributes *attributes, if (slot == -1) slot = cur_slot; if (slot != cur_slot) { return Status(Status::RedisExecErr, - "CROSSSLOT Keys in request don't hash to the same slot"); + "CROSSSLOT Attempted to access keys that don't hash to the same slot"); } } if (slot == -1) return Status::OK(); diff --git a/tests/cppunit/cluster_test.cc b/tests/cppunit/cluster_test.cc index 541d976972a..3530c45e6fd 100644 --- a/tests/cppunit/cluster_test.cc +++ b/tests/cppunit/cluster_test.cc @@ -49,35 +49,35 @@ TEST(Cluster, CluseterSetNodes) { "master 07c37dfeb235213a872192d90877d0cd55635b91 5461-10922"; s = cluster.SetClusterNodes(invalid_port, 1, false); ASSERT_FALSE(s.IsOK()); - ASSERT_TRUE(s.Msg() == "Invalid cluste node port"); + ASSERT_TRUE(s.Msg() == "Invalid cluster node port"); const std::string slave_has_no_master = "07c37dfeb235213a872192d90877d0cd55635b91 127.0.0.1 30004 " "slave -"; s = cluster.SetClusterNodes(slave_has_no_master, 1, false); ASSERT_FALSE(s.IsOK()); - ASSERT_TRUE(s.Msg() == "Invalid cluste node master id"); + ASSERT_TRUE(s.Msg() == "Invalid cluster node id"); const std::string master_has_master = "67ed2db8d677e59ec4a4cefb06858cf2a1a89fa1 127.0.0.1 30002 " "master 07c37dfeb235213a872192d90877d0cd55635b91 5461-10922"; s = cluster.SetClusterNodes(master_has_master, 1, false); ASSERT_FALSE(s.IsOK()); - ASSERT_TRUE(s.Msg() == "Invalid cluste node master id"); + ASSERT_TRUE(s.Msg() == "Invalid cluster node id"); const std::string invalid_slot_range = "67ed2db8d677e59ec4a4cefb06858cf2a1a89fa1 127.0.0.1 30002 " "master - 5461-0"; s = cluster.SetClusterNodes(invalid_slot_range, 1, false); ASSERT_FALSE(s.IsOK()); - ASSERT_TRUE(s.Msg() == "Invalid cluste slot range"); + ASSERT_TRUE(s.Msg() == "Slot is out of range"); const std::string invalid_slot_id = "67ed2db8d677e59ec4a4cefb06858cf2a1a89fa1 127.0.0.1 30002 " "master - 54610"; s = cluster.SetClusterNodes(invalid_slot_id, 1, false); ASSERT_FALSE(s.IsOK()); - ASSERT_TRUE(s.Msg() == "Invalid cluste slot range"); + ASSERT_TRUE(s.Msg() == "Slot is out of range"); const std::string overlapped_slot_id = "67ed2db8d677e59ec4a4cefb06858cf2a1a89fa1 127.0.0.1 30002 " diff --git a/tests/cppunit/rwlock_test.cc b/tests/cppunit/rwlock_test.cc index 4c6ef90ce1c..30a71838585 100644 --- a/tests/cppunit/rwlock_test.cc +++ b/tests/cppunit/rwlock_test.cc @@ -167,7 +167,7 @@ TEST(ReadWriteLock, WriteLockGurad_First) { }); } else { ths[i] = std::thread([&rwlock, &val]() { - usleep(1000); // To avoid it is the first to run, just sleep 100ms + usleep(100000); // To avoid it is the first to run, just sleep 100ms auto ptr = std::unique_ptr(new RWLock::ReadLock(rwlock)); ASSERT_EQ(val, 3); }); diff --git a/tests/tcl/tests/integration/cluster.tcl b/tests/tcl/tests/integration/cluster.tcl index 5b3f63970f3..2b2d31a64e7 100644 --- a/tests/tcl/tests/integration/cluster.tcl +++ b/tests/tcl/tests/integration/cluster.tcl @@ -89,7 +89,7 @@ start_server {tags {"cluster"} overrides {cluster-enabled yes}} { assert_match "*CLUSTER*" $err catch {[r clusterx setnodes a -1]} err - assert_match "*Invalid version*" $err + assert_match "*Invalid cluster version*" $err catch {[r clusterx setslot 16384 07c37dfeb235213a872192d90877d0cd55635b91 1]} err assert_match "*CLUSTER*" $err From f02029abbe440ac508c5b7361259bf5cfd1d613b Mon Sep 17 00:00:00 2001 From: ellutionist Date: Tue, 20 Sep 2022 16:06:07 +0800 Subject: [PATCH 16/22] fix: non positive ttl in slot migration --- src/slot_migrate.cc | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/slot_migrate.cc b/src/slot_migrate.cc index a65a3cbb479..f8a31716dcf 100644 --- a/src/slot_migrate.cc +++ b/src/slot_migrate.cc @@ -554,6 +554,12 @@ Status SlotMigrate::MigrateOneKey(const rocksdb::Slice &key, const rocksdb::Slic return Status(Status::cOK, "empty"); } + int64_t now; + rocksdb::Env::Default()->GetCurrentTime(&now); + if (metadata.expire <= now) { + return Status(Status::cOK, "expired"); + } + // Construct command according to type of the key switch (metadata.Type()) { case kRedisString: { From 5dfee330e668d46580d9a8f7b6152f026a131807 Mon Sep 17 00:00:00 2001 From: ellutionist Date: Tue, 20 Sep 2022 18:59:02 +0800 Subject: [PATCH 17/22] check meta.expire greater than zero --- src/slot_migrate.cc | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/slot_migrate.cc b/src/slot_migrate.cc index f8a31716dcf..ce6f816c603 100644 --- a/src/slot_migrate.cc +++ b/src/slot_migrate.cc @@ -554,10 +554,12 @@ Status SlotMigrate::MigrateOneKey(const rocksdb::Slice &key, const rocksdb::Slic return Status(Status::cOK, "empty"); } - int64_t now; - rocksdb::Env::Default()->GetCurrentTime(&now); - if (metadata.expire <= now) { - return Status(Status::cOK, "expired"); + if (metadata.expire > 0) { + int64_t now; + rocksdb::Env::Default()->GetCurrentTime(&now); + if (metadata.expire <= now) { + return Status(Status::cOK, "expired"); + } } // Construct command according to type of the key From 469fd84426b1210d867c153cc78296ba9821d4aa Mon Sep 17 00:00:00 2001 From: ellutionist Date: Thu, 22 Sep 2022 22:54:29 +0800 Subject: [PATCH 18/22] implement "set exat|pxat" && employ in migration --- src/redis_cmd.cc | 43 +++++++++++++++++++++++++++++++++++++++++++ src/slot_migrate.cc | 15 ++++----------- 2 files changed, 47 insertions(+), 11 deletions(-) diff --git a/src/redis_cmd.cc b/src/redis_cmd.cc index 9e6d1771f8d..a0954a4f9ae 100644 --- a/src/redis_cmd.cc +++ b/src/redis_cmd.cc @@ -488,6 +488,36 @@ class CommandSet : public Commander { return Status(Status::RedisParseErr, errValueNotInterger); } if (ttl_ <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); + } else if (opt == "exat" && !ttl_ && !expire_ && !last_arg) { + try { + std::string s = args_[++i]; + std::string::size_type sz; + expire_ = std::stol(s, &sz); + if (sz != s.size()) { + return Status(Status::RedisParseErr, errValueNotInterger); + } + } catch (std::exception &e) { + return Status(Status::RedisParseErr, errValueNotInterger); + } + if (expire_ <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); + } else if (opt == "pxat" && !ttl_ && !expire_ && !last_arg) { + uint64_t expire_ms = 0; + try { + std::string s = args_[++i]; + std::string::size_type sz; + expire_ms = std::stoul(s, &sz); + if (sz != s.size()) { + return Status(Status::RedisParseErr, errValueNotInterger); + } + } catch (std::exception &e) { + return Status(Status::RedisParseErr, errValueNotInterger); + } + if (expire_ms <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); + if (expire_ms < 1000) { + expire_ = 1; + } else { + expire_ = static_cast(expire_ms/1000); + } } else if (opt == "px" && !ttl_ && !last_arg) { int64_t ttl_ms = 0; std::string s = args_[++i]; @@ -512,6 +542,18 @@ class CommandSet : public Commander { int ret; Redis::String string_db(svr->storage_, conn->GetNamespace()); rocksdb::Status s; + + if (!ttl_ && expire_) { + int64_t now; + rocksdb::Env::Default()->GetCurrentTime(&now); + ttl_ = expire_ - now; + if (ttl_ <= 0) { + string_db.Del(args_[1]); + *output = Redis::SimpleString("OK"); + return Status::OK(); + } + } + if (nx_) { s = string_db.SetNX(args_[1], args_[2], ttl_, &ret); } else if (xx_) { @@ -534,6 +576,7 @@ class CommandSet : public Commander { bool xx_ = false; bool nx_ = false; int ttl_ = 0; + int64_t expire_ = 0; }; class CommandSetEX : public Commander { diff --git a/src/slot_migrate.cc b/src/slot_migrate.cc index ce6f816c603..c52571079e4 100644 --- a/src/slot_migrate.cc +++ b/src/slot_migrate.cc @@ -554,12 +554,8 @@ Status SlotMigrate::MigrateOneKey(const rocksdb::Slice &key, const rocksdb::Slic return Status(Status::cOK, "empty"); } - if (metadata.expire > 0) { - int64_t now; - rocksdb::Env::Default()->GetCurrentTime(&now); - if (metadata.expire <= now) { - return Status(Status::cOK, "expired"); - } + if (metadata.Expired()) { + return Status(Status::cOK, "expired"); } // Construct command according to type of the key @@ -595,11 +591,8 @@ bool SlotMigrate::MigrateSimpleKey(const rocksdb::Slice &key, const Metadata &me const std::string &bytes, std::string *restore_cmds) { std::vector command = {"set", key.ToString(), bytes.substr(5, bytes.size() - 5)}; if (metadata.expire > 0) { - int64_t now; - rocksdb::Env::Default()->GetCurrentTime(&now); - int32_t ttl = metadata.expire - uint32_t(now); - command.emplace_back("EX"); - command.emplace_back(std::to_string(ttl)); + command.emplace_back("EXAT"); + command.emplace_back(std::to_string(metadata.expire)); } *restore_cmds += Redis::MultiBulkString(command, false); current_pipeline_size_++; From 33121ef38865e879dd0e9daa4d4901b932c6a72e Mon Sep 17 00:00:00 2001 From: ellutionist Date: Fri, 23 Sep 2022 11:37:34 +0800 Subject: [PATCH 19/22] test: cases for EXAT|PXAT && slot migrate --- tests/tcl/tests/integration/slotmigrate.tcl | 8 +++ tests/tcl/tests/unit/type/string.tcl | 63 +++++++++++++++++---- 2 files changed, 59 insertions(+), 12 deletions(-) diff --git a/tests/tcl/tests/integration/slotmigrate.tcl b/tests/tcl/tests/integration/slotmigrate.tcl index aa949447abf..ff662779588 100644 --- a/tests/tcl/tests/integration/slotmigrate.tcl +++ b/tests/tcl/tests/integration/slotmigrate.tcl @@ -147,6 +147,7 @@ start_server {tags {"Src migration server"} overrides {cluster-enabled yes}} { # Set keys set slot1_tag [lindex $::CRC16_SLOT_TABLE 1] set slot1_key_string string_{$slot1_tag} + set slot1_key_string2 string2_{$slot1_tag} set slot1_key_list list_{$slot1_tag} set slot1_key_hash hash_{$slot1_tag} set slot1_key_set set_{$slot1_tag} @@ -168,6 +169,11 @@ start_server {tags {"Src migration server"} overrides {cluster-enabled yes}} { $r0 set $slot1_key_string $slot1_key_string $r0 expire $slot1_key_string 10000 + # Expired string key + $r0 set $slot1_key_string2 $slot1_key_string2 ex 1 + after 3000 + assert_equal [$r0 get $slot1_key_string2] {} + # Type: list $r0 rpush $slot1_key_list 0 1 2 3 4 5 $r0 lpush $slot1_key_list 9 3 7 3 5 4 @@ -242,6 +248,8 @@ start_server {tags {"Src migration server"} overrides {cluster-enabled yes}} { set expire_time [$r1 ttl $slot1_key_string] assert {$expire_time > 1000 && $expire_time <= 10000} + assert_equal [$r1 get $slot1_key_string2] {} + # Check list and expired time assert {[$r1 lrange $slot1_key_list 0 -1] eq $lvalue} set expire_time [$r1 ttl $slot1_key_list] diff --git a/tests/tcl/tests/unit/type/string.tcl b/tests/tcl/tests/unit/type/string.tcl index 7db12afd316..4f51e0b34ae 100644 --- a/tests/tcl/tests/unit/type/string.tcl +++ b/tests/tcl/tests/unit/type/string.tcl @@ -556,17 +556,41 @@ start_server {tags {"string"}} { assert {$ttl <= 10 && $ttl > 5} } - # test "Extended SET EXAT option" { - # r del foo - # r set foo bar exat [expr [clock seconds] + 10] - # assert_range [r ttl foo] 5 10 - # } + test {Extended SET EXAT option} { + r del foo + r set foo bar exat [expr [clock seconds] + 10] + assert_range [r ttl foo] 5 10 + } - # test "Extended SET PXAT option" { - # r del foo - # r set foo bar pxat [expr [clock milliseconds] + 10000] - # assert_range [r ttl foo] 5 10 - # } + test {Extended SET EXAT option with expired timestamp} { + r del foo + assert_equal [r set foo bar exat 1] OK + assert_equal [r get foo] {} + + r set foo bar + assert_equal [r get foo] bar + + assert_equal [r set foo bar exat [expr [clock seconds] - 5]] OK + assert_equal [r get foo] {} + } + + test {Extended SET PXAT option} { + r del foo + r set foo bar pxat [expr [clock milliseconds] + 10000] + assert_range [r ttl foo] 5 10 + } + + test {Extended SET PXAT option with expired timestamp} { + r del foo + assert_equal [r set foo bar pxat 1] OK + assert_equal [r get foo] {} + + r set foo bar + assert_equal [r get foo] bar + + assert_equal [r set foo bar pxat [expr [clock milliseconds] - 5000]] OK + assert_equal [r get foo] {} + } test {Extended SET with incorrect use of multi options should result in syntax err} { catch {r set foo bar ex 10 px 10000} err1 @@ -576,8 +600,23 @@ start_server {tags {"string"}} { test {Extended SET with incorrect expire value} { catch {r set foo bar ex 1234xyz} e - set e - } {*not an integer*} + assert_match {*not an integer*} $e + + catch {r set foo bar ex 0} e + assert_match {*invalid expire time*} $e + + catch {r set foo bar exat 1234xyz} e + assert_match {*not an integer*} $e + + catch {r set foo bar exat 0} e + assert_match {*invalid expire time*} $e + + catch {r set foo bar pxat 1234xyz} e + assert_match {*not an integer*} $e + + catch {r set foo bar pxat 0} e + assert_match {*invalid expire time*} $e + } test {Extended SET using multiple options at once} { r set foo val From 8398e7916a7635a43928fbdd7769836db2999c8c Mon Sep 17 00:00:00 2001 From: ellutionist Date: Mon, 26 Sep 2022 20:34:56 +0800 Subject: [PATCH 20/22] use ParseInt --- src/redis_cmd.cc | 37 ++++++++++++------------------------- 1 file changed, 12 insertions(+), 25 deletions(-) diff --git a/src/redis_cmd.cc b/src/redis_cmd.cc index a0954a4f9ae..546ebb26d30 100644 --- a/src/redis_cmd.cc +++ b/src/redis_cmd.cc @@ -477,41 +477,28 @@ class CommandSet : public Commander { } else if (opt == "xx" && !nx_) { xx_ = true; } else if (opt == "ex" && !ttl_ && !last_arg) { - try { - std::string s = args_[++i]; - std::string::size_type sz; - ttl_ = std::stoi(s, &sz); - if (sz != s.size()) { - return Status(Status::RedisParseErr, errValueNotInterger); - } - } catch (std::exception &e) { + std::string s = args_[++i]; + StatusOr parse_status = ParseInt(s); + if (!parse_status.IsOK()) { return Status(Status::RedisParseErr, errValueNotInterger); } + ttl_ = parse_status.GetValue(); if (ttl_ <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); } else if (opt == "exat" && !ttl_ && !expire_ && !last_arg) { - try { - std::string s = args_[++i]; - std::string::size_type sz; - expire_ = std::stol(s, &sz); - if (sz != s.size()) { - return Status(Status::RedisParseErr, errValueNotInterger); - } - } catch (std::exception &e) { + std::string s = args_[++i]; + StatusOr parse_status = ParseInt(s); + if (!parse_status.IsOK()) { return Status(Status::RedisParseErr, errValueNotInterger); } + expire_ = parse_status.GetValue(); if (expire_ <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); } else if (opt == "pxat" && !ttl_ && !expire_ && !last_arg) { - uint64_t expire_ms = 0; - try { - std::string s = args_[++i]; - std::string::size_type sz; - expire_ms = std::stoul(s, &sz); - if (sz != s.size()) { - return Status(Status::RedisParseErr, errValueNotInterger); - } - } catch (std::exception &e) { + std::string s = args_[++i]; + StatusOr parse_status = ParseInt(s); + if (!parse_status.IsOK()) { return Status(Status::RedisParseErr, errValueNotInterger); } + uint64_t expire_ms = parse_status.GetValue(); if (expire_ms <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); if (expire_ms < 1000) { expire_ = 1; From cb6d1d8960eced12ec22ffcb40f57f8a81a8a03c Mon Sep 17 00:00:00 2001 From: ellutionist Date: Mon, 26 Sep 2022 22:15:47 +0800 Subject: [PATCH 21/22] fix template misuse --- src/redis_cmd.cc | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/redis_cmd.cc b/src/redis_cmd.cc index 546ebb26d30..f95dee0d95a 100644 --- a/src/redis_cmd.cc +++ b/src/redis_cmd.cc @@ -478,7 +478,7 @@ class CommandSet : public Commander { xx_ = true; } else if (opt == "ex" && !ttl_ && !last_arg) { std::string s = args_[++i]; - StatusOr parse_status = ParseInt(s); + auto parse_status = ParseInt(s); if (!parse_status.IsOK()) { return Status(Status::RedisParseErr, errValueNotInterger); } @@ -486,7 +486,7 @@ class CommandSet : public Commander { if (ttl_ <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); } else if (opt == "exat" && !ttl_ && !expire_ && !last_arg) { std::string s = args_[++i]; - StatusOr parse_status = ParseInt(s); + auto parse_status = ParseInt(s); if (!parse_status.IsOK()) { return Status(Status::RedisParseErr, errValueNotInterger); } @@ -494,7 +494,7 @@ class CommandSet : public Commander { if (expire_ <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); } else if (opt == "pxat" && !ttl_ && !expire_ && !last_arg) { std::string s = args_[++i]; - StatusOr parse_status = ParseInt(s); + auto parse_status = ParseInt(s); if (!parse_status.IsOK()) { return Status(Status::RedisParseErr, errValueNotInterger); } From 73a27c3a547c807c180fb9e3cd8cd2dcc80eedb2 Mon Sep 17 00:00:00 2001 From: ellutionist Date: Tue, 27 Sep 2022 13:50:17 +0800 Subject: [PATCH 22/22] make style more concise --- src/redis_cmd.cc | 21 +++++++++------------ 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/src/redis_cmd.cc b/src/redis_cmd.cc index f95dee0d95a..febe5e9b823 100644 --- a/src/redis_cmd.cc +++ b/src/redis_cmd.cc @@ -477,28 +477,25 @@ class CommandSet : public Commander { } else if (opt == "xx" && !nx_) { xx_ = true; } else if (opt == "ex" && !ttl_ && !last_arg) { - std::string s = args_[++i]; - auto parse_status = ParseInt(s); - if (!parse_status.IsOK()) { + auto parse_result = ParseInt(args_[++i], 10); + if (!parse_result) { return Status(Status::RedisParseErr, errValueNotInterger); } - ttl_ = parse_status.GetValue(); + ttl_ = *parse_result; if (ttl_ <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); } else if (opt == "exat" && !ttl_ && !expire_ && !last_arg) { - std::string s = args_[++i]; - auto parse_status = ParseInt(s); - if (!parse_status.IsOK()) { + auto parse_result = ParseInt(args_[++i], 10); + if (!parse_result) { return Status(Status::RedisParseErr, errValueNotInterger); } - expire_ = parse_status.GetValue(); + expire_ = *parse_result; if (expire_ <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); } else if (opt == "pxat" && !ttl_ && !expire_ && !last_arg) { - std::string s = args_[++i]; - auto parse_status = ParseInt(s); - if (!parse_status.IsOK()) { + auto parse_result = ParseInt(args[++i], 10); + if (!parse_result) { return Status(Status::RedisParseErr, errValueNotInterger); } - uint64_t expire_ms = parse_status.GetValue(); + uint64_t expire_ms = *parse_result; if (expire_ms <= 0) return Status(Status::RedisParseErr, errInvalidExpireTime); if (expire_ms < 1000) { expire_ = 1;