You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@kvrocks.apache.org by "infdahai (via GitHub)" <gi...@apache.org> on 2023/05/18 10:35:16 UTC

[GitHub] [incubator-kvrocks] infdahai opened a new pull request, #1444: feat: add sintercard command from redis 7.0

infdahai opened a new pull request, #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444

   closes #1083 #1116 (replace)
   
   This PR introduces a `sintercard` command to `kvrocks` from redis 7.0.
   
   BTW, the `go-redis` package's version is `v9.0.0` in the project so it doesn't support this command. And I just write a bare test in `go-test`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] git-hulk merged pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk merged PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] git-hulk commented on pull request #1444: feat: add sintercard command from redis 7.0

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1556629165

   The flaky test still exists after applying #1449 and it should be caused by the retry in Go redis client.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1198068036


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,46 @@ class CommandSInter : public Commander {
   }
 };
 
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command:  SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4) {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 2], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = std::max(0, *parse_limit);
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);
+    }
+
+    redis::Set set_db(svr->storage, conn->GetNamespace());
+    int ret = 0;

Review Comment:
   >  the go-redis package's version is v9.0.0-beta.2 in the project so it doesn't support this command
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199579994


##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,43 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit != 0;
+  bool limited = false;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {

Review Comment:
   Why do we search for the `member` element three times in `member_counters` continuously? It looks so bad to me.
   
   ```c++
   auto iter = find(member); // find it
   use1(iter); // use it
   use2(iter); // use it again
   
   // **NOT**:
   use1(find(member));
   use2(find(member));
   use3(find(member));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1216727167


##########
tests/cppunit/types/set_test.cc:
##########
@@ -188,9 +188,41 @@ TEST_F(RedisSetTest, Inter) {
   std::vector<std::string> members;
   set_->Inter({k1, k2, k3}, &members);
   EXPECT_EQ(1, members.size());
+  set_->Inter({k1, k2, k4}, &members);
+  EXPECT_EQ(0, members.size());
+  set_->Del(k1);
+  set_->Del(k2);
+  set_->Del(k3);
+  set_->Del(k4);
+}
+
+TEST_F(RedisSetTest, InterCard) {
+  uint64_t ret = 0;
+  std::string k1 = "key1", k2 = "key2", k3 = "key3", k4 = "key4";
+  rocksdb::Status s = set_->Add(k1, {"a", "b", "c", "d"}, &ret);
+  EXPECT_EQ(ret, 4);
+  set_->Add(k2, {"c", "d", "e"}, &ret);
+  EXPECT_EQ(ret, 3);
+  set_->Add(k3, {"e", "f"}, &ret);
+  EXPECT_EQ(ret, 2);
+  set_->InterCard({k1, k2}, 0, &ret);
+  EXPECT_EQ(ret, 2);
+  set_->InterCard({k1, k2}, 1, &ret);
+  EXPECT_EQ(ret, 1);
+  set_->InterCard({k1, k2}, 3, &ret);
+  EXPECT_EQ(ret, 2);
+  set_->InterCard({k2, k3}, 1, &ret);
+  EXPECT_EQ(ret, 1);
+  set_->InterCard({k1, k3}, 5, &ret);
+  EXPECT_EQ(ret, 0);
+  set_->InterCard({k1, k4}, 5, &ret);
+  EXPECT_EQ(ret, 0);
+  set_->InterCard({k1}, 0, &ret);
+  EXPECT_EQ(ret, 4);

Review Comment:
   If I add the following test:
   
   ```
   set_->InterCard({k1}, 2, &ret);
   EXPECT_EQ(ret, 2);
   ```
   
   it will fail.
   
   I tested it here: https://redis.io/commands/sintercard/
   
   ```
   redis> sintercard 1 key1 limit 0
   (integer) 4
   redis> sintercard 1 key1 limit 2
   (integer) 2
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1204266158


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,65 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *    syntax:   `SINTERCARD numkeys key [key ...] [LIMIT limit]`
+ *
+ *    limit:    the valid limit is an non-negative integer.
+ */
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command: for example, SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4 && util::ToLower(args[numkeys_ + 2]) == "limit") {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 3], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = *parse_limit;
+      if (limit_ < 0) {
+        return {Status::RedisParseErr, errLimitIsNegative};
+      }
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);

Review Comment:
   Any user input via redis protocol cannot just make kvrocks crash. That is the **principal commitment** of development.
   
   And you cannot enforce users to always use go-redis wrapped command functions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1197971321


##########
tests/cppunit/types/set_test.cc:
##########
@@ -191,6 +191,23 @@ TEST_F(RedisSetTest, Inter) {
   set_->Del(k3);
 }
 
+TEST_F(RedisSetTest, InterCard) {

Review Comment:
   I add these in `inter` tests later beacause `intercard` is a subset of calling `inter` .



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1198068036


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,46 @@ class CommandSInter : public Commander {
   }
 };
 
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command:  SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4) {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 2], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = std::max(0, *parse_limit);
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);
+    }
+
+    redis::Set set_db(svr->storage, conn->GetNamespace());
+    int ret = 0;

Review Comment:
   >  the go-redis package's version is v9.0.0-beta.2 in the project so it doesn't support this command
   
   So we should wait for updating the go-redis package and then write go tests. 
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1218293285


