You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kvrocks.apache.org by tw...@apache.org on 2023/01/17 03:42:29 UTC

[incubator-kvrocks] branch unstable updated: `HrangebyLex` supports specify intervals (#1120)

This is an automated email from the ASF dual-hosted git repository.

twice pushed a commit to branch unstable
in repository https://gitbox.apache.org/repos/asf/incubator-kvrocks.git


The following commit(s) were added to refs/heads/unstable by this push:
     new 003d8fb1 `HrangebyLex` supports specify intervals (#1120)
003d8fb1 is described below

commit 003d8fb19f84601e9da5e908b4a83092695eeaf8
Author: Ruixiang Tan <81...@qq.com>
AuthorDate: Tue Jan 17 11:42:24 2023 +0800

    `HrangebyLex` supports specify intervals (#1120)
---
 src/commands/redis_cmd.cc                |  53 +++++++++-------
 src/common/range_spec.cc                 |  53 ++++++++++++++++
 src/common/range_spec.h                  |  36 +++++++++++
 src/types/redis_hash.cc                  |  50 +++++++++++----
 src/types/redis_hash.h                   |   5 +-
 src/types/redis_zset.cc                  |  39 +-----------
 src/types/redis_zset.h                   |  15 +----
 tests/cppunit/t_hash_test.cc             | 105 +++++++++++++++++++++++++++++--
 tests/cppunit/t_zset_test.cc             |   2 +-
 tests/gocase/unit/type/hash/hash_test.go |  63 ++++++++++++-------
 10 files changed, 307 insertions(+), 114 deletions(-)

diff --git a/src/commands/redis_cmd.cc b/src/commands/redis_cmd.cc
index a66b13c9..dd1ee74b 100644
--- a/src/commands/redis_cmd.cc
+++ b/src/commands/redis_cmd.cc
@@ -1614,34 +1614,39 @@ class CommandHGetAll : public Commander {
   }
 };
 
-class CommandHRange : public Commander {
+class CommandHRangeByLex : public Commander {
  public:
   Status Parse(const std::vector<std::string> &args) override {
-    if (args.size() != 6 && args.size() != 4) {
-      return {Status::RedisParseErr, errWrongNumOfArguments};
+    CommandParser parser(args, 4);
+    while (parser.Good()) {
+      if (parser.EatEqICase("REV")) {
+        spec_.reversed = true;
+      } else if (parser.EatEqICase("LIMIT")) {
+        spec_.offset = GET_OR_RET(parser.TakeInt());
+        spec_.count = GET_OR_RET(parser.TakeInt());
+      } else {
+        return parser.InvalidSyntax();
+      }
     }
-
-    if (args.size() == 6 && Util::ToLower(args[4]) != "limit") {
-      return {Status::RedisInvalidCmd, errInvalidSyntax};
+    Status s;
+    if (spec_.reversed) {
+      s = ParseRangeLexSpec(args[3], args[2], &spec_);
+    } else {
+      s = ParseRangeLexSpec(args[2], args[3], &spec_);
     }
-
-    if (args.size() == 6) {
-      auto parse_result = ParseInt<int64_t>(args_[5], 10);
-      if (!parse_result) return {Status::RedisParseErr, errValueNotInteger};
-
-      limit_ = *parse_result;
+    if (!s.IsOK()) {
+      return Status(Status::RedisParseErr, s.Msg());
     }
-    return Commander::Parse(args);
+    return Status::OK();
   }
 
   Status Execute(Server *svr, Connection *conn, std::string *output) override {
     Redis::Hash hash_db(svr->storage_, conn->GetNamespace());
     std::vector<FieldValue> field_values;
-    auto s = hash_db.Range(args_[1], args_[2], args_[3], limit_, &field_values);
+    rocksdb::Status s = hash_db.RangeByLex(args_[1], spec_, &field_values);
     if (!s.ok()) {
       return {Status::RedisExecErr, s.ToString()};
     }
-
     std::vector<std::string> kv_pairs;
     for (const auto &p : field_values) {
       kv_pairs.emplace_back(p.field);
@@ -1653,7 +1658,7 @@ class CommandHRange : public Commander {
   }
 
  private:
-  int64_t limit_ = LONG_MAX;
+  CommonRangeLexSpec spec_;
 };
 
 class CommandPush : public Commander {
@@ -2674,7 +2679,7 @@ class CommandZIncrBy : public Commander {
 class CommandZLexCount : public Commander {
  public:
   Status Parse(const std::vector<std::string> &args) override {
-    Status s = Redis::ZSet::ParseRangeLexSpec(args[2], args[3], &spec_);
+    Status s = ParseRangeLexSpec(args[2], args[3], &spec_);
     if (!s.IsOK()) {
       return {Status::RedisParseErr, s.Msg()};
     }
@@ -2695,7 +2700,7 @@ class CommandZLexCount : public Commander {
   }
 
  private:
-  ZRangeLexSpec spec_;
+  CommonRangeLexSpec spec_;
 };
 
 class CommandZPop : public Commander {
@@ -2808,9 +2813,9 @@ class CommandZRangeByLex : public Commander {
   Status Parse(const std::vector<std::string> &args) override {
     Status s;
     if (spec_.reversed) {
-      s = Redis::ZSet::ParseRangeLexSpec(args[3], args[2], &spec_);
+      s = ParseRangeLexSpec(args[3], args[2], &spec_);
     } else {
-      s = Redis::ZSet::ParseRangeLexSpec(args[2], args[3], &spec_);
+      s = ParseRangeLexSpec(args[2], args[3], &spec_);
     }
 
     if (!s.IsOK()) {
@@ -2844,7 +2849,7 @@ class CommandZRangeByLex : public Commander {
   }
 
  private:
-  ZRangeLexSpec spec_;
+  CommonRangeLexSpec spec_;
 };
 
 class CommandZRangeByScore : public Commander {
@@ -3031,7 +3036,7 @@ class CommandZRemRangeByScore : public Commander {
 class CommandZRemRangeByLex : public Commander {
  public:
   Status Parse(const std::vector<std::string> &args) override {
-    Status s = Redis::ZSet::ParseRangeLexSpec(args[2], args[3], &spec_);
+    Status s = ParseRangeLexSpec(args[2], args[3], &spec_);
     if (!s.IsOK()) {
       return {Status::RedisParseErr, s.Msg()};
     }
@@ -3051,7 +3056,7 @@ class CommandZRemRangeByLex : public Commander {
   }
 
  private:
-  ZRangeLexSpec spec_;
+  CommonRangeLexSpec spec_;
 };
 
 class CommandZScore : public Commander {
@@ -6420,7 +6425,7 @@ REDIS_REGISTER_COMMANDS(
     MakeCmdAttr<CommandHVals>("hvals", 2, "read-only", 1, 1, 1),
     MakeCmdAttr<CommandHGetAll>("hgetall", 2, "read-only", 1, 1, 1),
     MakeCmdAttr<CommandHScan>("hscan", -3, "read-only", 1, 1, 1),
-    MakeCmdAttr<CommandHRange>("hrange", -4, "read-only", 1, 1, 1),
+    MakeCmdAttr<CommandHRangeByLex>("hrangebylex", -4, "read-only", 1, 1, 1),
 
     MakeCmdAttr<CommandLPush>("lpush", -3, "write", 1, 1, 1), MakeCmdAttr<CommandRPush>("rpush", -3, "write", 1, 1, 1),
     MakeCmdAttr<CommandLPushX>("lpushx", -3, "write", 1, 1, 1),
diff --git a/src/common/range_spec.cc b/src/common/range_spec.cc
new file mode 100644
index 00000000..82b695c6
--- /dev/null
+++ b/src/common/range_spec.cc
@@ -0,0 +1,53 @@
+/*
+ * 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 "range_spec.h"
+
+Status ParseRangeLexSpec(const std::string &min, const std::string &max, CommonRangeLexSpec *spec) {
+  if (min == "+" || max == "-") {
+    return Status(Status::NotOK, "min > max");
+  }
+
+  if (min == "-") {
+    spec->min = "";
+  } else {
+    if (min[0] == '(') {
+      spec->minex = true;
+    } else if (min[0] == '[') {
+      spec->minex = false;
+    } else {
+      return Status(Status::NotOK, "the min is illegal");
+    }
+    spec->min = min.substr(1);
+  }
+
+  if (max == "+") {
+    spec->max_infinite = true;
+  } else {
+    if (max[0] == '(') {
+      spec->maxex = true;
+    } else if (max[0] == '[') {
+      spec->maxex = false;
+    } else {
+      return Status(Status::NotOK, "the max is illegal");
+    }
+    spec->max = max.substr(1);
+  }
+  return Status::OK();
+}
diff --git a/src/common/range_spec.h b/src/common/range_spec.h
new file mode 100644
index 00000000..cc1a2868
--- /dev/null
+++ b/src/common/range_spec.h
@@ -0,0 +1,36 @@
+/*
+ * 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 <string>
+
+#include "status.h"
+struct CommonRangeLexSpec {
+  std::string min, max;
+  bool minex, maxex; /* are min or max exclusive */
+  bool max_infinite; /* are max infinite */
+  int64_t offset, count;
+  bool removed, reversed;
+  CommonRangeLexSpec()
+      : minex(false), maxex(false), max_infinite(false), offset(-1), count(-1), removed(false), reversed(false) {}
+};
+
+Status ParseRangeLexSpec(const std::string &min, const std::string &max, CommonRangeLexSpec *spec);
diff --git a/src/types/redis_hash.cc b/src/types/redis_hash.cc
index d7001b67..b2bf589b 100644
--- a/src/types/redis_hash.cc
+++ b/src/types/redis_hash.cc
@@ -276,10 +276,10 @@ rocksdb::Status Hash::MSet(const Slice &user_key, const std::vector<FieldValue>
   return storage_->Write(storage_->DefaultWriteOptions(), &batch);
 }
 
-rocksdb::Status Hash::Range(const Slice &user_key, const Slice &start, const Slice &stop, int64_t limit,
-                            std::vector<FieldValue> *field_values) {
+rocksdb::Status Hash::RangeByLex(const Slice &user_key, const CommonRangeLexSpec &spec,
+                                 std::vector<FieldValue> *field_values) {
   field_values->clear();
-  if (start.compare(stop) >= 0 || limit <= 0) {
+  if (spec.count == 0) {
     return rocksdb::Status::OK();
   }
   std::string ns_key;
@@ -287,26 +287,54 @@ rocksdb::Status Hash::Range(const Slice &user_key, const Slice &start, const Sli
   HashMetadata metadata(false);
   rocksdb::Status s = GetMetadata(ns_key, &metadata);
   if (!s.ok()) return s.IsNotFound() ? rocksdb::Status::OK() : s;
-  limit = std::min(static_cast<int64_t>(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);
+
+  std::string start_member = spec.reversed ? spec.max : spec.min;
+  std::string start_key, prefix_key, next_version_prefix_key;
+  InternalKey(ns_key, start_member, metadata.version, storage_->IsSlotIdEncoded()).Encode(&start_key);
+  InternalKey(ns_key, "", metadata.version, storage_->IsSlotIdEncoded()).Encode(&prefix_key);
+  InternalKey(ns_key, "", metadata.version + 1, storage_->IsSlotIdEncoded()).Encode(&next_version_prefix_key);
   rocksdb::ReadOptions read_options;
   LatestSnapShot ss(db_);
   read_options.snapshot = ss.GetSnapShot();
-  rocksdb::Slice upper_bound(stop_key);
+  rocksdb::Slice upper_bound(next_version_prefix_key);
   read_options.iterate_upper_bound = &upper_bound;
+  rocksdb::Slice lower_bound(prefix_key);
+  read_options.iterate_lower_bound = &lower_bound;
   read_options.fill_cache = false;
 
   auto iter = DBUtil::UniqueIterator(db_, read_options);
-  iter->Seek(start_key);
-  for (int64_t i = 0; iter->Valid() && i <= limit - 1; ++i) {
+  if (!spec.reversed) {
+    iter->Seek(start_key);
+  } else {
+    if (spec.max_infinite) {
+      iter->SeekToLast();
+    } else {
+      iter->SeekForPrev(start_key);
+    }
+  }
+  int64_t pos = 0;
+  for (; iter->Valid() && iter->key().starts_with(prefix_key); (!spec.reversed ? iter->Next() : iter->Prev())) {
     FieldValue tmp_field_value;
     InternalKey ikey(iter->key(), storage_->IsSlotIdEncoded());
+    if (spec.reversed) {
+      if (ikey.GetSubKey().ToString() < spec.min || (spec.minex && ikey.GetSubKey().ToString() == spec.min)) {
+        break;
+      }
+      if ((spec.maxex && ikey.GetSubKey().ToString() == spec.max) ||
+          (!spec.max_infinite && ikey.GetSubKey().ToString() > spec.max)) {
+        continue;
+      }
+    } else {
+      if (spec.minex && ikey.GetSubKey().ToString() == spec.min) continue;  // the min member was exclusive
+      if ((spec.maxex && ikey.GetSubKey().ToString() == spec.max) ||
+          (!spec.max_infinite && ikey.GetSubKey().ToString() > spec.max))
+        break;
+    }
+    if (spec.offset >= 0 && pos++ < spec.offset) continue;
     tmp_field_value.field = ikey.GetSubKey().ToString();
     tmp_field_value.value = iter->value().ToString();
     field_values->emplace_back(tmp_field_value);
-    iter->Next();
+    if (spec.count > 0 && field_values && field_values->size() >= static_cast<unsigned>(spec.count)) break;
   }
   return rocksdb::Status::OK();
 }
diff --git a/src/types/redis_hash.h b/src/types/redis_hash.h
index 0ed4c972..a9add33b 100644
--- a/src/types/redis_hash.h
+++ b/src/types/redis_hash.h
@@ -25,6 +25,7 @@
 #include <string>
 #include <vector>
 
+#include "common/range_spec.h"
 #include "encoding.h"
 #include "storage/redis_db.h"
 #include "storage/redis_metadata.h"
@@ -48,8 +49,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<FieldValue> &field_values, bool nx, int *ret);
-  rocksdb::Status Range(const Slice &user_key, const Slice &start, const Slice &stop, int64_t limit,
-                        std::vector<FieldValue> *field_values);
+  rocksdb::Status RangeByLex(const Slice &user_key, const CommonRangeLexSpec &spec,
+                             std::vector<FieldValue> *field_values);
   rocksdb::Status MGet(const Slice &user_key, const std::vector<Slice> &fields, std::vector<std::string> *values,
                        std::vector<rocksdb::Status> *statuses);
   rocksdb::Status GetAll(const Slice &user_key, std::vector<FieldValue> *field_values,
diff --git a/src/types/redis_zset.cc b/src/types/redis_zset.cc
index 57aecd71..bc2b7ef5 100644
--- a/src/types/redis_zset.cc
+++ b/src/types/redis_zset.cc
@@ -414,8 +414,8 @@ rocksdb::Status ZSet::RangeByScore(const Slice &user_key, ZRangeSpec spec, std::
   return rocksdb::Status::OK();
 }
 
-rocksdb::Status ZSet::RangeByLex(const Slice &user_key, const ZRangeLexSpec &spec, std::vector<std::string> *members,
-                                 int *size) {
+rocksdb::Status ZSet::RangeByLex(const Slice &user_key, const CommonRangeLexSpec &spec,
+                                 std::vector<std::string> *members, int *size) {
   if (size) *size = 0;
   if (members) members->clear();
   if (spec.offset > -1 && spec.count == 0) {
@@ -564,7 +564,7 @@ rocksdb::Status ZSet::RemoveRangeByScore(const Slice &user_key, ZRangeSpec spec,
   return RangeByScore(user_key, spec, nullptr, ret);
 }
 
-rocksdb::Status ZSet::RemoveRangeByLex(const Slice &user_key, ZRangeLexSpec spec, int *ret) {
+rocksdb::Status ZSet::RemoveRangeByLex(const Slice &user_key, CommonRangeLexSpec spec, int *ret) {
   spec.removed = true;
   return RangeByLex(user_key, spec, nullptr, ret);
 }
@@ -799,39 +799,6 @@ Status ZSet::ParseRangeSpec(const std::string &min, const std::string &max, ZRan
   return Status::OK();
 }
 
-Status ZSet::ParseRangeLexSpec(const std::string &min, const std::string &max, ZRangeLexSpec *spec) {
-  if (min == "+" || max == "-") {
-    return Status(Status::NotOK, "min > max");
-  }
-
-  if (min == "-") {
-    spec->min = "";
-  } else {
-    if (min[0] == '(') {
-      spec->minex = true;
-    } else if (min[0] == '[') {
-      spec->minex = false;
-    } else {
-      return Status(Status::NotOK, "the min is illegal");
-    }
-    spec->min = min.substr(1);
-  }
-
-  if (max == "+") {
-    spec->max_infinite = true;
-  } else {
-    if (max[0] == '(') {
-      spec->maxex = true;
-    } else if (max[0] == '[') {
-      spec->maxex = false;
-    } else {
-      return Status(Status::NotOK, "the max is illegal");
-    }
-    spec->max = max.substr(1);
-  }
-  return Status::OK();
-}
-
 rocksdb::Status ZSet::Scan(const Slice &user_key, const std::string &cursor, uint64_t limit,
                            const std::string &member_prefix, std::vector<std::string> *members,
                            std::vector<double> *scores) {
diff --git a/src/types/redis_zset.h b/src/types/redis_zset.h
index ac2625de..fc8a4ecb 100644
--- a/src/types/redis_zset.h
+++ b/src/types/redis_zset.h
@@ -25,6 +25,7 @@
 #include <string>
 #include <vector>
 
+#include "common/range_spec.h"
 #include "storage/redis_db.h"
 #include "storage/redis_metadata.h"
 
@@ -43,15 +44,6 @@ struct ZRangeSpec {
   ZRangeSpec() = default;
 };
 
-struct ZRangeLexSpec {
-  std::string min, max;
-  bool minex = false, maxex = false; /* are min or max exclusive */
-  bool max_infinite = false;         /* are max infinite */
-  int offset = -1, count = -1;
-  bool removed = false, reversed = false;
-  ZRangeLexSpec() = default;
-};
-
 struct KeyWeight {
   std::string key;
   double weight;
@@ -107,17 +99,16 @@ class ZSet : public SubKeyScanner {
   rocksdb::Status IncrBy(const Slice &user_key, const Slice &member, double increment, double *score);
   rocksdb::Status Range(const Slice &user_key, int start, int stop, uint8_t flags, std::vector<MemberScore> *mscores);
   rocksdb::Status RangeByScore(const Slice &user_key, ZRangeSpec spec, std::vector<MemberScore> *mscores, int *size);
-  rocksdb::Status RangeByLex(const Slice &user_key, const ZRangeLexSpec &spec, std::vector<std::string> *members,
+  rocksdb::Status RangeByLex(const Slice &user_key, const CommonRangeLexSpec &spec, std::vector<std::string> *members,
                              int *size);
   rocksdb::Status Rank(const Slice &user_key, const Slice &member, bool reversed, int *ret);
   rocksdb::Status Remove(const Slice &user_key, const std::vector<Slice> &members, int *ret);
   rocksdb::Status RemoveRangeByScore(const Slice &user_key, ZRangeSpec spec, int *ret);
-  rocksdb::Status RemoveRangeByLex(const Slice &user_key, ZRangeLexSpec spec, int *ret);
+  rocksdb::Status RemoveRangeByLex(const Slice &user_key, CommonRangeLexSpec spec, int *ret);
   rocksdb::Status RemoveRangeByRank(const Slice &user_key, int start, int stop, int *ret);
   rocksdb::Status Pop(const Slice &user_key, int count, bool min, std::vector<MemberScore> *mscores);
   rocksdb::Status Score(const Slice &user_key, const Slice &member, double *score);
   static Status ParseRangeSpec(const std::string &min, const std::string &max, ZRangeSpec *spec);
-  static Status ParseRangeLexSpec(const std::string &min, const std::string &max, ZRangeLexSpec *spec);
   rocksdb::Status Scan(const Slice &user_key, const std::string &cursor, uint64_t limit,
                        const std::string &member_prefix, std::vector<std::string> *members,
                        std::vector<double> *scores = nullptr);
diff --git a/tests/cppunit/t_hash_test.cc b/tests/cppunit/t_hash_test.cc
index 9f0eabd6..7caa367d 100644
--- a/tests/cppunit/t_hash_test.cc
+++ b/tests/cppunit/t_hash_test.cc
@@ -21,6 +21,7 @@
 #include <gtest/gtest.h>
 
 #include <algorithm>
+#include <climits>
 #include <memory>
 #include <random>
 #include <string>
@@ -160,8 +161,8 @@ TEST_F(RedisHashTest, HIncrByFloat) {
   hash->Del(key_);
 }
 
-TEST_F(RedisHashTest, HRange) {
-  int ret;
+TEST_F(RedisHashTest, HRangeByLex) {
+  int ret = 0;
   std::vector<FieldValue> fvs;
   for (size_t i = 0; i < 4; i++) {
     fvs.emplace_back(FieldValue{"key" + std::to_string(i), "value" + std::to_string(i)});
@@ -180,19 +181,113 @@ TEST_F(RedisHashTest, HRange) {
     s = hash->MSet(key_, fvs, false, &ret);
     EXPECT_EQ(ret, 0);
     std::vector<FieldValue> result;
-    s = hash->Range(key_, "key0", "key4", INT_MAX, &result);
+    CommonRangeLexSpec spec;
+    spec.offset = 0;
+    spec.count = INT_MAX;
+    spec.min = "key0";
+    spec.max = "key3";
+    s = hash->RangeByLex(key_, spec, &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);
+    EXPECT_EQ("key3", result[3].field);
     hash->Del(key_);
   }
+
+  rocksdb::Status s = hash->MSet(key_, tmp, false, &ret);
+  EXPECT_TRUE(s.ok() && static_cast<int>(tmp.size()) == ret);
+  // use offset and count
+  std::vector<FieldValue> result;
+  CommonRangeLexSpec spec;
+  spec.offset = 0;
+  spec.count = INT_MAX;
+  spec.min = "key0";
+  spec.max = "key3";
+  spec.offset = 1;
+  s = hash->RangeByLex(key_, spec, &result);
+  EXPECT_TRUE(s.ok());
+  EXPECT_EQ(3, result.size());
+  EXPECT_EQ("key1", result[0].field);
+  EXPECT_EQ("key2", result[1].field);
+  EXPECT_EQ("key3", result[2].field);
+
+  spec.offset = 1;
+  spec.count = 1;
+  s = hash->RangeByLex(key_, spec, &result);
+  EXPECT_TRUE(s.ok());
+  EXPECT_EQ(1, result.size());
+  EXPECT_EQ("key1", result[0].field);
+
+  spec.offset = 0;
+  spec.count = 0;
+  s = hash->RangeByLex(key_, spec, &result);
+  EXPECT_TRUE(s.ok());
+  EXPECT_EQ(0, result.size());
+
+  spec.offset = 1000;
+  spec.count = 1000;
+  s = hash->RangeByLex(key_, spec, &result);
+  EXPECT_TRUE(s.ok());
+  EXPECT_EQ(0, result.size());
+  // exclusive range
+  spec.offset = 0;
+  spec.count = -1;
+  spec.minex = true;
+  s = hash->RangeByLex(key_, spec, &result);
+  EXPECT_TRUE(s.ok());
+  EXPECT_EQ(3, result.size());
+  EXPECT_EQ("key1", result[0].field);
+  EXPECT_EQ("key2", result[1].field);
+  EXPECT_EQ("key3", result[2].field);
+
+  spec.offset = 0;
+  spec.count = -1;
+  spec.maxex = true;
+  spec.minex = false;
+  s = hash->RangeByLex(key_, spec, &result);
+  EXPECT_TRUE(s.ok());
+  EXPECT_EQ(3, result.size());
+  EXPECT_EQ("key0", result[0].field);
+  EXPECT_EQ("key1", result[1].field);
+  EXPECT_EQ("key2", result[2].field);
+
+  spec.offset = 0;
+  spec.count = -1;
+  spec.maxex = true;
+  spec.minex = true;
+  s = hash->RangeByLex(key_, spec, &result);
+  EXPECT_TRUE(s.ok());
+  EXPECT_EQ(2, result.size());
+  EXPECT_EQ("key1", result[0].field);
+  EXPECT_EQ("key2", result[1].field);
+
+  // inf and revered
+  spec.minex = false;
+  spec.maxex = false;
+  spec.min = "-";
+  spec.max = "+";
+  spec.max_infinite = true;
+  spec.reversed = true;
+  s = hash->RangeByLex(key_, spec, &result);
+  EXPECT_TRUE(s.ok());
+  EXPECT_EQ(4 + 26, result.size());
+  EXPECT_EQ("key3", result[0].field);
+  EXPECT_EQ("key2", result[1].field);
+  EXPECT_EQ("key1", result[2].field);
+  EXPECT_EQ("key0", result[3].field);
+  hash->Del(key_);
 }
 
-TEST_F(RedisHashTest, HRangeNonExistingKey) {
+TEST_F(RedisHashTest, HRangeByLexNonExistingKey) {
   std::vector<FieldValue> result;
-  auto s = hash->Range("non-existing-key", "any-start-key", "any-end-key", 10, &result);
+  CommonRangeLexSpec spec;
+  spec.offset = 0;
+  spec.count = INT_MAX;
+  spec.min = "any-start-key";
+  spec.max = "any-end-key";
+  auto s = hash->RangeByLex("non-existing-key", spec, &result);
   EXPECT_TRUE(s.ok());
   EXPECT_EQ(result.size(), 0);
 }
diff --git a/tests/cppunit/t_zset_test.cc b/tests/cppunit/t_zset_test.cc
index 288fbe11..8dc39175 100644
--- a/tests/cppunit/t_zset_test.cc
+++ b/tests/cppunit/t_zset_test.cc
@@ -174,7 +174,7 @@ TEST_F(RedisZSetTest, RangeByLex) {
   zset->Add(key_, ZAddFlags::Default(), &mscores, &ret);
   EXPECT_EQ(fields_.size(), ret);
 
-  ZRangeLexSpec spec;
+  CommonRangeLexSpec spec;
   spec.min = fields_[0].ToString();
   spec.max = fields_[fields_.size() - 1].ToString();
   std::vector<std::string> members;
diff --git a/tests/gocase/unit/type/hash/hash_test.go b/tests/gocase/unit/type/hash/hash_test.go
index 8ffb6a89..f8a23bc1 100644
--- a/tests/gocase/unit/type/hash/hash_test.go
+++ b/tests/gocase/unit/type/hash/hash_test.go
@@ -614,54 +614,71 @@ func TestHash(t *testing.T) {
 		})
 
 		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) {
+		t.Run("HrangeByLex BYLEX 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())
+			require.EqualValues(t, []interface{}{"key1", "value1", "key10", "value10"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[key1", "[key10").Val())
+			require.EqualValues(t, []interface{}{"key1", "value1", "key10", "value10", "key2", "value2"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[key1", "[key2").Val())
+			require.EqualValues(t, []interface{}{"key1", "value1", "key10", "value10", "key2", "value2"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[key1", "(key3").Val())
+			require.EqualValues(t, []interface{}{"key10", "value10", "key2", "value2", "key3", "value3"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "(key1", "[key3", "limit", 0, -1).Val())
+			require.EqualValues(t, []interface{}{"key10", "value10", "key2", "value2"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "(key1", "(key3").Val())
+			require.EqualValues(t, []interface{}{"a", "a", "b", "b"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "-", "[b").Val())
+			require.EqualValues(t, []interface{}{"x", "x", "z", "z"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[x", "+").Val())
+			require.EqualValues(t, []interface{}{"z", "z", "x", "x"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "+", "[x", "REV").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, "HrangeByLex", "hashkey", "-", "+").Val())
+			require.EqualValues(t, []interface{}{"z", "z", "x", "x", "key3", "value3", "key2", "value2", "key10", "value10", "key1", "value1", "e", "e", "d", "d", "c", "c", "b", "b", "a", "a"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "+", "-", "REV").Val())
 		})
 
-		t.Run("HRange stop <= start", func(t *testing.T) {
+		t.Run("HrangeByLex BYLEX 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())
+			require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[key2", "[key1", "limit", 0, 100).Val())
+			require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HrangeByLex", "hashkey", "(key1", "(key1", "limit", 0, 100).Val())
 		})
 
-		t.Run("HRange limit", func(t *testing.T) {
+		t.Run("HrangeByLex BYLEX 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())
+			require.EqualValues(t, []interface{}{"a", "a", "b", "b"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a", "[z", "limit", 0, 2).Val())
+			require.EqualValues(t, []interface{}{"z", "z", "x", "x"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[z", "[a", "limit", 0, 2, "REV").Val())
+			require.EqualValues(t, []interface{}{"b", "b", "c", "c"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a", "[z", "limit", 1, 2).Val())
+			require.EqualValues(t, []interface{}{"x", "x", "key3", "value3"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[z", "[a", "limit", 1, 2, "REV").Val())
+			require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a", "[z", "limit", 1000, -1).Val())
+			require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a", "[z", "limit", 0, 0).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, "HrangeByLex", "hashkey", "[a", "[zzz", "limit", 0, 10000).Val())
 		})
 
-		t.Run("HRange limit is negative", func(t *testing.T) {
+		t.Run("HrangeByLex BYLEX 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())
+			require.EqualValues(t, []interface{}{"x", "x", "z", "z"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[x", "[z", "limit", 0, -100).Val())
+			require.EqualValues(t, []interface{}{"z", "z", "x", "x"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[z", "[x", "limit", 0, -100, "REV").Val())
+			require.EqualValues(t, []interface{}{"x", "x", "z", "z"}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[x", "[z", "limit", 0, -10).Val())
 		})
 
-		t.Run("HRange nonexistent key", func(t *testing.T) {
+		t.Run("HrangeByLex BYLEX 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())
+			require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a", "[z").Val())
+			require.EqualValues(t, []interface{}{}, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a", "[z").Val())
 		})
 
-		t.Run("HRange limit typo", func(t *testing.T) {
+		t.Run("HrangeByLex 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")
+			require.ErrorContains(t, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a", "[z", "limitzz", 0, 10000).Err(), "ERR syntax")
+			require.ErrorContains(t, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a", "[z", "limit", 0, 10000, "BYLE").Err(), "ERR syntax")
+			require.ErrorContains(t, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a", "[z", "limit", 0, 10000, "RE").Err(), "ERR syntax")
+			require.ErrorContains(t, rdb.Do(ctx, "HrangeByLex", "hashkey", "a", "z").Err(), "illegal")
 		})
 
-		t.Run("HRange wrong number of arguments", func(t *testing.T) {
+		t.Run("HrangeByLex 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")
+			require.ErrorContains(t, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a", "[z", "limit", 10000, 1, 1, 1, 1).Err(), "syntax error")
+			require.ErrorContains(t, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a", "[z", "limit").Err(), "no more item to parse")
+			require.ErrorContains(t, rdb.Do(ctx, "HrangeByLex", "hashkey", "[a").Err(), "wrong number of arguments")
+			require.ErrorContains(t, rdb.Do(ctx, "HrangeByLex", "hashkey").Err(), "wrong number of arguments")
+			require.ErrorContains(t, rdb.Do(ctx, "HrangeByLex").Err(), "wrong number of arguments")
 		})
 	}
 }