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/08/03 07:54:57 UTC

[GitHub] [incubator-kvrocks] PragmaTwice opened a new pull request, #768: Add `StatusOr` for error handling in modern C++ style

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

   In kvrocks we use a style like `Status process(input..., T* output)` to handle errors. It works well, but in C++ we can do better via constructing type-safe union types.
   
   In this PR, we propose a new class template `StatusOr<T>`, which contains either a value typed T or a status. Here is a simple example:
   
   ```c++
   StatusOr<std::string> hello(std::string x) {
     if (x.size() > 10) {
       return {Status::NotOK, "string too long"};
     }
   
     return x + " hello";
   };
   
   StatusOr<std::string> hi(std::string x) {
     if (x.size() < 5) {
       return {Status::NotOK, "string too short"};
     }
   
     auto res = f(x);
     if (!res) return res;
   
     return "hi " + *res;
   };
   
   assert(*hi("twice") == "hi twice hello");
   
   auto short = hi("x");
   assert(!short && short.Msg() == "string too short");
   
   auto long= hi("xxxxxxxxxxx");
   assert(!long && long.Msg() == "string too long");
   ```
   
   We maximize the use of move semantics in C++ to eliminate redundant copies and optimize the storage, so that developers do not need to worry too much about its performance when using 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] caipengbo commented on pull request #768: Add `StatusOr` for error handling in modern C++ style

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

   This approach may be a more modern C++ approach, but I don't think it's very friendly for a rookie like me. It may take me some time to get familiar with how to use `StatusOr` and what the implementation means, which may not be very friendly for a beginner like me to contribute code. RocksDB as the storage engine, also uses the `Status` without major performance issues, so I think `Status` is better.


-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   Hi @tisonkun, thanks for your review.
   
   > I wonder if we can generate a compile time error instead of runtime fatal in this case.
   
   Good idea. We consider an example for this:
   ```c++
   auto res = someProcess(...); // get result typed StatusOr<T>
   // we cannot deref `res` here
   
   if(!res) { 
     processError(res.GetCode(), res.Msg()); /* or */ return res; // we cannot deref `res` here
   }
   
   doSomething(*res); // we can deref it
   ```
   
   In this example, only the `doSomething` part can actually dereference the `res`. 
   We cannot dereference it both before `if(!res)` and inside `if(!res)`.
   So only in the execution path with the condition `res.IsOk() == true` being satisfied, we can dereference `res`,
   
   It is hardly to express in compile time, since it is related to the control flow.
   Like a raw/smart pointer, we should check it does point to a value (not null) before dereference 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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   `GetValue()` will trigger a fatal error.
   It is just like you call `f` as `Status f(int *output)`, but do not check the returned `Status` and directly dereference the `output`.



-- 
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] jackwener commented on pull request #768: Add `StatusOr` for error handling in modern C++ style

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

   Agree with use `StatusOr`
   
   - No extensive code changes required
   - It's easy to use and more intuitive
   - Morden C++ style
   
   `StatusOr` is also used by `folly` which is used wildly.


-- 
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 merged pull request #768: Add `StatusOr` for error handling in modern C++ style

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