##########
src/types/redis_set.cc:
##########
@@ -332,29 +332,66 @@ rocksdb::Status Set::Union(const std::vector<Slice> &keys, std::vector<std::stri
  * key3 = {a,c,e}
  * INTER key1 key2 key3 = {c}
  */
-rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::string> *members) {
+rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::string> *members, uint64_t *cnt) {
   members->clear();
 
   std::map<std::string, size_t> member_counters;
   std::vector<std::string> target_members;
   auto s = Members(keys[0], &target_members);
   if (!s.ok() || target_members.empty()) return s;
+
+  uint64_t limit = cnt ? *cnt : 0;
+  bool has_limit = limit != 0;
+  bool limited = false;

Review Comment:
   Yeah, I should split the func.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1587239822

   > @infdahai You didn't actually split the `Inter` function into two separate :) I meant that the `Inter` function should calculate the intersection between sets and return members while `InterCard` should only calculate the cardinality and return the computed value. The command `Inter` will call `Set::Inter` and the command `InterCard` will call the `Set::InterCard` (currently the `Set::InterCard` function calls `Set::Inter` and all the complexity is still in the `Set::Inter` function). In this case, you will get rid of some of the `if` statements and simplify the code.
   
   Ok. I just thought if I split the functions, these two functions exist duplicate codes before and we add the limit `0` to `Set::Inter` to unify the two functions currently. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199583977


##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,43 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit != 0;
+  bool limited = false;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {
+        members->emplace_back(member);
+        if (--limit == 0) {
+          limited = true;
+          break;
+        }
+      }
     }
+
+    if (limited) break;
   }
-  for (const auto &iter : member_counters) {
-    if (iter.second == keys.size()) {  // all the sets contain this member
-      members->emplace_back(iter.first);
+
+  if (!has_limit) {
+    for (const auto &iter : member_counters) {
+      if (iter.second == keys.size()) {  // all the sets contain this member
+        members->emplace_back(iter.first);
+      }
     }
   }
+
+  return rocksdb::Status::OK();
+}
+
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, int *ret) {

Review Comment:
   @PragmaTwice we exec the statement `*output = redis::Integer(ret);` to return the result to the client in cmd_set.cc.
   https://github.com/apache/incubator-kvrocks/blob/fb0e3d42031a6e0691c7a736711d0a01d72bdb1a/src/server/redis_reply.h#L38-L41
   
   Add the `Integer` func checks whether compatibility with `int` is required(Of course, `size_t` is right). I see src/types/*.h, many funcs with this situation use `int`. 



##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,43 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit != 0;
+  bool limited = false;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {
+        members->emplace_back(member);
+        if (--limit == 0) {
+          limited = true;
+          break;
+        }
+      }
     }
+
+    if (limited) break;
   }
-  for (const auto &iter : member_counters) {
-    if (iter.second == keys.size()) {  // all the sets contain this member
-      members->emplace_back(iter.first);
+
+  if (!has_limit) {
+    for (const auto &iter : member_counters) {
+      if (iter.second == keys.size()) {  // all the sets contain this member
+        members->emplace_back(iter.first);
+      }
     }
   }
+
+  return rocksdb::Status::OK();
+}
+
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, int *ret) {

Review Comment:
   @PragmaTwice I exec the statement `*output = redis::Integer(ret);` to return the result to the client in cmd_set.cc.
   https://github.com/apache/incubator-kvrocks/blob/fb0e3d42031a6e0691c7a736711d0a01d72bdb1a/src/server/redis_reply.h#L38-L41
   
   Add the `Integer` func checks whether compatibility with `int` is required(Of course, `size_t` is right). I see src/types/*.h, many funcs with this situation use `int`. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1197971321


##########
tests/cppunit/types/set_test.cc:
##########
@@ -191,6 +191,23 @@ TEST_F(RedisSetTest, Inter) {
   set_->Del(k3);
 }
 
+TEST_F(RedisSetTest, InterCard) {

Review Comment:
   I plan to add these in `inter` tests later beacause `intercard` is a subset of calling `inter` .



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199580764


##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,43 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit != 0;
+  bool limited = false;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {
+        members->emplace_back(member);
+        if (--limit == 0) {
+          limited = true;
+          break;
+        }
+      }
     }
+
+    if (limited) break;
   }
-  for (const auto &iter : member_counters) {
-    if (iter.second == keys.size()) {  // all the sets contain this member
-      members->emplace_back(iter.first);
+
+  if (!has_limit) {
+    for (const auto &iter : member_counters) {
+      if (iter.second == keys.size()) {  // all the sets contain this member
+        members->emplace_back(iter.first);
+      }
     }
   }
+
+  return rocksdb::Status::OK();
+}
+
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, int *ret) {

Review Comment:
   Maybe not related to this PR, but there are two points:
   - `ret` is not a descriptive word
   - `int` type is not fit here: the return variable `members.size()` is typed `size_t`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1198068036


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,46 @@ class CommandSInter : public Commander {
   }
 };
 
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command:  SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4) {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 2], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = std::max(0, *parse_limit);
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);
+    }
+
+    redis::Set set_db(svr->storage, conn->GetNamespace());
+    int ret = 0;

Review Comment:
   > You don't use this ret variable
   yeah, I fix it now.
   
   >  the go-redis package's version is v9.0.0-beta.2 in the project so it doesn't support this command
   
   So we should wait for updating the go-redis package and then write go tests. 
   



##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,46 @@ class CommandSInter : public Commander {
   }
 };
 
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command:  SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4) {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 2], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = std::max(0, *parse_limit);
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);
+    }
+
+    redis::Set set_db(svr->storage, conn->GetNamespace());
+    int ret = 0;

Review Comment:
   > You don't use this ret variable
   
   yeah, I add it now.
   
   >  the go-redis package's version is v9.0.0-beta.2 in the project so it doesn't support this command
   
   So we should wait for updating the go-redis package and then write go tests. 
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1204147698


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,65 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *    syntax:   `SINTERCARD numkeys key [key ...] [LIMIT limit]`
+ *
+ *    limit:    the valid limit is an non-negative integer.
+ */
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command: for example, SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4 && util::ToLower(args[numkeys_ + 2]) == "limit") {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 3], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = *parse_limit;
+      if (limit_ < 0) {
+        return {Status::RedisParseErr, errLimitIsNegative};
+      }
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);

