You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@kvrocks.apache.org by GitBox <gi...@apache.org> on 2022/10/23 15:20:51 UTC

[GitHub] [incubator-kvrocks] PragmaTwice opened a new pull request, #1032: Add command parser

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

   The current `CommandParser` is a prototype that adds only a few methods that are really usable at the moment (but does not affect it being merged).
   
   To demonstrate the use of `CommandParser`, I rewrote the `Parse` implementation for a few commands in `redis_cmd.cc`. It is easy to see that the amount of code is massively (multiplied) reduced, and the parsing logic can be expressed in just a few lines of code, especially when complex-syntax commands are encountered.
   
   


-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002849743


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   I think it is hard to keep all error messages same as redis (and there are currently lots of different error message between redis and kvrocks, including the two error message in your comment), and it may make the develop of kvrocks more and unnecessarily complex. And actually I think there is nearly no difference between "syntax error" and "wrong number of arguments", or "encounter non-integer characters" and "not a integer".



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004110808


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > The latter is possible to cause cref a dangling reference.
   
   It is actually not a problem of this macro.
   
   It is a problem for all C++ program, you need to care about lifetime.
   
   You can meet this situation in everywhere of C++, e.g.
   
   ```
   const auto& a = create_string(...).ref_element(0); // aha, dangling reference
   ```
   
   And if you want it to work rather than dangling reference, I can fix it. But I think it is not necessary.



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004115594


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > This seems like an uncommon code snippet to me. How can I read it?
   
   We need a "template" keyword before a template member function. It is not "uncommon". 



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
git-hulk commented on PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#issuecomment-1290419049

   > > In my option, the parser should only care about how to iterator and the type(or range) is right. For whether those flags are exclusive or not, it'd better to handle outside the parser, or the parser will become more and more complex.
   > 
   > In this PR, I added only about 5 lines of code to successfully solve this problem (it is so common in redis command, almost in every redis command with a optional flag), and simplified the code hugely (remove SO MANY duplicated code related to this logic). So I do not think it is unnecessary in the parsing framework.
   > 
   > I think a parser should care about every parsing logic, because every logic is related to whether the parser should move next or hold on.
   
   Yes, the parsing framework truly removes many duplicate codes. My proposition is whether we can reduce the learning cost if we expect all commands depend on it. And for the parsing framework should care about every logic or not, I have no the solid reason now, so I think we can leave as it be.


-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#issuecomment-1296157892

   Hi everyone, any new thoughts on this PR?