-- 
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] ShooterIT commented on a diff in pull request #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   what would happen if calling `f(11).GetValue()`?



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   Hi @tisonkun, thanks for your review.
   
   > I wonder if we can generate a compile time error instead of runtime fatal in this case.
   
   Good idea. We consider an example for this:
   ```c++
   auto res = someProcess(...); // get result typed StatusOr<T>
   // we cannot deref `res` here
   
   if(!res) { 
     processError(res.GetCode(), res.Msg()); /* or */ return res; // we cannot deref `res` here
   }
   
   doSomething(*res); // we can deref it
   ```
   
   In this example, only the `doSomething` part can actually dereference the `res`. 
   We cannot dereference it both before `if(!res)` and inside `if(!res)`.
   So only in the execution path with the condition `res.IsOk() == true` being satisfied, we can dereference `res`,
   
   It is hardly to express in compile time, since it is related to the control flow.
   Like a raw/smart pointer, we should check it does point to a value (not null) before dereference it.
   
   I have implemented a checker in Clang Static Analyzer that checks whether a type like `StatusOr` is dereferenced after checking `IsOk()` using [symbolic execution](https://en.wikipedia.org/wiki/Symbolic_execution), and I think it can be done in this way in the future.



-- 
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 a diff in pull request #768: Add `StatusOr` for error handling in modern C++ style

Posted by GitBox <gi...@apache.org>.
git-hulk commented on code in PR #768:
URL: https://github.com/apache/incubator-kvrocks/pull/768#discussion_r937358592


##########
src/status.h:
##########
@@ -61,20 +66,204 @@ class Status {
   };
 
   Status() : Status(cOK) {}
-  explicit Status(Code code, std::string msg = {}) : code_(code), msg_(std::move(msg)) {}
-  bool IsOK() { return code_ == cOK; }
-  bool IsNotFound() { return code_ == NotFound; }
-  bool IsImorting() { return code_ == SlotImport; }
-  bool IsBlockingCommand() { return code_ == BlockingCmd; }
-  std::string Msg() {
-    if (IsOK()) {
-      return "ok";
-    }
+  explicit Status(Code code, std::string msg = {})
+    : code_(code), msg_(std::move(msg)) {}
+
+  template <Code code>
+  bool Is() const { return code_ == code; }
+
+  bool IsOK() const { return Is<cOK>(); }
+  operator bool() const { return IsOK(); }
+
+  Code GetCode() const {
+    return code_;
+  }
+
+  std::string Msg() const& {
+    if (*this) return ok_msg();
     return msg_;
   }
+
+  std::string Msg() && {
+    if (*this) return ok_msg();
+    return std::move(msg_);
+  }
+
   static Status OK() { return {}; }
 
  private:
   Code code_;
   std::string msg_;
+
+  static constexpr const char* ok_msg() {
+    return "ok";
+  }
+
+  template <typename T>
+  friend struct StatusOr;
+};
+
+template <typename ...Ts>
+using first_element = typename std::tuple_element<0, std::tuple<Ts...>>::type;
+
+template <typename T>
+using remove_cvref_t = typename std::remove_cv<typename std::remove_reference<T>::type>::type;
+
+template <typename T>
+struct StatusOr;
+
+template <typename T>
+struct IsStatusOr : std::integral_constant<bool, false> {};
+
+template <typename T>
+struct IsStatusOr<StatusOr<T>> : std::integral_constant<bool, true> {};
+
+template <typename T>
+struct StatusOr {
+  static_assert(!std::is_same<T, Status>::value, "value_type cannot be Status");
+  static_assert(!std::is_same<T, Status::Code>::value, "value_type cannot be Status::Code");
+  static_assert(!IsStatusOr<T>::value, "value_type cannot be StatusOr");
+  static_assert(!std::is_reference<T>::value, "value_type cannot be reference");
+
+  using value_type = T;
+
+  // we use std::unique_ptr to make the error part as small as enough
+  using error_type = std::unique_ptr<std::string>;
+
+  using Code = Status::Code;
+
+  explicit StatusOr(Status s) : code_(s.code_) {
+    CHECK(!s);
+    new(storage_) error_type(new std::string(std::move(s.msg_)));
+  }
+
+  StatusOr(Code code, std::string msg = {}) : code_(code) { // NOLINT
+    CHECK(code != Code::cOK);
+    new(storage_) error_type(new std::string(std::move(msg)));
+  }
+
+  template <typename ...Ts,
+    typename std::enable_if<
+      (sizeof...(Ts) > 0 &&
+        !std::is_same<Status, remove_cvref_t<first_element<Ts...>>>::value &&
+        !std::is_same<Code, remove_cvref_t<first_element<Ts...>>>::value &&
+        !std::is_same<value_type, remove_cvref_t<first_element<Ts...>>>::value &&
+        !std::is_same<StatusOr, remove_cvref_t<first_element<Ts...>>>::value
+      ), int>::type = 0> // NOLINT
+  explicit StatusOr(Ts && ... args) : code_(Code::cOK) {
+    new(storage_) value_type(std::forward<Ts>(args)...);
+  }
+
+  StatusOr(T&& value) : code_(Code::cOK) { // NOLINT
+    new(storage_) value_type(std::move(value));
+  }
+
+  StatusOr(const T& value) : code_(Code::cOK) { // NOLINT
+    new(storage_) value_type(value);
+  }
+
+  StatusOr(const StatusOr&) = delete;
+  StatusOr(StatusOr&& other) : code_(other.code_) {
+    if (code_ == Code::cOK) {
+      new(storage_) value_type(std::move(other.getValue()));
+    } else {
+      new(storage_) error_type(std::move(other.getError()));
+    }
+  }
+
+  Status& operator=(const Status&) = delete;
+
+  template <Code code>
+  bool Is() const { return code_ == code; }
+
+  bool IsOK() const { return Is<Code::cOK>(); }
+  operator bool() const { return IsOK(); }
+
+  Status ToStatus() const& {
+    if (*this) return Status::OK();
+    return Status(code_, *getError());
+  }
+
+  Status ToStatus() && {
+    if (*this) return Status::OK();
+    return Status(code_, std::move(*getError()));
+  }
+
+  Code GetCode() const {
+    return code_;
+  }
+
+  value_type& GetValue() & {
+    CHECK(*this);
+    return getValue();
+  }
+
+  value_type&& GetValue() && {
+    CHECK(*this);
+    return std::move(getValue());
+  }
+
+  const value_type& GetValue() const& {
+    CHECK(*this);
+    return getValue();
+  }
+
+  value_type& operator*() & {
+    return GetValue();
+  }
+
+  value_type&& operator*() && {
+    return std::move(GetValue());
+  }
+
+  const value_type& operator*() const& {
+    return GetValue();
+  }
+
+  value_type* operator->() {
+    return &GetValue();
+  }
+
+  const value_type* operator->() const {
+    return &GetValue();
+  }
+
+  std::string Msg() const& {
+    if (*this) return Status::ok_msg();
+    return *getError();
+  }
+
+  std::string Msg() && {
+    if (*this) return Status::ok_msg();
+    return std::move(*getError());
+  }
+
+  ~StatusOr() {
+    if (*this) {
+      getValue().~value_type();
+    } else {
+      getError().~error_type();
+    }
+  }
+
+ private:
+  Status::Code code_;
+  alignas(value_type) alignas(error_type) unsigned char storage_

Review Comment:
   Is it a good idea to use `value_` instead of `storage_` here? coz we used the `storage_` heavily in `storage.cc` but it's totally different. So I'm wondering if this will
   confuse others.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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

   Yes, I also think `StatusOr` is more intuitive than before


-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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

   > I wonder why kvrocks' Status uses std::string, it would be large, likely 24B on stack
   
   To be honest, we didn't think carefully about this case before, because AFAIK the bottleneck
   of Kvrocks is commonly at RocksDB or disk io latency. Of course, I think it will be good to optimize as much as possible if it can keep the code base simple at the same time.
   
   
   > Secondly, sometimes when we want to find the root cause of the problem, status make it hard to debug. apache doris thinks using exception is a better method: https://github.com/apache/doris/discussions/8406
   
   I guess the reason why no one mentioned this issue: call the chain of Kvrocks is simple engouh: `command->data structure->rocksdb`, so we can tell the root cause even without exception stacks. Correct me if others don't agree with me.
   


-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   Hi @ShooterIT, if you look at the definition of `StatusOr<T>::GetValue`:
   
   ```c++
     value_type& GetValue() & {
       CHECK(*this);
       return getValue();
     }
   
     value_type&& GetValue() && {
       CHECK(*this);
       return std::move(getValue());
     }
   
     const value_type& GetValue() const& {
       CHECK(*this);
       return getValue();
     }
   ```
   
   You will find there is three different overload of this method via changing the qualifier of `*this`.
   
   Hence, in `f(x).GetValue()`, obviously `f(x)` is a [prvalue](https://en.cppreference.com/w/cpp/language/value_category#prvalue) since a temporary object is returned, and then the signature `value_type&& GetValue() &&` is decided to be called after the [overload resolution](https://en.cppreference.com/w/cpp/language/overload_resolution), and thus you get an [xvalue](https://en.cppreference.com/w/cpp/language/value_category#xvalue) (which is a special kind of rvalue) referenced to the underlying value of the returned `StatusOr`.
   
   So we should take care of the lifetime in this case, the rvalue will become invalid after the computation of the current statement. We can extend it via a call to a move ctor (`T(T&&)`). This is not a defect of this type, but it is an idiom of C++, we should always care about lifetime of references to avoid dangling references in all standards of C++ (98/03/11/14/17/20 ...).



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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

   Hi @ShooterIT, your understanding is generally correct. I'm still holding on to the hope that we'll be able to merge this PR, which I state below on various fronts.
   
   First, the original error handling method has several flaws:
   - the pointer passed in to store the result is readable, C++ doesn't have any flag to indicate that a pointer is write-only: this causes semantic confusion, in fact, we don't need the value the pointer is currently pointing to, this value is garbage when the result is not written;
   - the pointer passed in to store the result points to an already constructed object, which is usually default-constructed: this prevents us from constructing an object after processing (probably not via the default constructor), which It's a very popular way of doing things in C++.
   - we cannot use `auto` to deduce the output type because we have to construct it before calling the function.
   
   Second, `StatusOr` is designed to be very intuitive. This type has two states, one in which it stores a value representing the result, and one in which it stores an error (including an error code and error message). I always feel that `StatusOr` will be easier to use than the current form, and we can more intuitively express what we want to do.
   
   Lastly, I have no current plans to replace the parts of the current project that use `Status`, and I've even modified `Status` in this PR to make better use of move semantics in certain situation. Anyone can continue to use the previous form for error handling, it's just that I'll use it to make it easier to build some command parsing related 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] tisonkun commented on a diff in pull request #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   > checks whether a type like StatusOr is dereferenced after checking IsOk() using [symbolic execution](https://en.wikipedia.org/wiki/Symbolic_execution)
   
   Sounds good tome.



##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   > checks whether a type like StatusOr is dereferenced after checking IsOk() using [symbolic execution](https://en.wikipedia.org/wiki/Symbolic_execution)
   
   Sounds good to me.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   `GetValue()` will trigger a fatal error.
   It is just like you call `f` as `Status f(int *output)`, but do not check the returned `Status` and directly dereference the `output`. It is not allowed.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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

   To be honest, this change looks a bit complex at first glance. 
   But after taking a look at how the Status/StatusOr works and uses, 
   I think it can simplify a lot on how we add a new status code
   and return a status with value. 
   
   For myself, I would happy to see that we can use the modern way
   to improve our codebase, even it needs some time to learn for guys
   from C or legacy C++ code style.


-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   BTW, The classic usage of `StatusOr<T>` we expected is:
   ```
   // 1
   StatusOr<T> someProcess(...); // as a return type
   
   // 2
   T val;
   ...
   return val; // return val typed T to StatusOr<T>
   
   // 3
   return {Status::someErrCode, "msg..."} // return an error status
   
   // 4
   auto res = someProcess(...); // get result typed StatusOr<T>
   if(!res) return res; // forward it if it is an error status
   
   doSomething(*res); // process the value inside the StatusOr
   ```



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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

   > @PragmaTwice
   > 
   > If using exception to report error instead of status, we don't need Status and this StatusOr at all.
   > 
   > What is the better way to report error in C++? error code or exception? Here are same posts to discuss this question:
   > 
   > * https://isocpp.org/wiki/faq/exceptions
   > * https://stackoverflow.com/questions/253314/conventions-for-exceptions-or-error-codes
   
   Hi @wy-ei , a large part of C++ projects do not use exception as the mainly error handling method (but use it while some fatal or very accidental error occurred) since exception is relatively expensive in bad path (or say, failure path, maybe x10 - x20 slower than conditional jumping, but zero-cost on happy path)[1-2], like all Google products, LLVM projects, etc.
   
   Another defect is that C++ does not have checked exception, this makes exception flows not easy to maintain. So I don't have a strong desire to use exceptions.
   
   [1] https://pspdfkit.com/blog/2020/performance-overhead-of-exceptions-in-cpp
   [2] https://www.open-std.org/jtc1/sc22/wg21/docs/papers/2022/p2544r0.html


-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   BTW, The classic usage of `StatusOr<T>` we expected is:
   ```c++
   // 1
   StatusOr<T> someProcess(...); // as a return type
   
   // 2
   T val;
   ...
   return val; // return val typed T to StatusOr<T>
   
   // 3
   return {Status::someErrCode, "msg..."}; // return an error status
   
   // 4
   auto res = someProcess(...); // get result typed StatusOr<T>
   if(!res) return res; // forward it if it is an error status
   
   doSomething(*res); // process the value inside the StatusOr
   ```
   Use `StatusOr` as an rvalue is totally (well) supported, but everyone should be aware of what they are doing.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   Hi @tisonkun, thanks for your review.
   
   > I wonder if we can generate a compile time error instead of runtime fatal in this case.
   
   Good idea. We consider an example for this:
   ```c++
   auto res = someProcess(...); // get result typed StatusOr<T>
   // we cannot deref `res` here
   
   if(!res) { 
     processError(res.GetCode(), res.Msg()); /* or */ return res; // we cannot deref `res` here
   }
   
   doSomething(*res); // we can deref it
   ```
   
   In this example, only the `doSomething` part can actually dereference the `res`. 
   We cannot dereference it both before `if(!res)` and inside `if(!res)`.
   So only in the execution path with the condition `res.IsOk() == true` being satisfied, we can dereference `res`,
   
   It is hardly to express in compile time, since it is related to the control flow.
   Like a raw/smart pointer, we should check it does point to a value (not null) before dereference it.
   
   I have implemented a checker in Clang Static Analyzer for OneFlow that checks whether a type like `StatusOr` is dereferenced after checking `IsOk()` using [symbolic execution](https://en.wikipedia.org/wiki/Symbolic_execution), and I think it can be done in this way in the future.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   `GetValue()` will trigger a fatal error.
   It is just like you call `f` as `Status f(int *output)`, but do not check the returned `Status` and directly dereference the `output`. It is not allowed, you must check status before dereference.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   If you look at the definition of `StatusOr<T>::GetValue`:
   
   ```c++
     value_type& GetValue() & {
       CHECK(*this);
       return getValue();
     }
   
     value_type&& GetValue() && {
       CHECK(*this);
       return std::move(getValue());
     }
   
     const value_type& GetValue() const& {
       CHECK(*this);
       return getValue();
     }
   ```
   
   You will find there is three different overload of this method via changing the qualifier of `*this`.
   
   Hence, in `f(x).GetValue()`, obviously `f(x)` is a [prvalue](https://en.cppreference.com/w/cpp/language/value_category#prvalue) since a temporary object is returned, and then the signature `value_type&& GetValue() &&` is decided to be called after the [overload resolution](https://en.cppreference.com/w/cpp/language/overload_resolution), and thus you get an [xvalue](https://en.cppreference.com/w/cpp/language/value_category#xvalue) (which is a special kind of rvalue) referenced to the underlying value of the returned `StatusOr`.
   
   So you should take care of the lifetime in this case, the rvalue will become invalid after the computation of the current statement. You can extend it via a call to a move ctor (`T(T&&)`). This is not a defect of this type, but it is an idiom of C++, you should always care about lifetime of references to avoid dangling references in all standards of C++ (98/03/11/14/17/20 ...).



-- 
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] ShooterIT commented on pull request #768: Add `StatusOr` for error handling in modern C++ style

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

   Wait a minute, i still need some time to put my thoughts


-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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

   > This approach may be a more modern C++ approach, but I don't think it's very friendly for a rookie like me. It may take me some time to get familiar with how to use `StatusOr` and what the implementation means, which may not be very friendly for a beginner like me to contribute code.
   > 
   > RocksDB as the storage engine, also uses the `Status` without major performance issues, so I think `Status` is better.
   
   First of all, I do not think contributors need to understand the implementation of `StatusOr`, just as they do not need to understand the implementation of `std::optional` (c++17), `std::varaint` (c++17), `std:;expected` (c++20), `std::any` or any other containers in STL (or called standard library). The interface of `StatusOr` is straightforward enough for all usage without learning it's implementation. 


-- 
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] mapleFU commented on pull request #768: Add `StatusOr` for error handling in modern C++ style

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

   I wonder why kvrocks' `Status` uses `std::string`, it would be large, likely 24B on stack ( for example: https://github.com/llvm-mirror/libcxx/blob/master/include/string#L765-L784 )
   
   When I gothrough the code of project, I found that most cases would be like:
   
   ```
   Status s = Func(...);
   if (!s.ok()) {
      // print log for `s`
     return Status::Err(...);  // return a "bad" status with a new status
   }
   ...
   ```
   
   or:
   
   ```
   Status s = Func(...);
   if (!s.ok()) {
      return s;
   }
   ...
   ```
   
   Firstly, on bad path, `Status` would be like: `Status(Status::DBGetWALErr, "iterator not valid");`, sometimes it would be so large that it can't be optimized by sso, causing some temp-on-heap allocating.
   
   Secondly, sometimes when we want to find the root cause of the problem, status make it hard to debug. apache doris thinks using exception is a better method: https://github.com/apache/doris/discussions/8406
   
   


-- 
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] ShooterIT commented on a diff in pull request #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   i want to describe `11` is greater than `10`, `GetValue()` will return error?



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   If you look at the definition of `StatusOr<T>::GetValue`:
   
   ```c++
     value_type& GetValue() & {
       CHECK(*this);
       return getValue();
     }
   
     value_type&& GetValue() && {
       CHECK(*this);
       return std::move(getValue());
     }
   
     const value_type& GetValue() const& {
       CHECK(*this);
       return getValue();
     }
   ```
   
   You will find there is three different overload of this method via changing the qualifier of `*this`.
   
   Hence, in `f(x).GetValue()`, obviously `f(x)` is a [prvalue](https://en.cppreference.com/w/cpp/language/value_category#prvalue) since a temporary object is returned, and then the signature `value_type&& GetValue() &&` is decided to be called after the [overload resolution](https://en.cppreference.com/w/cpp/language/overload_resolution), and thus you get an [xvalue](https://en.cppreference.com/w/cpp/language/value_category#xvalue) (which is a special kind of rvalue) referenced to the underlying value of the returned `StatusOr`.
   
   So you should take care of the lifetime in this case, the rvalue will become invalid after the computation of the current expression. You can extend it via a call to a move ctor.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   BTW, The classic usage of `StatusOr<T>` we expected is:
   ```c++
   // 1
   StatusOr<T> someProcess(...); // as a return type
   
   // 2
   T val;
   ...
   return val; // return val typed T to StatusOr<T>
   
   // 3
   return {Status::someErrCode, "msg..."}; // return an error status
   
   // 4
   auto res = someProcess(...); // get result typed StatusOr<T>
   if(!res) return res; // forward it if it is an error status
   
   doSomething(*res); // process the value inside the StatusOr
   ```
   Use `StatusOr` as an rvalue is totally (well) supported, but everyone should be aware of what they are doing, otherwise just follow the classic usage is fine.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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

   Hi everyone, feel free to review and ask questions. 
   
   If there is no further discussion, I will merge it. :rocket:


-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
src/status.h:
##########
@@ -61,20 +66,204 @@ class Status {
   };
 
   Status() : Status(cOK) {}
-  explicit Status(Code code, std::string msg = {}) : code_(code), msg_(std::move(msg)) {}
-  bool IsOK() { return code_ == cOK; }
-  bool IsNotFound() { return code_ == NotFound; }
-  bool IsImorting() { return code_ == SlotImport; }
-  bool IsBlockingCommand() { return code_ == BlockingCmd; }
-  std::string Msg() {
-    if (IsOK()) {
-      return "ok";
-    }
+  explicit Status(Code code, std::string msg = {})
+    : code_(code), msg_(std::move(msg)) {}
+
+  template <Code code>
+  bool Is() const { return code_ == code; }
+
+  bool IsOK() const { return Is<cOK>(); }
+  operator bool() const { return IsOK(); }
+
+  Code GetCode() const {
+    return code_;
+  }
+
+  std::string Msg() const& {
+    if (*this) return ok_msg();
     return msg_;
   }
+
+  std::string Msg() && {
+    if (*this) return ok_msg();
+    return std::move(msg_);
+  }
+
   static Status OK() { return {}; }
 
  private:
   Code code_;
   std::string msg_;
+
+  static constexpr const char* ok_msg() {
+    return "ok";
+  }
+
+  template <typename T>
+  friend struct StatusOr;
+};
+
+template <typename ...Ts>
+using first_element = typename std::tuple_element<0, std::tuple<Ts...>>::type;
+
+template <typename T>
+using remove_cvref_t = typename std::remove_cv<typename std::remove_reference<T>::type>::type;
+
+template <typename T>
+struct StatusOr;
+
+template <typename T>
+struct IsStatusOr : std::integral_constant<bool, false> {};
+
+template <typename T>
+struct IsStatusOr<StatusOr<T>> : std::integral_constant<bool, true> {};
+
+template <typename T>
+struct StatusOr {
+  static_assert(!std::is_same<T, Status>::value, "value_type cannot be Status");
+  static_assert(!std::is_same<T, Status::Code>::value, "value_type cannot be Status::Code");
+  static_assert(!IsStatusOr<T>::value, "value_type cannot be StatusOr");
+  static_assert(!std::is_reference<T>::value, "value_type cannot be reference");
+
+  using value_type = T;
+
+  // we use std::unique_ptr to make the error part as small as enough
+  using error_type = std::unique_ptr<std::string>;
+
+  using Code = Status::Code;
+
+  explicit StatusOr(Status s) : code_(s.code_) {
+    CHECK(!s);
+    new(storage_) error_type(new std::string(std::move(s.msg_)));
+  }
+
+  StatusOr(Code code, std::string msg = {}) : code_(code) { // NOLINT
+    CHECK(code != Code::cOK);
+    new(storage_) error_type(new std::string(std::move(msg)));
+  }
+
+  template <typename ...Ts,
+    typename std::enable_if<
+      (sizeof...(Ts) > 0 &&
+        !std::is_same<Status, remove_cvref_t<first_element<Ts...>>>::value &&
+        !std::is_same<Code, remove_cvref_t<first_element<Ts...>>>::value &&
+        !std::is_same<value_type, remove_cvref_t<first_element<Ts...>>>::value &&
+        !std::is_same<StatusOr, remove_cvref_t<first_element<Ts...>>>::value
+      ), int>::type = 0> // NOLINT
+  explicit StatusOr(Ts && ... args) : code_(Code::cOK) {
+    new(storage_) value_type(std::forward<Ts>(args)...);
+  }
+
+  StatusOr(T&& value) : code_(Code::cOK) { // NOLINT
+    new(storage_) value_type(std::move(value));
+  }
+
+  StatusOr(const T& value) : code_(Code::cOK) { // NOLINT
+    new(storage_) value_type(value);
+  }
+
+  StatusOr(const StatusOr&) = delete;
+  StatusOr(StatusOr&& other) : code_(other.code_) {
+    if (code_ == Code::cOK) {
+      new(storage_) value_type(std::move(other.getValue()));
+    } else {
+      new(storage_) error_type(std::move(other.getError()));
+    }
+  }
+
+  Status& operator=(const Status&) = delete;
+
+  template <Code code>
+  bool Is() const { return code_ == code; }
+
+  bool IsOK() const { return Is<Code::cOK>(); }
+  operator bool() const { return IsOK(); }
+
+  Status ToStatus() const& {
+    if (*this) return Status::OK();
+    return Status(code_, *getError());
+  }
+
+  Status ToStatus() && {
+    if (*this) return Status::OK();
+    return Status(code_, std::move(*getError()));
+  }
+
+  Code GetCode() const {
+    return code_;
+  }
+
+  value_type& GetValue() & {
+    CHECK(*this);
+    return getValue();
+  }
+
+  value_type&& GetValue() && {
+    CHECK(*this);
+    return std::move(getValue());
+  }
+
+  const value_type& GetValue() const& {
+    CHECK(*this);
+    return getValue();
+  }
+
+  value_type& operator*() & {
+    return GetValue();
+  }
+
+  value_type&& operator*() && {
+    return std::move(GetValue());
+  }
+
+  const value_type& operator*() const& {
+    return GetValue();
+  }
+
+  value_type* operator->() {
+    return &GetValue();
+  }
+
+  const value_type* operator->() const {
+    return &GetValue();
+  }
+
+  std::string Msg() const& {
+    if (*this) return Status::ok_msg();
+    return *getError();
+  }
+
+  std::string Msg() && {
+    if (*this) return Status::ok_msg();
+    return std::move(*getError());
+  }
+
+  ~StatusOr() {
+    if (*this) {
+      getValue().~value_type();
+    } else {
+      getError().~error_type();
+    }
+  }
+
+ private:
+  Status::Code code_;
+  alignas(value_type) alignas(error_type) unsigned char storage_

Review Comment:
   Thanks. Fixed.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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

   > C++23 std::expected [2] is used to contains expected value or an error, does this a more generic way, Can we implement same thing like this. We can combine Status and the expected class do the things StatusOr does.
   
   Hi @wy-ei , `StatusOr<T>` has same purpose and similar implementation with `std::expected<T, E>`. Actually I has implemented something equal to `std::expected` for [oneflow](https://github.com/Oneflow-Inc/oneflow), and you can find its source code [here](https://github.com/Oneflow-Inc/oneflow/tree/master/oneflow/maybe), `oneflow::maybe::Maybe<T, E>`, `oneflow::maybe::Variant<T...>` and `oneflow::maybe::Optional<T>` is the basic error handling method in oneflow.
   
   But obviously, the implementation of `std::expected` is only more complicated and difficult to understand, while the implementation of `StatusOr` is relatively simple.
   
   > For StatusOr I have one question. StatusOr use a char[] to store T or Status, I don't known why, why don't use union?
   
   The same problem is faced with the union implementation, you also need to evaluate some conditions before calling the ctor dtor, so I don't see a difference.
   
   >  The Status class implemented in leveldb only take 8 bytes (only one member which is const char *state_). If status is ok, state_ is nullptr, if status is not ok, the state_ store the error code and error message. If we change the Status implementation in kvrocks to the leveldb way, the size of Status is not a problem.
   
   I think it's only a little different from `std::unique_ptr<Status>` and performs worse than `StatusOr`, since error code is on the stack in `StatusOr`.
   
   > Another problem of Status for me is it occupy the place of the return value, we must pass a pointer to take the return value. Golang solve this problem by returning multi value. A sample and easily understand way is return a tuple or another Class will contain return value T and status.
   
   This is actually a huge flaw in golang: golang has no [sum type](https://en.wikipedia.org/wiki/Tagged_union). So in golang, only two values can be returned instead of expressing an "either .. or .." relationship. Of course, interface can achieve this, but it is obviously far more troublesome than sum 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] tisonkun commented on pull request #768: Add `StatusOr` for error handling in modern C++ style

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

   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] wy-ei commented on pull request #768: Add `StatusOr` for error handling in modern C++ style

Posted by GitBox <gi...@apache.org>.
wy-ei commented on PR #768:
URL: https://github.com/apache/incubator-kvrocks/pull/768#issuecomment-1211606587

   @PragmaTwice 
   
   Throwing exception is more than 100 times slower than return error code, If the probability of throwing exception is less than 1/100, the cost of throwing exception can be ignored.
   
   I run the benchmark in  [1] on my machine, here is the result:
   
   ```
   Run on (8 X 24.1247 MHz CPU s)
   CPU Caches:
     L1 Data 64 KiB
     L1 Instruction 128 KiB
     L2 Unified 4096 KiB (x8)
   Load Average: 4.60, 3.61, 3.14
   ----------------------------------------------------------------------
   Benchmark                            Time             CPU   Iterations
   ----------------------------------------------------------------------
   BM_exitWithBasicException          356 ns          356 ns      1949354
   BM_exitWithMessageException        423 ns          412 ns      1812467
   BM_exitWithReturn                 3.09 ns         3.09 ns    218084841
   BM_exitWithErrorCode              3.04 ns         3.04 ns    233471081
   ```
   
   In this benchmark, the exception was thrown  every two function calls, but in real situation, exception unlikely be thrown this often.
   
   I change the benchmark code to make the exception be thrown every 200 function calls. (change the variable `randomRange` to 200),and run benchmark again, here is the result:
   
   ```
   Run on (8 X 24.1216 MHz CPU s)
   CPU Caches:
     L1 Data 64 KiB
     L1 Instruction 128 KiB
     L2 Unified 4096 KiB (x8)
   Load Average: 3.45, 2.54, 3.10
   ----------------------------------------------------------------------
   Benchmark                            Time             CPU   Iterations
   ----------------------------------------------------------------------
   BM_exitWithBasicException         6.78 ns         6.78 ns     75871712
   BM_exitWithMessageException       7.03 ns         7.03 ns     99061744
   BM_exitWithReturn                 2.98 ns         2.98 ns    235143942
   BM_exitWithErrorCode              2.98 ns         2.98 ns    234956097
   ```
   
   Since throwing exception is a small part of the code, in real situation, I think the cost of exception is not a big problem.
   
   > So I don't think it's a good idea to use exceptions on some execution path that is not a very low probability error or is user  controllable.
   
   If user input cause error,  then error may occur with high probability, use exception would be inefficient. In this situation, some pre check code can be added to return error fast,  and comparing with disk and network read and write, the cost of exception is not the main part. 
   
   > Another defect is that C++ does not have checked exception, this makes exception flows not easy to maintain. In addition, it is not easy to ensure the exception safety of the program, which requires the programmer to have a deep background knowledge of C++.
   
   C++ do not force user catch the exception, but if you miss an exception the process will crash. We can ignore an error status, and something weird happened, but we didn't known the reason. If we keep using RAII, the exception safety can almost be ensure. 
   
   I mentioned exception here because I was think is there another way to deal with error instead Status. I think both exception and status are ok. But using exception in kvrocks will break the consistency of the function design (return Status and pass result as a pointer in last argument), I think this will be a big change, and I except this won't happed in the near future. 
   
   I was off topic for a long time,  Back to Status and StatusOr.
   
   In kvrocks, Status class is a large object which include a std::string in it.  The Status class implemented in leveldb only take 8 bytes (only one member which is  const char *state_).  If status is ok, state_ is nullptr, if status is not ok, the state_ store the error code and error message. If we change the Status implementation in kvrocks to the leveldb way, the size of Status is not a problem. 
   
   Another problem of Status for me is it occupy the place of the return value, we must pass a pointer to take the return value. Golang solve this problem by returning multi value.  A sample and easily understand way is return a tuple  or another Class will contain return value T and status.
   
   For StatusOr I have one question. StatusOr use a char[] to store T or Status, I don't known why, why don't use union?
   
   C++23 std::expected [2] is used to contains expected value or an error, does this a more generic way, Can we implement same thing like this. We can combine Status and the expected class do the things StatusOr does.
   
   
   [1] https://pspdfkit.com/blog/2020/performance-overhead-of-exceptions-in-cpp
   [2] https://en.cppreference.com/w/cpp/header/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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   BTW, the classic usage of `StatusOr<T>` we expected is:
   ```c++
   // 1
   StatusOr<T> someProcess(...); // as a return type
   
   // 2
   T val;
   ...
   return val; // return val typed T to StatusOr<T>
   
   // 3
   return {Status::someErrCode, "msg..."}; // return an error status
   
   // 4
   auto res = someProcess(...); // get result typed StatusOr<T>
   if(!res) return res; // forward it if it is an error status
   
   doSomething(*res); // process the value inside the StatusOr
   ```
   Use `StatusOr` as an rvalue is totally (well) supported, but everyone should be aware of what they are doing, otherwise just follow the classic usage is fine.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   BTW, The classic usage of `StatusOr<T>` we expected is:
   ```c++
   // 1
   StatusOr<T> someProcess(...); // as a return type
   
   // 2
   T val;
   ...
   return val; // return val typed T to StatusOr<T>
   
   // 3
   return {Status::someErrCode, "msg..."} // return an error status
   
   // 4
   auto res = someProcess(...); // get result typed StatusOr<T>
   if(!res) return res; // forward it if it is an error status
   
   doSomething(*res); // process the value inside the StatusOr
   ```



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   Confirm its failure as:
   
   ```
   WARNING: Logging before InitGoogleLogging() is written to STDERR
   F20220812 15:30:19.518088 1124890 status.h:202] Check failed: *this 
   *** Check failure stack trace: ***
   ```



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   If you look at the definition of `StatusOr<T>::GetValue`:
   
   ```c++
     value_type& GetValue() & {
       CHECK(*this);
       return getValue();
     }
   
     value_type&& GetValue() && {
       CHECK(*this);
       return std::move(getValue());
     }
   
     const value_type& GetValue() const& {
       CHECK(*this);
       return getValue();
     }
   ```
   
   You will find there is three different overload of this method via changing the qualifier of `*this`.
   
   Hence, in `f(x).GetValue()`, obviously `f(x)` is a [prvalue](https://en.cppreference.com/w/cpp/language/value_category#prvalue) since a temporary object is returned, and then the signature `value_type&& GetValue() &&` is decided to be called after the [overload resolution](https://en.cppreference.com/w/cpp/language/overload_resolution), and thus you get an [xvalue](https://en.cppreference.com/w/cpp/language/value_category#xvalue) (which is a special kind of rvalue) referenced to the underlying value of the returned `StatusOr`.
   
   So you should take care of the lifetime in this case, the rvalue will become invalid after the computation of the current expression. You can extend it via a call to a move ctor. This is not a defect of this type, but it is an idiom of C++, you should always care about lifetime of references to avoid dangling references in all standards of C++ (98/03/11/14/17/20 ...).



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   If you look at the definition of `StatusOr<T>::GetValue`:
   
   ```c++
     value_type& GetValue() & {
       CHECK(*this);
       return getValue();
     }
   
     value_type&& GetValue() && {
       CHECK(*this);
       return std::move(getValue());
     }
   
     const value_type& GetValue() const& {
       CHECK(*this);
       return getValue();
     }
   ```
   
   You will find there is three different overload of this method via changing the qualifier of `*this`.
   
   Hence, in `f(x).GetValue()`, obviously `f(x)` is a [prvalue](https://en.cppreference.com/w/cpp/language/value_category#prvalue) since a temporary object is returned, and then the signature `value_type&& GetValue() &&` is decided to be called after the [overload resolution](https://en.cppreference.com/w/cpp/language/overload_resolution), and thus you get an [xvalue](https://en.cppreference.com/w/cpp/language/value_category#xvalue) (which is a special kind of rvalue) referenced to the underlying value of the returned `StatusOr`.
   
   So you should take care of the lifetime in this case, the rvalue will become invalid after the computation of the current expression. You can extend it via a call to a move ctor (`T(T&&)`). This is not a defect of this type, but it is an idiom of C++, you should always care about lifetime of references to avoid dangling references in all standards of C++ (98/03/11/14/17/20 ...).



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   If you look at the definition of `StatusOr<T>::GetValue`:
   
   ```c++
     value_type& GetValue() & {
       CHECK(*this);
       return getValue();
     }
   
     value_type&& GetValue() && {
       CHECK(*this);
       return std::move(getValue());
     }
   
     const value_type& GetValue() const& {
       CHECK(*this);
       return getValue();
     }
   ```
   
   You will find there is three different overload of this method via changing the qualifier of `*this`.
   
   Hence, in `f(x).GetValue()`, obviously `f(x)` is a [prvalue](https://en.cppreference.com/w/cpp/language/value_category#prvalue) since a temporary object is returned, and then the signature `value_type&& GetValue() &&` is decided to be called after the [overload resolution](https://en.cppreference.com/w/cpp/language/overload_resolution), and thus you get an [xvalue](https://en.cppreference.com/w/cpp/language/value_category#xvalue) (which is a special kind of rvalue) referenced to the underlying value of the returned `StatusOr`.
   
   



-- 
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] ShooterIT commented on pull request #768: Add `StatusOr` for error handling in modern C++ style

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

   To be honest, this is the first time to see this usage. I study your PR and https://abseil.io/docs/cpp/guides/status, i find it truly satisfies my requirement that i want to return status or value if ok when calling function, a bit like multi return value, such as
   ```
   StatusOr<Foo> result = Calculation();
   if (result.ok()) {
     result->DoSomethingCool();
   } else {
     LOG(ERROR) << result.status();
   }
   ```
   
   before you PR, our usage is like the following:
   ```
   MyType myvalue;
   Status s = Func(&myvalue);
   if (s.ok()) {
     DoThings(myvalue);
   } else {
     ErrorReport();
   }
   ```
   
   I think you may want to refactor these usages above, right?
   
   i am not a c++ language expert, i don't see this usage frequently, i think there are a few people accept quickly, so i am not sure we should merge this. Maybe i am lazy, and not want to learn new knowledge.


-- 
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] wy-ei commented on pull request #768: Add `StatusOr` for error handling in modern C++ style

Posted by GitBox <gi...@apache.org>.
wy-ei commented on PR #768:
URL: https://github.com/apache/incubator-kvrocks/pull/768#issuecomment-1210570826

   @PragmaTwice 
   
   If using exception to report error instead of status, we don't need Status and this StatusOr at all.
   
   What is the better way to report error in C++? error code or exception? There are many posts discuss this question, here are some of them:
   
   - [https://isocpp.org/wiki/faq/exceptions](https://isocpp.org/wiki/faq/exceptions)
   - [https://stackoverflow.com/questions/253314/conventions-for-exceptions-or-error-codes](https://stackoverflow.com/questions/253314/conventions-for-exceptions-or-error-codes)


-- 
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] mapleFU commented on pull request #768: Add `StatusOr` for error handling in modern C++ style

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

   I wonder why kvrocks' `Status` uses `std::string`, it would be large, likely 24B on stack ( for example: https://github.com/llvm-mirror/libcxx/blob/master/include/string#L765-L784 )
   
   When I gothrough the code of project, I found that status code is widely used, introducing exception might be ok or not (some says it can help trace the error stack: https://github.com/apache/doris/discussions/8406 , and some doesn't like it because of performance lost, like https://github.com/scylladb/seastar/issues/73 . But introducing exception will change the code style greatly, so I don't thinks it's a good idea to change to exception.
   
   Using `StatusOr` can change the error handling style without changing lots of codes, and it doesn't need changing other code using `Status`, so I think is ok to introducing 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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   BTW, The classic usage of `StatusOr<T>` we expected is:
   ```c++
   // 1
   StatusOr<T> someProcess(...); // as a return type
   
   // 2
   T val;
   ...
   return val; // return val typed T to StatusOr<T>
   
   // 3
   return {Status::someErrCode, "msg..."}; // return an error status
   
   // 4
   auto res = someProcess(...); // get result typed StatusOr<T>
   if(!res) return res; // forward it if it is an error status
   
   doSomething(*res); // process the value inside the StatusOr
   ```



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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

   In fact, I think `StatusOr` is not only more expressive, but also performs better than `Status` in some cases, especially when template parameters are primitive types. 
   
   This is because `Status` is a large structure (`std::string` is usually larger than two words because SSO is generally used in the implementation of `std::string`, for example, the implementation of `std::string` in libstdc++ is 32 bytes under 64 bits), so just `Status` is 40 bytes long, not including pointers for output. And Status inevitably constructs a `std::string` for any execution path.
   
   But `StatusOr` instance like `StatusOr<int>` is only 16 bytes long and does not initialize `std::string` when no error occurs, and it initializes the resulting integer directly in-place inside these 16 bytes.


-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   If you look at the definition of `StatusOr<T>::GetValue`:
   
   ```c++
     value_type& GetValue() & {
       CHECK(*this);
       return getValue();
     }
   
     value_type&& GetValue() && {
       CHECK(*this);
       return std::move(getValue());
     }
   
     const value_type& GetValue() const& {
       CHECK(*this);
       return getValue();
     }
   ```
   
   You will find there is three different overload of this method via changing the qualifier of `*this`.
   
   Hence, in `f(x).GetValue()`, obviously `f(x)` is a [prvalue](https://en.cppreference.com/w/cpp/language/value_category#prvalue) since a temporary object is returned, and then the signature `value_type&& GetValue() &&` is decided to be called after the [overload resolution](https://en.cppreference.com/w/cpp/language/overload_resolution), and thus you get an [xvalue](https://en.cppreference.com/w/cpp/language/value_category#xvalue) (which is a special kind of rvalue) referenced to the underlying value of the returned `StatusOr`.
   
   So you should take care of the lifetime in this case, the rvalue will become invalid after the computation of the current expression. You can extend it via a call to a move ctor. This is not a defect of this type, but it is an idiom of C++, you should always care about lifetime of references to avoid dangling references.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   I think that guarded by `CHECK(*this)`, it can cause a fatal error.
   
   I wonder if we can generate a compile time error instead of runtime fatal in this case.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   Hi @tisonkun, thanks for your review.
   
   > I wonder if we can generate a compile time error instead of runtime fatal in this case.
   
   Good idea. We consider an example for this:
   ```c++
   auto res = someProcess(...); // get result typed StatusOr<T>
   // we cannot deref `res` here
   
   if(!res) { 
     processError(res.GetCode(), res.Msg()); /* or */ return res; // we cannot deref `res` here
   }
   
   doSomething(*res); // we can deref it
   ```
   
   In this example, only the `doSomething` part can actually dereference the `res`. 
   We cannot dereference it both before `if(!res)` and inside `if(!res)`.
   So only in the execution path with the condition `res.IsOk() == true` being satisfied, we can dereference `res`,
   
   It is hardly to express in compile time, since it is related to the control flow.
   Like a raw/smart pointer, we should check it does point to a value (not null) before dereference it.
   
   I have implemented a checker in Clang Static Analyzer (included in clang-tidy) for OneFlow that checks whether a type like `StatusOr` is dereferenced after checking `IsOk()` using [symbolic execution](https://en.wikipedia.org/wiki/Symbolic_execution), and I think it can be done in this way in the future.



-- 
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 #768: Add `StatusOr` for error handling in modern C++ style

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


##########
tests/cppunit/status_test.cc:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include <gtest/gtest.h>
+
+#include <memory>
+#include <status.h>
+
+TEST(StatusOr, Scalar) {
+  auto f = [](int x) -> StatusOr<int> {
+    if (x > 10) {
+        return {Status::NotOK, "x large than 10"};
+    }
+
+    return 2 * x + 5;
+  };
+
+  ASSERT_EQ(*f(1), 7);
+  ASSERT_EQ(*f(5), 15);
+  ASSERT_EQ(f(7).GetValue(), 19);
+  ASSERT_EQ(f(7).GetCode(), Status::cOK);
+  ASSERT_EQ(f(7).Msg(), "ok");
+  ASSERT_TRUE(f(6));
+  ASSERT_EQ(f(11).GetCode(), Status::NotOK);
+  ASSERT_EQ(f(11).Msg(), "x large than 10");

Review Comment:
   Hi @tisonkun, thanks for your review.
   
   > I wonder if we can generate a compile time error instead of runtime fatal in this case.
   
   Good idea. We consider an example for this:
   ```c++
   auto res = someProcess(...); // get result typed StatusOr<T>
   // we cannot deref `res` here
   
   if(!res) { 
     processError(res.GetCode(), res.Msg()); /* or */ return res; // we cannot deref `res` here
   }
   
   doSomething(*res); // we can deref it
   ```
   
   In this example, only the `doSomething` part can actually dereference the `res`. 
   We cannot dereference it both before `if(!res)` and inside `if(!res)`.
   So only in the execution path with the condition `res.IsOk() == true` being satisfied, we can dereference `res`,
   
   It is hardly to express in compile time, since it is related to the control flow.
   Like a raw/smart pointer, we should check it does point to a value (not null) before dereference it.
   
   I have implemented a checker in Clang Static Analyzer that checks whether a type like `StatusOr` is dereferenced after checking `IsOk()`, and I think it can be done in this way in the future.



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