Review Comment:
   It seems there is no check for the boundary of `args_`, e.g. for `SINTERCARD 99 a b` will the `Execute` function just crash on this line?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1586167091

   https://github.com/apache/incubator-kvrocks/actions/runs/5234808336/jobs/9452175246
   
   Does we need to ignore this?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1227111882


##########
src/types/redis_set.cc:
##########
@@ -358,6 +358,54 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   return rocksdb::Status::OK();
 }
 
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, uint64_t *cnt) {
+  *cnt = 0;
+
+  std::map<std::string, size_t> member_counters;
+  std::vector<std::string> target_members;
+
+  auto s = Members(keys[0], &target_members);
+  if (!s.ok() || target_members.empty()) return s;
+  for (const auto &member : target_members) {
+    member_counters[member] = 1;
+  }
+  if (limit == 0) {
+    limit = target_members.size();
+  }
+
+  size_t keys_size = keys.size();
+  if (keys_size == 1) {
+    *cnt = std::min(static_cast<uint64_t>(target_members.size()), limit);
+    return rocksdb::Status::OK();
+  }
+
+  int cardinality = 0;
+  bool limit_reached = false;
+  for (size_t i = 1; i < keys_size; i++) {
+    s = Members(keys[i], &target_members);
+    if (!s.ok() || target_members.empty()) {
+      return s;
+    }
+
+    for (const auto &member : target_members) {
+      auto iter = member_counters.find(member);
+      if (iter == member_counters.end()) continue;
+      if (++iter->second == keys_size) {
+        cardinality++;

Review Comment:
   ignore this. I don't like accessing pointers all the time.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] git-hulk commented on pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1588540097

   Thanks all, merging...


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199579994


##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,43 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit != 0;
+  bool limited = false;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {

Review Comment:
   Why do we search for the `member` element three times in `member_counters` continuously? It looks so bad to me.
   
   ```c++
   auto iter = find(member); // find it
   use1(iter); // use it
   use2(iter);
   
   // **NOT**:
   use1(find(member));
   use2(find(member));
   use3(find(member));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1198030725


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,46 @@ class CommandSInter : public Commander {
   }
 };
 
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command:  SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4) {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 2], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = std::max(0, *parse_limit);
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);
+    }
+
+    redis::Set set_db(svr->storage, conn->GetNamespace());
+    int ret = 0;

Review Comment:
   You don't use this `ret` variable, you don't send any response to the command. Please, add some integration tests in Go to check the proper responses of the newly created command.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1201885840


##########
src/commands/cmd_set.cc:
##########
@@ -371,6 +418,7 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandSAdd>("sadd", -3, "write", 1, 1, 1),
                         MakeCmdAttr<CommandSDiff>("sdiff", -2, "read-only", 1, -1, 1),
                         MakeCmdAttr<CommandSUnion>("sunion", -2, "read-only", 1, -1, 1),
                         MakeCmdAttr<CommandSInter>("sinter", -2, "read-only", 1, -1, 1),
+                        MakeCmdAttr<CommandSInterCard>("sintercard", -3, "read-only", 1, -1, 1),

Review Comment:
   Thanks. I've got it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1204234631


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,65 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *    syntax:   `SINTERCARD numkeys key [key ...] [LIMIT limit]`
+ *
+ *    limit:    the valid limit is an non-negative integer.
+ */
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command: for example, SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4 && util::ToLower(args[numkeys_ + 2]) == "limit") {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 3], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = *parse_limit;
+      if (limit_ < 0) {
+        return {Status::RedisParseErr, errLimitIsNegative};
+      }
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);

Review Comment:
   Thank you so much. So many vulnerabilities.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1204276633


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,65 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *    syntax:   `SINTERCARD numkeys key [key ...] [LIMIT limit]`
+ *
+ *    limit:    the valid limit is an non-negative integer.
+ */
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command: for example, SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4 && util::ToLower(args[numkeys_ + 2]) == "limit") {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 3], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = *parse_limit;
+      if (limit_ < 0) {
+        return {Status::RedisParseErr, errLimitIsNegative};
+      }
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);

Review Comment:
   OK, I understand this and thanks again.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1201862674


##########
src/commands/cmd_set.cc:
##########
@@ -371,6 +418,7 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandSAdd>("sadd", -3, "write", 1, 1, 1),
                         MakeCmdAttr<CommandSDiff>("sdiff", -2, "read-only", 1, -1, 1),
                         MakeCmdAttr<CommandSUnion>("sunion", -2, "read-only", 1, -1, 1),
                         MakeCmdAttr<CommandSInter>("sinter", -2, "read-only", 1, -1, 1),
+                        MakeCmdAttr<CommandSInterCard>("sintercard", -3, "read-only", 1, -1, 1),

Review Comment:
   @PragmaTwice Would it be better to use this change based on the syntax `SINTERCARD numkeys key [key ...] [LIMIT limit]` ? It seems this range is related to `watch`
   ```suggestion
                           MakeCmdAttr<CommandSInterCard>("sintercard", -3, "read-only", 2, -1, 1),
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1204266158


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,65 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *    syntax:   `SINTERCARD numkeys key [key ...] [LIMIT limit]`
+ *
+ *    limit:    the valid limit is an non-negative integer.
+ */
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command: for example, SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4 && util::ToLower(args[numkeys_ + 2]) == "limit") {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 3], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = *parse_limit;
+      if (limit_ < 0) {
+        return {Status::RedisParseErr, errLimitIsNegative};
+      }
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);

Review Comment:
   Any user input via redis protocol cannot just make kvrocks crash. That is the principal commitment of development.
   
   And you cannot enforce users to always use go-redis wrapped command functions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] git-hulk commented on pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1588574216

   @infdahai Can add the SINTERCARD command to https://github.com/apache/incubator-kvrocks-website/blob/main/docs/supported-commands.md


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1218242162


##########
src/types/redis_set.cc:
##########
@@ -332,29 +332,66 @@ rocksdb::Status Set::Union(const std::vector<Slice> &keys, std::vector<std::stri
  * key3 = {a,c,e}
  * INTER key1 key2 key3 = {c}
  */
-rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::string> *members) {
+rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::string> *members, uint64_t *cnt) {
   members->clear();
 
   std::map<std::string, size_t> member_counters;
   std::vector<std::string> target_members;
   auto s = Members(keys[0], &target_members);
   if (!s.ok() || target_members.empty()) return s;
+
+  uint64_t limit = cnt ? *cnt : 0;
+  bool has_limit = limit != 0;
+  bool limited = false;

Review Comment:
   I think you can rename this variable to `limit_reached`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1225749446


##########
src/types/redis_set.cc:
##########
@@ -332,21 +332,21 @@ rocksdb::Status Set::Union(const std::vector<Slice> &keys, std::vector<std::stri
  * key3 = {a,c,e}
  * INTER key1 key2 key3 = {c}
  */
-rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::string> *members, uint64_t *cnt) {
+rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::string> *members, uint64_t limit,
+                           uint64_t *cnt) {
   members->clear();
 
   std::map<std::string, size_t> member_counters;
   std::vector<std::string> target_members;
   auto s = Members(keys[0], &target_members);
   if (!s.ok() || target_members.empty()) return s;
 
-  uint64_t limit = cnt ? *cnt : 0;
   bool has_limit = limit != 0;
-  bool limited = false;
+  bool limit_reached = false;
   size_t keys_size = keys.size();
 
   if (keys_size == 1 && has_limit) {
-    if (limit > target_members.size()) *cnt = target_members.size();
+    *cnt = limit > target_members.size() ? target_members.size() : limit;

Review Comment:
   Here maybe a nullptr check is needed for `cnt`. But IMHO, we can assume that `cnt` can never be nullptr, remove all these check and the default argument of `cnt` in the function declaration.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1225749446


##########
src/types/redis_set.cc:
##########
@@ -332,21 +332,21 @@ rocksdb::Status Set::Union(const std::vector<Slice> &keys, std::vector<std::stri
  * key3 = {a,c,e}
  * INTER key1 key2 key3 = {c}
  */
-rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::string> *members, uint64_t *cnt) {
+rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::string> *members, uint64_t limit,
+                           uint64_t *cnt) {
   members->clear();
 
   std::map<std::string, size_t> member_counters;
   std::vector<std::string> target_members;
   auto s = Members(keys[0], &target_members);
   if (!s.ok() || target_members.empty()) return s;
 
-  uint64_t limit = cnt ? *cnt : 0;
   bool has_limit = limit != 0;
-  bool limited = false;
+  bool limit_reached = false;
   size_t keys_size = keys.size();
 
   if (keys_size == 1 && has_limit) {
-    if (limit > target_members.size()) *cnt = target_members.size();
+    *cnt = limit > target_members.size() ? target_members.size() : limit;

Review Comment:
   Here maybe a nullptr check is needed for `cnt`. But IMHO, we can assume that `cnt` can never be nullptr, remove all these check and the default argument of `cnt` (`nullptr`) in the function declaration.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199581695


##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,43 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit != 0;
+  bool limited = false;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {
+        members->emplace_back(member);
+        if (--limit == 0) {
+          limited = true;
+          break;
+        }
+      }
     }
+
+    if (limited) break;
   }
-  for (const auto &iter : member_counters) {
-    if (iter.second == keys.size()) {  // all the sets contain this member
-      members->emplace_back(iter.first);
+
+  if (!has_limit) {
+    for (const auto &iter : member_counters) {
+      if (iter.second == keys.size()) {  // all the sets contain this member
+        members->emplace_back(iter.first);
+      }
     }
   }
+
+  return rocksdb::Status::OK();
+}
+
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, int *ret) {

Review Comment:
   We find a few functions with `int *ret` in [redis_set.cc](https://github.com/apache/incubator-kvrocks/blob/unstable/src/types/redis_set.cc). However the result of `ret` is unsinged long(size_t)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1197814030


##########
tests/cppunit/types/set_test.cc:
##########
@@ -191,6 +191,23 @@ TEST_F(RedisSetTest, Inter) {
   set_->Del(k3);
 }
 
+TEST_F(RedisSetTest, InterCard) {

Review Comment:
   Additionally, you can add a test for a corner/edge case when one of the sets is empty. And one for the case when there are no common elements in both sets.



##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,42 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit > 0;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {
+        members->emplace_back(member);
+        if (--limit == 0) {
+          limit = -1;

Review Comment:
   Maybe the following code should be considered as a loop break:
   
   ```
   if (has_limit && member_counters[member] == keys.size()) {
         members->emplace_back(member);
         if (members->size() >= limit) {
           break;
         }
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1201874256


##########
src/commands/cmd_set.cc:
##########
@@ -371,6 +418,7 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandSAdd>("sadd", -3, "write", 1, 1, 1),
                         MakeCmdAttr<CommandSDiff>("sdiff", -2, "read-only", 1, -1, 1),
                         MakeCmdAttr<CommandSUnion>("sunion", -2, "read-only", 1, -1, 1),
                         MakeCmdAttr<CommandSInter>("sinter", -2, "read-only", 1, -1, 1),
+                        MakeCmdAttr<CommandSInterCard>("sintercard", -3, "read-only", 1, -1, 1),

Review Comment:
   The best solution here is write a `CommandKeyRangeGen` function, since we have to determine the last position of the key sequence. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199765125


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,55 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *
+ *    syntax: SINTERCARD numkeys key [key ...] [LIMIT limit]
+ *
+ *    params: the checked cardinality is unlimited (limit <= 0).

Review Comment:
   Redis prohibits the negative `LIMIT` parameter:
   
   ```
   127.0.0.1:6379> sintercard 2 set1 set2 limit -5
   (error) ERR LIMIT can't be negative
   127.0.0.1:6379> 
   ```
   
   BTW: right now the phrase `the checked cardinality is unlimited` sounds weird to me. 
   Since `Kvrocks` supports `Redis` protocol, if you want to add some documentation, you can just copy-paste sentences directly from `Redis` docs.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199767092


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,55 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *
+ *    syntax: SINTERCARD numkeys key [key ...] [LIMIT limit]
+ *
+ *    params: the checked cardinality is unlimited (limit <= 0).

Review Comment:
   Thanks, I got it. I just read the docs about this command so I make a mistake.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1564168092

   The PR doesn't have complete tests for wrong parsing msgs  currently. Please wait me for a few days(Sorry, I am moving home).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1204147698


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,65 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *    syntax:   `SINTERCARD numkeys key [key ...] [LIMIT limit]`
+ *
+ *    limit:    the valid limit is an non-negative integer.
+ */
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command: for example, SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4 && util::ToLower(args[numkeys_ + 2]) == "limit") {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 3], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = *parse_limit;
+      if (limit_ < 0) {
+        return {Status::RedisParseErr, errLimitIsNegative};
+      }
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);

Review Comment:
   It seems there is no check for the boundary of `args_`, e.g. for `SINTERCARD 99 a b` will the `Execute` function just crash on this line since `args_[2]` and so on do not exist?



##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,65 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *    syntax:   `SINTERCARD numkeys key [key ...] [LIMIT limit]`
+ *
+ *    limit:    the valid limit is an non-negative integer.
+ */
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command: for example, SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4 && util::ToLower(args[numkeys_ + 2]) == "limit") {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 3], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = *parse_limit;
+      if (limit_ < 0) {
+        return {Status::RedisParseErr, errLimitIsNegative};
+      }
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);

Review Comment:
   It seems there is no check for the boundary of `args_`, e.g. for `SINTERCARD 99 a b` will the `Execute` function just crash on this line since `args_[4]` and so on do not exist?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1204147698


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,65 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *    syntax:   `SINTERCARD numkeys key [key ...] [LIMIT limit]`
+ *
+ *    limit:    the valid limit is an non-negative integer.
+ */
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command: for example, SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4 && util::ToLower(args[numkeys_ + 2]) == "limit") {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 3], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = *parse_limit;
+      if (limit_ < 0) {
+        return {Status::RedisParseErr, errLimitIsNegative};
+      }
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);

Review Comment:
   It seems there is no check for the boundary of `args_`, e.g. for `SINTERCARD 99 a b` will the `Execute` function just crash on this line since `args_[2]` and so on are not exist?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1217430746


##########
tests/cppunit/types/set_test.cc:
##########
@@ -188,9 +188,41 @@ TEST_F(RedisSetTest, Inter) {
   std::vector<std::string> members;
   set_->Inter({k1, k2, k3}, &members);
   EXPECT_EQ(1, members.size());
+  set_->Inter({k1, k2, k4}, &members);
+  EXPECT_EQ(0, members.size());
+  set_->Del(k1);
+  set_->Del(k2);
+  set_->Del(k3);
+  set_->Del(k4);
+}
+
+TEST_F(RedisSetTest, InterCard) {
+  uint64_t ret = 0;
+  std::string k1 = "key1", k2 = "key2", k3 = "key3", k4 = "key4";
+  rocksdb::Status s = set_->Add(k1, {"a", "b", "c", "d"}, &ret);
+  EXPECT_EQ(ret, 4);
+  set_->Add(k2, {"c", "d", "e"}, &ret);
+  EXPECT_EQ(ret, 3);
+  set_->Add(k3, {"e", "f"}, &ret);
+  EXPECT_EQ(ret, 2);
+  set_->InterCard({k1, k2}, 0, &ret);
+  EXPECT_EQ(ret, 2);
+  set_->InterCard({k1, k2}, 1, &ret);
+  EXPECT_EQ(ret, 1);
+  set_->InterCard({k1, k2}, 3, &ret);
+  EXPECT_EQ(ret, 2);
+  set_->InterCard({k2, k3}, 1, &ret);
+  EXPECT_EQ(ret, 1);
+  set_->InterCard({k1, k3}, 5, &ret);
+  EXPECT_EQ(ret, 0);
+  set_->InterCard({k1, k4}, 5, &ret);
+  EXPECT_EQ(ret, 0);
+  set_->InterCard({k1}, 0, &ret);
+  EXPECT_EQ(ret, 4);

Review Comment:
   OK, I'll add more tests later in corner cases. The test for a single key has been updated. Thanks a lot.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] torwig commented on pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1587053050

   @infdahai You didn't actually split the `Inter` function into two separate :) I meant that the `Inter` function should calculate the intersection between sets and return members while `InterCard` should only calculate the cardinality and return the computed value. The command `Inter` will call `Set::Inter` and the command `InterCard` will call the `Set::InterCard` (currently the `Set::InterCard` function calls `Set::Inter` and all the complexity is still in the `Set::Inter` function). In this case, you will get rid of some of the `if` statements and simplify the code.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1204234631


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,65 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *    syntax:   `SINTERCARD numkeys key [key ...] [LIMIT limit]`
+ *
+ *    limit:    the valid limit is an non-negative integer.
+ */
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command: for example, SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4 && util::ToLower(args[numkeys_ + 2]) == "limit") {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 3], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = *parse_limit;
+      if (limit_ < 0) {
+        return {Status::RedisParseErr, errLimitIsNegative};
+      }
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);

Review Comment:
   Thank you so much. So many vulnerabilities. But I have a question about this, `go-redis` compute the `num_key` value and pass the all arguments like `redis` commands to the rpc.
   
   Shall we do this check? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] git-hulk commented on pull request #1444: feat: add sintercard command from redis 7.0

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1561551367

   The flaky test should be caused by timeout, but didn't check the error in this case: https://github.com/apache/incubator-kvrocks/actions/runs/5070341119/jobs/9105869084#step:12:101


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1197795058


##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,42 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit > 0;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {
+        members->emplace_back(member);
+        if (--limit == 0) {
+          limit = -1;

Review Comment:
   The variable `limit` is of type `uint64_t`. How it could be assigned to `-1` and later checked via the condition `limit < 0`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1198098930


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,46 @@ class CommandSInter : public Commander {
   }
 };
 
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command:  SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4) {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 2], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = std::max(0, *parse_limit);
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);
+    }
+
+    redis::Set set_db(svr->storage, conn->GetNamespace());
+    int ret = 0;

Review Comment:
   OK, I get it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1197868818


##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,42 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit > 0;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {
+        members->emplace_back(member);
+        if (--limit == 0) {
+          limit = -1;

Review Comment:
   oh, I forgot this type.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1198084312


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,46 @@ class CommandSInter : public Commander {
   }
 };
 
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command:  SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4) {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 2], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = std::max(0, *parse_limit);
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);
+    }
+
+    redis::Set set_db(svr->storage, conn->GetNamespace());
+    int ret = 0;

Review Comment:
   You can just use `Do()` function of the client to send raw request instead of the command, for example: https://github.com/apache/incubator-kvrocks/blob/unstable/tests/gocase/unit/type/stream/stream_test.go#L45. Or update the package.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199581695


##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,43 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit != 0;
+  bool limited = false;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {
+        members->emplace_back(member);
+        if (--limit == 0) {
+          limited = true;
+          break;
+        }
+      }
     }
+
+    if (limited) break;
   }
-  for (const auto &iter : member_counters) {
-    if (iter.second == keys.size()) {  // all the sets contain this member
-      members->emplace_back(iter.first);
+
+  if (!has_limit) {
+    for (const auto &iter : member_counters) {
+      if (iter.second == keys.size()) {  // all the sets contain this member
+        members->emplace_back(iter.first);
+      }
     }
   }
+
+  return rocksdb::Status::OK();
+}
+
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, int *ret) {

Review Comment:
   We find a few functions with `int *ret` in [redis_set.cc](https://github.com/apache/incubator-kvrocks/blob/unstable/src/types/redis_set.cc). However the result of `ret` is unsinged long(size_t). I will check the correct size in multiple files.



##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,43 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit != 0;
+  bool limited = false;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {
+        members->emplace_back(member);
+        if (--limit == 0) {
+          limited = true;
+          break;
+        }
+      }
     }
+
+    if (limited) break;
   }
-  for (const auto &iter : member_counters) {
-    if (iter.second == keys.size()) {  // all the sets contain this member
-      members->emplace_back(iter.first);
+
+  if (!has_limit) {
+    for (const auto &iter : member_counters) {
+      if (iter.second == keys.size()) {  // all the sets contain this member
+        members->emplace_back(iter.first);
+      }
     }
   }
+
+  return rocksdb::Status::OK();
+}
+
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, int *ret) {

Review Comment:
   We find a few functions with `int *ret` in [redis_set.cc](https://github.com/apache/incubator-kvrocks/blob/unstable/src/types/redis_set.cc). However the result of `ret` is unsinged long(size_t). I will check the correct size type in multiple files.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199583977


##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,43 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit != 0;
+  bool limited = false;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {
+        members->emplace_back(member);
+        if (--limit == 0) {
+          limited = true;
+          break;
+        }
+      }
     }
+
+    if (limited) break;
   }
-  for (const auto &iter : member_counters) {
-    if (iter.second == keys.size()) {  // all the sets contain this member
-      members->emplace_back(iter.first);
+
+  if (!has_limit) {
+    for (const auto &iter : member_counters) {
+      if (iter.second == keys.size()) {  // all the sets contain this member
+        members->emplace_back(iter.first);
+      }
     }
   }
+
+  return rocksdb::Status::OK();
+}
+
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, int *ret) {

Review Comment:
   @PragmaTwice I exec the statement `*output = redis::Integer(ret);` to return the result to the client in cmd_set.cc.
   https://github.com/apache/incubator-kvrocks/blob/fb0e3d42031a6e0691c7a736711d0a01d72bdb1a/src/server/redis_reply.h#L38-L41
   
   Add the `Integer` func checks whether compatibility with `int` is required(Of course, `size_t` is right). I see src/types/*.h and many funcs with this situation use `int`. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199580764


##########
src/types/redis_set.cc:
##########
@@ -338,19 +338,43 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   for (const auto &member : target_members) {
     member_counters[member] = 1;
   }
+
+  bool has_limit = limit != 0;
+  bool limited = false;
   for (size_t i = 1; i < keys.size(); i++) {
     s = Members(keys[i], &target_members);
     if (!s.ok() || target_members.empty()) return s;
     for (const auto &member : target_members) {
-      if (member_counters.find(member) == member_counters.end()) continue;
+      if (member_counters.count(member) == 0) continue;
       member_counters[member]++;
+      if (has_limit && member_counters[member] == keys.size()) {
+        members->emplace_back(member);
+        if (--limit == 0) {
+          limited = true;
+          break;
+        }
+      }
     }
+
+    if (limited) break;
   }
-  for (const auto &iter : member_counters) {
-    if (iter.second == keys.size()) {  // all the sets contain this member
-      members->emplace_back(iter.first);
+
+  if (!has_limit) {
+    for (const auto &iter : member_counters) {
+      if (iter.second == keys.size()) {  // all the sets contain this member
+        members->emplace_back(iter.first);
+      }
     }
   }
+
+  return rocksdb::Status::OK();
+}
+
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, int *ret) {

Review Comment:
   Maybe not related to this PR, but there are two points:
   - `ret` is not a descriptive word
   - `int` type is not fit here: the return variable `members.size()` is typed `size_t`, not `int`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199581200


##########
tests/gocase/unit/type/set/set_test.go:
##########
@@ -282,6 +284,8 @@ func TestSet(t *testing.T) {
 			}
 		})
 
+		t.Run("SINTERCARD against three sets - "+dstype, func(t *testing.T) {})

Review Comment:
   maybe we can add more tests here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] git-hulk commented on pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1564177845

   @infdahai Thanks, no hurry. Can ping back while your PR is ready to review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1575386850

   I was resurrected. Ready to review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1227096570


##########
src/types/redis_set.cc:
##########
@@ -358,6 +358,54 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   return rocksdb::Status::OK();
 }
 
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, uint64_t *cnt) {
+  *cnt = 0;
+
+  std::map<std::string, size_t> member_counters;
+  std::vector<std::string> target_members;
+
+  auto s = Members(keys[0], &target_members);
+  if (!s.ok() || target_members.empty()) return s;
+  for (const auto &member : target_members) {
+    member_counters[member] = 1;
+  }
+  if (limit == 0) {
+    limit = target_members.size();
+  }
+
+  size_t keys_size = keys.size();
+  if (keys_size == 1) {
+    *cnt = std::min(static_cast<uint64_t>(target_members.size()), limit);
+    return rocksdb::Status::OK();
+  }
+
+  int cardinality = 0;
+  bool limit_reached = false;
+  for (size_t i = 1; i < keys_size; i++) {
+    s = Members(keys[i], &target_members);
+    if (!s.ok() || target_members.empty()) {
+      return s;
+    }
+
+    for (const auto &member : target_members) {
+      auto iter = member_counters.find(member);
+      if (iter == member_counters.end()) continue;
+      if (++iter->second == keys_size) {
+        cardinality++;

Review Comment:
   Is there any reason to introduce a new variable `cardinality` and later increment it instead of incrementing the `cnt` variable?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1227111882


##########
src/types/redis_set.cc:
##########
@@ -358,6 +358,54 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   return rocksdb::Status::OK();
 }
 
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, uint64_t *cnt) {
+  *cnt = 0;
+
+  std::map<std::string, size_t> member_counters;
+  std::vector<std::string> target_members;
+
+  auto s = Members(keys[0], &target_members);
+  if (!s.ok() || target_members.empty()) return s;
+  for (const auto &member : target_members) {
+    member_counters[member] = 1;
+  }
+  if (limit == 0) {
+    limit = target_members.size();
+  }
+
+  size_t keys_size = keys.size();
+  if (keys_size == 1) {
+    *cnt = std::min(static_cast<uint64_t>(target_members.size()), limit);
+    return rocksdb::Status::OK();
+  }
+
+  int cardinality = 0;
+  bool limit_reached = false;
+  for (size_t i = 1; i < keys_size; i++) {
+    s = Members(keys[i], &target_members);
+    if (!s.ok() || target_members.empty()) {
+      return s;
+    }
+
+    for (const auto &member : target_members) {
+      auto iter = member_counters.find(member);
+      if (iter == member_counters.end()) continue;
+      if (++iter->second == keys_size) {
+        cardinality++;

Review Comment:
   ignore this. I don't often like accessing pointers.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1198068036


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,46 @@ class CommandSInter : public Commander {
   }
 };
 
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command:  SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4) {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 2], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = std::max(0, *parse_limit);
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);
+    }
+
+    redis::Set set_db(svr->storage, conn->GetNamespace());
+    int ret = 0;

Review Comment:
   >  the go-redis package's version is v9.0.0-beta.2 in the project so it doesn't support this command
   
   So we should pend for updating the go-redis package and then write go tests. 
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1199767092


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,55 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *
+ *    syntax: SINTERCARD numkeys key [key ...] [LIMIT limit]
+ *
+ *    params: the checked cardinality is unlimited (limit <= 0).

Review Comment:
   Thanks, I got it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1554798033

   @git-hulk I haven't finished go tests because `redis-go` because of waiting for https://github.com/apache/incubator-kvrocks/pull/1446.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1201862674


##########
src/commands/cmd_set.cc:
##########
@@ -371,6 +418,7 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandSAdd>("sadd", -3, "write", 1, 1, 1),
                         MakeCmdAttr<CommandSDiff>("sdiff", -2, "read-only", 1, -1, 1),
                         MakeCmdAttr<CommandSUnion>("sunion", -2, "read-only", 1, -1, 1),
                         MakeCmdAttr<CommandSInter>("sinter", -2, "read-only", 1, -1, 1),
+                        MakeCmdAttr<CommandSInterCard>("sintercard", -3, "read-only", 1, -1, 1),

Review Comment:
   @PragmaTwice Would it be better to use this change on the syntax `SINTERCARD numkeys key [key ...] [LIMIT limit]` ? It seems this range is related to `watch`
   ```suggestion
                           MakeCmdAttr<CommandSInterCard>("sintercard", -3, "read-only", 2, -1, 1),
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1201874256


##########
src/commands/cmd_set.cc:
##########
@@ -371,6 +418,7 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandSAdd>("sadd", -3, "write", 1, 1, 1),
                         MakeCmdAttr<CommandSDiff>("sdiff", -2, "read-only", 1, -1, 1),
                         MakeCmdAttr<CommandSUnion>("sunion", -2, "read-only", 1, -1, 1),
                         MakeCmdAttr<CommandSInter>("sinter", -2, "read-only", 1, -1, 1),
+                        MakeCmdAttr<CommandSInterCard>("sintercard", -3, "read-only", 1, -1, 1),

Review Comment:
   The best solution here is write a `CommandKeyRangeGen` function, since we have to determine the last position of the key sequence. 
   
   And of course, the position of the first key is 2.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1560449440

   ready for reviews.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: feat: add sintercard command from redis 7.0

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1204234631


##########
src/commands/cmd_set.cc:
##########
@@ -283,6 +285,65 @@ class CommandSInter : public Commander {
   }
 };
 
+/*
+ * description:
+ *    syntax:   `SINTERCARD numkeys key [key ...] [LIMIT limit]`
+ *
+ *    limit:    the valid limit is an non-negative integer.
+ */
+class CommandSInterCard : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    auto parse_numkey = ParseInt<int>(args[1], 10);
+    if (!parse_numkey) {
+      return {Status::RedisParseErr, errValueNotInteger};
+    }
+    numkeys_ = *parse_numkey;
+
+    // command: for example, SINTERCARD 2 key1 key2 LIMIT 1
+    if (args.size() == numkeys_ + 4 && util::ToLower(args[numkeys_ + 2]) == "limit") {
+      auto parse_limit = ParseInt<int>(args[numkeys_ + 3], 10);
+      if (!parse_limit) {
+        return {Status::RedisParseErr, errValueNotInteger};
+      }
+      limit_ = *parse_limit;
+      if (limit_ < 0) {
+        return {Status::RedisParseErr, errLimitIsNegative};
+      }
+    }
+    return Commander::Parse(args);
+  }
+
+  Status Execute(Server *svr, Connection *conn, std::string *output) override {
+    std::vector<Slice> keys;
+    for (size_t i = 2; i < numkeys_ + 2; i++) {
+      keys.emplace_back(args_[i]);

Review Comment:
   Thank you so much. So many vulnerabilities. But I have a question about this, `go-redis` compute the `num_key` value and pass the all arguments like `redis` commands to the rpc.
   
   Shall we do this check?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#issuecomment-1586173914

   > https://github.com/apache/incubator-kvrocks/actions/runs/5234808336/jobs/9452175246
   > 
   > Do we need to ignore this?
   
   Nope. The build procedure need to succeed in Darwin. It cannot be ignored.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] infdahai commented on a diff in pull request #1444: Add support of the new command SINTERCARD(Redis 7)

Posted by "infdahai (via GitHub)" <gi...@apache.org>.
infdahai commented on code in PR #1444:
URL: https://github.com/apache/incubator-kvrocks/pull/1444#discussion_r1227418437


##########
src/types/redis_set.cc:
##########
@@ -358,6 +358,52 @@ rocksdb::Status Set::Inter(const std::vector<Slice> &keys, std::vector<std::stri
   return rocksdb::Status::OK();
 }
 
+rocksdb::Status Set::InterCard(const std::vector<Slice> &keys, uint64_t limit, uint64_t *cardinality) {
+  *cardinality = 0;
+
+  std::map<std::string, size_t> member_counters;
+  std::vector<std::string> target_members;
+
+  auto s = Members(keys[0], &target_members);
+  if (!s.ok() || target_members.empty()) return s;
+  for (const auto &member : target_members) {
+    member_counters[member] = 1;
+  }
+  if (limit == 0) {
+    limit = target_members.size();
+  }
+
+  size_t keys_size = keys.size();
+  if (keys_size == 1) {
+    *cardinality = std::min(static_cast<uint64_t>(target_members.size()), limit);
+    return rocksdb::Status::OK();
+  }
+
+  bool limit_reached = false;
+  for (size_t i = 1; i < keys_size; i++) {
+    s = Members(keys[i], &target_members);
+    if (!s.ok() || target_members.empty()) {
+      return s;
+    }
+
+    for (const auto &member : target_members) {
+      auto iter = member_counters.find(member);
+      if (iter == member_counters.end()) continue;
+      if (++iter->second == keys_size) {
+        *cardinality++;

Review Comment:
   ```suggestion
           *cardinality += 1;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org