-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002849743


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   I think it is hard to keep all error messages same as redis, and it may make the develop of kvrocks more hard. And actually I think there is nearly no difference between "syntax error" and "wrong number of arguments", or "encounter non-integer characters" and "not a integer".



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004108462


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > For value combination, we may implement or use a result type with map/flatmap, or possibly use technology described [here (coroutine_monad)](https://github.com/toby-allsopp/coroutine_monad).
   
   C++17 does not have coroutine, and the technology used in the `coroutine_monad` repo is far more tricky than this macro.
   
   > It won't increase too much code.
   
   There are huge difference between expression and statement, e.g.
   ```c++
   auto x = h(GET_OR_RET(f(GET_OR_RET(g()))));
   ```
   Can you remove `GET_OR_RET` and still keep it simple?



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
git-hulk commented on PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#issuecomment-1290340725

   @PragmaTwice Thanks for your explanation.
   
   > We cannot always move next: for example, to parse (EX v1) | (PX v2) | v3, we need first peek the token (EX or PX), then we can move next, otherwise we may lose v3. For a parser, moving next at every step will severely damage its parsing ability.
   
   Yes, I got your point. What if we use the parser to iterator all tokens instead of only flags. I will take `ZADD` command as example:
   
   ```C++
   while(token = parser.next()) {
     case "NX":
       _flags = nx;
     case "INCR":
       _flags = incr;
     default:
       break;
   }
   while(parse.has_next()) {
      status = parser.expected<double>()
      parse.next()
      status = parser.expected<string>()
   }
   ```
   
   > We need a method to forward error: this is where the sample code is idealized, error handling needs to be abstracted
   
   Yes, it's just a rough idea which didn't think carefully.
   
   > We need a method to prevent different flags in the same layer: for example, to parse [EX a | PX b] | [X | Y], we need to reject something like EX v PX v, X Y or EX v X PX v, and accept EX v EX v, EX v X or Y PX v.
   
   In my option, the parser should only care about how to iterator and the type(or range) is right. For whether those flags are exclusive or not, it'd better to handle outside the parser, or the parser will become more and more complex.
   


-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#issuecomment-1290472131

   @git-hulk There is an [example in unit tests](https://github.com/apache/incubator-kvrocks/blob/85ae20ddff43bb71e8370b7a2b19c51c746b6871/tests/cppunit/command_parser_test.cc#L26) which parses some command in the syntax `[ HELLO i1 v1 | HI v2 ] [X i2 | Y]`, and I think it demonstrate how to use the CommandParser well.


-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002849743


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   I think it is hard to keep all error messages same as redis, and it may make the develop of kvrocks more and unnecessarily complex. And actually I think there is nearly no difference between "syntax error" and "wrong number of arguments", or "encounter non-integer characters" and "not a integer".



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
torwig commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002747022


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   @PragmaTwice Really nice job! 
   These error messages (and others like `ERR wrong number of arguments`) were written that way to be consistent with the `Redis` protocol.
   
   ```
   127.0.0.1:6379> set foo bar ex 1234tyg
   (error) ERR value is not an integer or out of range
   127.0.0.1:6379> set foo bar ex 0
   (error) ERR invalid expire time in 'set' command
   127.0.0.1:6379> 
   ```
   So I'm not sure if it's correct to change them.



##########
src/commands/command_parser.h:
##########
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#pragma once
+
+#include <algorithm>
+#include <cctype>
+#include <functional>
+#include <iterator>
+
+#include "parse_util.h"
+#include "status.h"
+#include "util.h"
+
+template <typename Iter>
+struct MoveIterator : Iter {
+  explicit MoveIterator(Iter iter) : Iter(iter){};
+
+  using Iter::value_type;
+
+  typename Iter::value_type&& operator*() const { return std::move(this->Iter::operator*()); }
+};
+
+template <typename Iter>
+struct CommandParser {
+ public:
+  using value_type = typename Iter::value_type;
+
+  CommandParser(Iter begin, Iter end) : begin(begin), end(end) {}
+
+  template <typename Container>
+  explicit CommandParser(const Container& con, size_t skip_num = 0)
+      : begin(std::begin(con) + skip_num), end(std::end(con)) {}
+
+  template <typename Container>
+  explicit CommandParser(Container&& con, size_t skip_num = 0)
+      : begin(MoveIterator(std::begin(con) + skip_num)), end(MoveIterator(std::end(con))) {}
+
+  decltype(auto) RawPeek() const { return *begin; }
+
+  decltype(auto) RawTake() { return *begin++; }
+
+  decltype(auto) RawNext() { ++begin; }
+
+  bool Good() const { return begin != end; }
+
+  template <typename Pred>
+  bool EatPred(Pred&& pred) {
+    if (Good() && std::forward<Pred>(pred)(RawPeek())) {
+      RawNext();
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  bool EatEqICase(std::string_view str) {
+    return EatPred([str](const auto& v) { return Util::EqualICase(str, v); });

Review Comment:
   In some places, you have `&` near the type but in the `kvrocks` it's usually used near the argument's name. It's just about the style consistency across the project.



##########
src/commands/redis_cmd.cc:
##########
@@ -393,15 +358,18 @@ class CommandGet : public Commander {
 class CommandGetEx : public Commander {
  public:
   Status Parse(const std::vector<std::string> &args) override {
-    white_list_ = {{"persist", false}};
-    auto s = ParseTTL(std::vector<std::string>(args.begin() + 2, args.end()), &white_list_, &ttl_);
-    if (!s.IsOK()) {
-      return s;
-    }
-    if (white_list_["persist"] && args.size() > 3) {
-      return Status(Status::NotOK, errInvalidSyntax);
+    CommandParser parser(args, 2);
+    std::string_view ttl_flag;
+    while (parser.Good()) {
+      if (auto v = GET_OR_RET(ParseTTL(parser, ttl_flag))) {
+        ttl_ = *v;
+      } else if (parser.EatEqICaseFlag("PERSIST", ttl_flag)) {
+        persist_ = true;
+      } else {
+        return parser.InvalidSyntax();
+      }
     }
-    return Commander::Parse(args);
+    return {};

Review Comment:
   Personally, I'd like to see more verbose but explicit `Status::OK()` instead of `{}`.



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004115594


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > This seems like an uncommon code snippet to me. How can I read it?
   
   I cannot see anything "uncommon".



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
git-hulk commented on PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#issuecomment-1290279358

   @PragmaTwice After looking through the PR, I was a bit worry that it needs to take some time (include myself) to understand how to use it and how it works. And for Redis command arguments, there're only three argument types:
   * string
   * bool, like NX/EX/PX and so on
   * number(int/float) like TTL and score
   
   So I'm wondering if we can simplify the parser API like below:
   
   ```C++
   while(token = parse.next()) {
    switch tolower(token):
    case "ex":
       status = parser.expect<int>(&ttl)
    case "px":
       status = parser.expect<int64_t>(&ttl_ms)  
    ...
   }
   ```
   
   So that users can only concern token and what's next is expected.


-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002974048


##########
src/commands/command_parser.h:
##########
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#pragma once
+
+#include <algorithm>
+#include <cctype>
+#include <functional>
+#include <iterator>
+
+#include "parse_util.h"
+#include "status.h"
+#include "util.h"
+
+template <typename Iter>
+struct MoveIterator : Iter {
+  explicit MoveIterator(Iter iter) : Iter(iter){};
+
+  using Iter::value_type;
+
+  typename Iter::value_type&& operator*() const { return std::move(this->Iter::operator*()); }
+};
+
+template <typename Iter>
+struct CommandParser {
+ public:
+  using value_type = typename Iter::value_type;
+
+  CommandParser(Iter begin, Iter end) : begin(begin), end(end) {}
+
+  template <typename Container>
+  explicit CommandParser(const Container& con, size_t skip_num = 0)
+      : begin(std::begin(con) + skip_num), end(std::end(con)) {}
+
+  template <typename Container>
+  explicit CommandParser(Container&& con, size_t skip_num = 0)
+      : begin(MoveIterator(std::begin(con) + skip_num)), end(MoveIterator(std::end(con))) {}
+
+  decltype(auto) RawPeek() const { return *begin; }
+
+  decltype(auto) RawTake() { return *begin++; }
+
+  decltype(auto) RawNext() { ++begin; }
+
+  bool Good() const { return begin != end; }
+
+  template <typename Pred>
+  bool EatPred(Pred&& pred) {
+    if (Good() && std::forward<Pred>(pred)(RawPeek())) {
+      RawNext();
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  bool EatEqICase(std::string_view str) {
+    return EatPred([str](const auto& v) { return Util::EqualICase(str, v); });

Review Comment:
   Oh, I changed it, but clang-format changed it back.



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002849743


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   I think it is hard to keep all error messages same as redis (and there are currently lots of different error message between redis and kvrocks, including the two error message in your comments), and it may make the develop of kvrocks more and unnecessarily complex. And actually I think there is nearly no difference between "syntax error" and "wrong number of arguments", or "encounter non-integer characters" and "not a integer".



-- 
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] tisonkun commented on a diff in pull request #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
tisonkun commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004098746


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   For value combination, we may implement or use a `result` type with `map`/`flatmap`, or possibly use technology described [here (coroutine_monad)](https://github.com/toby-allsopp/coroutine_monad).



##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   According to https://gcc.gnu.org/onlinedocs/gcc/Statement-Exprs.html:
   
   > In G++, the result value of a statement expression undergoes array and function pointer decay, and is returned by value to the enclosing expression. For instance, if A is a class, then constructs a temporary A object to hold the result of the statement expression, and that is used to invoke Foo. Therefore the this pointer observed by Foo is not the address of a.
   
   Besides, comparing:
   
   ```c++
   const auto &cref = function(xxx);
   const auto &cref = GET_OR_RET(xxx);
   ```
   
   The latter is possible to cause `cref` a dangling reference.
   
   If we inline this trick, said for:
   
   ```c++
   ttl_ = GET_OR_RET(parser.TakeInt<int>(TTL_RANGE<int>));
   ```
   
   to
   
   ```c++
   res = parser.TakeInt<int>(TTL_RANGE<int>);
   if (!res) return res;
   ttl_ = res.GetValue();
   ```
   
   It won't increase too much code.



##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   Besides, I cannot read:
   
   ```c++
   return GET_OR_RET(parser.template TakeInt<int>(TTL_RANGE<int>));
   ```
   
   This seems like an uncommon code snippet to me. How can I read 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] PragmaTwice commented on a diff in pull request #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004108462


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > For value combination, we may implement or use a result type with map/flatmap, or possibly use technology described [here (coroutine_monad)](https://github.com/toby-allsopp/coroutine_monad).
   
   C++17 does not have coroutine, and the technology used in the `coroutine_monad` repo is far more tricky than this macro.
   
   > It won't increase too much code.
   
   There are huge difference between expression and statement, e.g.
   ```c++
   auto x = h(GET_OR_RET(f(GET_OR_RET(g()))));
   ```
   Can you remove `GET_OR_RET` and still keep simple?



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#issuecomment-1290298720

   > @PragmaTwice After looking through the PR, I was a bit worry that it needs to take some time for most developers(include myself) to understand how to use it and how it works. And for Redis command arguments, there're only three argument types:
   > 
   > * string
   > * bool, like NX/EX/PX and so on
   > * number(int/float) like TTL and score
   > 
   > So I'm wondering if we can simplify the parser API like below:
   > 
   > ```c++
   > while(token = parse.next()) {
   >  switch tolower(token):
   >  case "ex":
   >     status = parser.expect<int>(&ttl)
   >  case "px":
   >     status = parser.expect<int64_t>(&ttl_ms)  
   >  ...
   > }
   > ```
   > 
   > So that users can only concern token and what's next is expected.
   
   I think, there are lots of problem we need to handle:
   - We cannot always move `next`: for example, to parse `(EX v1) | (PX v2) | v3`, we need first `peek` the token (`EX` or `PX`), then we can move `next`, otherwise we may lose `v3`
   - We need a method to forward error: this is where the sample code is idealized, error handling needs to be abstracted
   - We need a method to prevent different flags in the same layer: for example, to parse `[EX a | PX b] | [X | Y]`, we need to reject something like `EX v PX v`, `X Y` or `EX v X PX v`, and accept `EX v EX v`, `EX v X` or `Y PX v`.
   
   
   Simplifying code means doing good abstraction, and of course good abstraction has a learning cost, but I still feel that the current abstraction is intuitive:
   - `parser.Good()`: to check if there is still element remain to parse
   - `parser.EatICaseFlag(str, flag)`: to match a specific flag token, move next while sucessful. It can be learned from [this example](https://github.com/apache/incubator-kvrocks/blob/85ae20ddff43bb71e8370b7a2b19c51c746b6871/tests/cppunit/command_parser_test.cc#L26).
   - `parser.TakeInt()` or `parser.TakeStr()`: to eat a new integer or string


-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002849743


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   I think it is hard to keep all error messages same as redis, and it may make the develop of kvrocks more complex. And actually I think there is nearly no difference between "syntax error" and "wrong number of arguments", or "encounter non-integer characters" and "not a integer".



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002849743


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   I think it is hard to keep all error messages same as redis (and there are currently lots of different error message between redis and kvrocks, **including the two error message in your comment**), and it may make the develop of kvrocks more and unnecessarily complex. And actually I think there is nearly no difference between "syntax error" and "wrong number of arguments", or "encounter non-integer characters" and "not a integer".



##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   I think it is hard to keep all error messages same as redis (and there are currently lots of different error message between redis and kvrocks, **including the two error message in your comments**), and it may make the develop of kvrocks more and unnecessarily complex. And actually I think there is nearly no difference between "syntax error" and "wrong number of arguments", or "encounter non-integer characters" and "not a integer".



-- 
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 merged pull request #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice merged PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032


-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004110808


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > The latter is possible to cause cref a dangling reference.
   
   It is actually not a problem of this macro.
   
   It is a problem for all C++ program, you need to care about lifetime.
   
   You can meet this situation in everywhere of C++, e.g.
   
   ```c++
   const auto& a = create_string(...).ref_element(0); // aha, dangling reference
   ```
   
   And if you want it to work rather than dangling reference, I can fix it. But I think it is not necessary.



-- 
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] tisonkun commented on a diff in pull request #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
tisonkun commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004432172


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   You convince me. My final comment on this stuff is whether we can name it `STATUS_UNWRAP` or simply `UNWRAP`.



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
torwig commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1007274247


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   Yes, you are right, it's not easy to keep identical error messages. Since Redis doesn't have error codes, I was wondering if Redis-clients parse error messages to get something useful from them or just signal error/no-error? Does error message be considered a part of Redis-protocol?



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#issuecomment-1293638516

   > To be honest, I'm still a bit hard understanding the implementation well(maybe I should learn more about C++ templates), especially in the part about the exclusive flag. I'm very happy to see this push forward if other folks feel good.
   
   I think if the API is clear, intuitive and easy to understand enough, then maybe developers do not need to care and understand the implementation details.
   A classic redis command parsing scenario is demonstrated in this code snippet below, and it can be seen that the parsing process is relatively intuitive.
   
   https://github.com/apache/incubator-kvrocks/blob/85ae20ddff43bb71e8370b7a2b19c51c746b6871/tests/cppunit/command_parser_test.cc#L35-L50


-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004110808


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > The latter is possible to cause cref a dangling reference.
   
   It is actually not a problem of this macro.
   
   It is a problem for all C++ program, you need to care about lifetime.



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004434705


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   I think `unwrap` in rust means that, if it has a value, return it, otherwise just abort. But here we will not abort the program, just return the error status to the upper layer. So I think it is not appropriate.



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002849743


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   I think it is hard to keep all error messages same as redis (and there are currently lots of different error message between redis and kvrocks), and it may make the develop of kvrocks more and unnecessarily complex. And actually I think there is nearly no difference between "syntax error" and "wrong number of arguments", or "encounter non-integer characters" and "not a integer".



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
git-hulk commented on PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#issuecomment-1291323604

   > mand in the syntax `[ HELLO i1 v1 | HI v2 ] [X i2 | Y]` (where `i1` `i2` are integers and `v1` `v2` are strings), and I think it demonstrate how to use the CommandParser well. I think from this example, the interface provided by the current framework can be quickly understood.
   
   @PragmaTwice Thank you! I'll have another pass 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] PragmaTwice commented on a diff in pull request #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004108462


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > For value combination, we may implement or use a result type with map/flatmap, or possibly use technology described [here (coroutine_monad)](https://github.com/toby-allsopp/coroutine_monad).
   
   C++17 does not have coroutine, and the technology used in the `coroutine_monad` repo is far more tricky than this macro.
   
   > It won't increase too much code.
   There are huge difference between expression and statement, e.g.
   ```c++
   auto x = h(GET_OR_RET(f(GET_OR_RET(g()))));
   ```
   Can you remove `GET_OR_RET` and still keep simple?



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004110808


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > The latter is possible to cause cref a dangling reference.
   
   It is actually not a problem of this macro.
   
   It is a problem for all C++ program, you need to care about lifetime.
   
   You can meet this situation in everywhere of C++, e.g.
   
   ```
   const auto& a = std::string("...").at(0); // aha, dangling reference
   ```



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002974048


##########
src/commands/command_parser.h:
##########
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#pragma once
+
+#include <algorithm>
+#include <cctype>
+#include <functional>
+#include <iterator>
+
+#include "parse_util.h"
+#include "status.h"
+#include "util.h"
+
+template <typename Iter>
+struct MoveIterator : Iter {
+  explicit MoveIterator(Iter iter) : Iter(iter){};
+
+  using Iter::value_type;
+
+  typename Iter::value_type&& operator*() const { return std::move(this->Iter::operator*()); }
+};
+
+template <typename Iter>
+struct CommandParser {
+ public:
+  using value_type = typename Iter::value_type;
+
+  CommandParser(Iter begin, Iter end) : begin(begin), end(end) {}
+
+  template <typename Container>
+  explicit CommandParser(const Container& con, size_t skip_num = 0)
+      : begin(std::begin(con) + skip_num), end(std::end(con)) {}
+
+  template <typename Container>
+  explicit CommandParser(Container&& con, size_t skip_num = 0)
+      : begin(MoveIterator(std::begin(con) + skip_num)), end(MoveIterator(std::end(con))) {}
+
+  decltype(auto) RawPeek() const { return *begin; }
+
+  decltype(auto) RawTake() { return *begin++; }
+
+  decltype(auto) RawNext() { ++begin; }
+
+  bool Good() const { return begin != end; }
+
+  template <typename Pred>
+  bool EatPred(Pred&& pred) {
+    if (Good() && std::forward<Pred>(pred)(RawPeek())) {
+      RawNext();
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  bool EatEqICase(std::string_view str) {
+    return EatPred([str](const auto& v) { return Util::EqualICase(str, v); });

Review Comment:
   Oh, I changed it, but then clang-format changed it back.



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
git-hulk commented on PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#issuecomment-1292163770

   To be honest, I'm still a bit hard understanding the implementation well(maybe I should learn more about C++ templates), especially in the part about the exclusive flag. I'm very happy to see this push forward if other folks feel good.


-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#issuecomment-1296271576

   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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002849743


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   I think it is hard to keep all error messages same as redis, and it may make the develop of kvrocks more hard. And actually I think there is nearly no difference between "syntax error" and "wrong number of arguments".



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1002973403


##########
src/commands/command_parser.h:
##########
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#pragma once
+
+#include <algorithm>
+#include <cctype>
+#include <functional>
+#include <iterator>
+
+#include "parse_util.h"
+#include "status.h"
+#include "util.h"
+
+template <typename Iter>
+struct MoveIterator : Iter {
+  explicit MoveIterator(Iter iter) : Iter(iter){};
+
+  using Iter::value_type;
+
+  typename Iter::value_type&& operator*() const { return std::move(this->Iter::operator*()); }
+};
+
+template <typename Iter>
+struct CommandParser {
+ public:
+  using value_type = typename Iter::value_type;
+
+  CommandParser(Iter begin, Iter end) : begin(begin), end(end) {}
+
+  template <typename Container>
+  explicit CommandParser(const Container& con, size_t skip_num = 0)
+      : begin(std::begin(con) + skip_num), end(std::end(con)) {}
+
+  template <typename Container>
+  explicit CommandParser(Container&& con, size_t skip_num = 0)
+      : begin(MoveIterator(std::begin(con) + skip_num)), end(MoveIterator(std::end(con))) {}
+
+  decltype(auto) RawPeek() const { return *begin; }
+
+  decltype(auto) RawTake() { return *begin++; }
+
+  decltype(auto) RawNext() { ++begin; }
+
+  bool Good() const { return begin != end; }
+
+  template <typename Pred>
+  bool EatPred(Pred&& pred) {
+    if (Good() && std::forward<Pred>(pred)(RawPeek())) {
+      RawNext();
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  bool EatEqICase(std::string_view str) {
+    return EatPred([str](const auto& v) { return Util::EqualICase(str, v); });

Review Comment:
   done



##########
src/commands/redis_cmd.cc:
##########
@@ -393,15 +358,18 @@ class CommandGet : public Commander {
 class CommandGetEx : public Commander {
  public:
   Status Parse(const std::vector<std::string> &args) override {
-    white_list_ = {{"persist", false}};
-    auto s = ParseTTL(std::vector<std::string>(args.begin() + 2, args.end()), &white_list_, &ttl_);
-    if (!s.IsOK()) {
-      return s;
-    }
-    if (white_list_["persist"] && args.size() > 3) {
-      return Status(Status::NotOK, errInvalidSyntax);
+    CommandParser parser(args, 2);
+    std::string_view ttl_flag;
+    while (parser.Good()) {
+      if (auto v = GET_OR_RET(ParseTTL(parser, ttl_flag))) {
+        ttl_ = *v;
+      } else if (parser.EatEqICaseFlag("PERSIST", ttl_flag)) {
+        persist_ = true;
+      } else {
+        return parser.InvalidSyntax();
+      }
     }
-    return Commander::Parse(args);
+    return {};

Review Comment:
   done



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004110808


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > The latter is possible to cause cref a dangling reference.
   
   It is actually not a problem of this macro.
   
   It is a problem for all C++ program, you need to care about lifetime.
   
   You can meet this situation in everywhere of C++, e.g.
   
   ```
   const auto& a = create_string("...").ref_element(0); // aha, dangling reference
   ```



##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > The latter is possible to cause cref a dangling reference.
   
   It is actually not a problem of this macro.
   
   It is a problem for all C++ program, you need to care about lifetime.
   
   You can meet this situation in everywhere of C++, e.g.
   
   ```
   const auto& a = create_string(...).ref_element(0); // aha, dangling reference
   ```



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004110808


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > The latter is possible to cause cref a dangling reference.
   
   It is actually not a problem of this macro.
   
   It is a problem for all C++ program, you need to care about lifetime.
   
   You can meet this situation in everywhere of C++, e.g.
   
   ```
   const auto& a = create_string(...).ref_element(0); // aha, dangling reference
   ```
   
   And if you want it to work rather than dangling reference, I can fix 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] PragmaTwice commented on a diff in pull request #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1004110808


##########
src/common/status.h:
##########
@@ -285,3 +282,10 @@ struct StatusOr {
   template <typename>
   friend struct StatusOr;
 };
+
+#define GET_OR_RET(...)                                         \
+  ({                                                            \
+    auto&& status = (__VA_ARGS__);                              \
+    if (!status) return std::forward<decltype(status)>(status); \
+    std::forward<decltype(status)>(status);                     \
+  }).GetValue()

Review Comment:
   > The latter is possible to cause cref a dangling reference.
   
   It is actually not a problem of this macro.
   
   It is a problem for all C++ program, you need to care about lifetime.
   
   You can meet this situation in everywhere of C++, e.g.
   
   ```
   const auto& a = std::string("...").ref_element(0); // aha, dangling reference
   ```



-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#issuecomment-1290355040

   > In my option, the parser should only care about how to iterator and the type(or range) is right. For whether those flags are exclusive or not, it'd better to handle outside the parser, or the parser will become more and more complex.
   
   In this PR, I added only about 5 lines of code to successfully solve this problem (it is so common in redis command, almost in every redis command with a optional flag), and simplified the code hugely (remove lots of duplicated code related to this logic).
   So I do not think it is unnecessary in the parsing framework.


-- 
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 #1032: Add command parser

Posted by GitBox <gi...@apache.org>.
PragmaTwice commented on code in PR #1032:
URL: https://github.com/apache/incubator-kvrocks/pull/1032#discussion_r1007571271


##########
tests/gocase/unit/type/strings/strings_test.go:
##########
@@ -605,12 +605,12 @@ func TestString(t *testing.T) {
 	})
 
 	t.Run("Extended SET with incorrect expire value", func(t *testing.T) {
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "exat", "0").Err(), "invalid expire time")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "1234xyz").Err(), "not an integer")
-		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "pxat", "0").Err(), "invalid expire time")
+		require.ErrorContains(t, rdb.Do(ctx, "SET", "foo", "bar", "ex", "1234xyz").Err(), "non-integer")

Review Comment:
   I think it is not possible to be compatible to all redis error, since there are already many errors that are different than redis, some of which are kvrocks-only error. So if they parse them, they cannot get the right message. And redis does not guarantee that they will keep old error message in new version, so I do not think it is necessary to keep error message identical to redis.



-- 
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