You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pegasus.apache.org by GitBox <gi...@apache.org> on 2021/04/20 08:20:10 UTC

[GitHub] [incubator-pegasus] levy5307 opened a new pull request #722: refactor: implement value_shcema_v0

levy5307 opened a new pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722


   ### What problem does this PR solve? <!--add issue link with summary if exists-->
   implement value_schema_v0, which represents data version 0.
   
   ### Checklist <!--REMOVE the items that are not applicable-->
   
   ##### Tests <!-- At least one of them must be included. -->
   
   - Unit test
   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] levy5307 commented on pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
levy5307 commented on pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#issuecomment-827252922


   > ```yaml
   > uint64_t
   > ```
   
   I don't think it's a good idea. Since we should add a different interface for each filed, for example: `extract_version_from_value`. This means you should add a interface if you add a filed. And In this interface, you should deal with all of these version even if this version does't support 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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] levy5307 commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
levy5307 commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r635905006



##########
File path: src/base/value_schema_v0.cpp
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "value_schema_v0.h"
+
+#include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/smart_pointers.h>
+
+namespace pegasus {
+std::unique_ptr<value_field> value_schema_v0::extract_field(dsn::string_view value,
+                                                            value_field_type type)
+{
+    std::unique_ptr<value_field> field = nullptr;
+    switch (type) {
+    case value_field_type::EXPIRE_TIMESTAMP:
+        field = extract_timestamp(value);
+        break;
+    default:
+        dassert_f(false, "Unsupported field type: {}", type);
+    }
+    return field;
+}
+
+dsn::blob value_schema_v0::extract_user_data(std::string &&value)
+{
+    auto ret = dsn::blob::create_from_bytes(std::move(value));
+    ret.range(sizeof(uint32_t));

Review comment:
       ignore the first 32 bits, which represents expire_ts in version 0, and other bits represents the user data.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] levy5307 commented on pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
levy5307 commented on pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#issuecomment-827254982


   > If your intention of this PR is to add new value schema more conveniently, I can give a simple solution:
   > 
   > ```c++
   > struct value_schema_v2
   > {
   >     static constexpr size_t version_field_offset = 0;
   >     typedef version_type = uint8_t;
   > 
   >     static constexpr size_t expire_ts_field_offset = 1;
   >     typedef expire_ts_type = uint32_t;
   > 
   >     static constexpr size_t timetag_field_offset = 5;
   >     typedef timetag_type = uint64_t;
   > 
   >     value_schema_v2() {
   >       _write_buf.resize(sizeof(uint8_t) + sizeof(uint32_t) + sizeof(uint64_t));
   >       _write_slices.resize(2);
   >     }
   > 
   >     rocksdb::SliceParts generate_value(uint8_t version, uint32_t expire_ts, uint64_t timetag, dsn::string_view user_data) {
   >       data_output out(_write_buf);
   >       out.write<uint8_t>(version);
   >       out.write<uint32_t>(expire_ts);
   >       out.write<uint64_t>(user_data);
   > 
   >       _write_buf[0] = rocksdb::Slice(_write_buf.data, )
   >       return {&_write_slices[0], static_cast<int>(_write_slices.size())};
   >     }
   > 
   >     static uint8_t extract_version(string_view val);
   >     static uint32_t extract_expire_ts(string_view val);
   >     static uint64_t extract_timetag(string_view val);
   >     static blob:: extract_user_data(std::string&& val);
   > 
   > private:
   >     std::string _write_buf;
   >     std::vector<rocksdb::Slice> _write_slices;
   > };
   > 
   > void extract_expire_ts_from_value(int version, string_view user_data);
   > void extract_timetag_from_value(int version, string_view user_data);
   > void extract_version_from_value(int version, string_view user_data);
   > ```
   > 
   > As you can see, this code is very similar to our thrift-generated code. It can be generated with a very simple template language, for example, yaml.
   > 
   > ```yaml
   > 0:
   >   - expire_ts : 
   >       type: uint32_t
   >       size: 4
   > 
   > 1:
   >   - expire_ts : 
   >       type: uint32_t
   >       size: 4
   >   - timetag : 
   >       type: uint64_t
   >       size: 8
   > 
   > 2:
   >   - version : 
   >       type: uint8_t
   >       size: 1
   >   - expire_ts : 
   >       type: uint32_t
   >       size: 4
   >   - timetag : 
   >       type: uint64_t
   >       size: 8
   > ```
   > 
   > I would suggest implementing a tool that can generate the above code according to the template, instead of using class inheritance.
   > The value encoding/decoding is extremely critical and requires efficiency. I think we should avoid runtime type deduction as much as we can, for example:
   > 
   > ```c++
   > void extract_expire_ts_from_value(value_schema *schema, string_view user_data) {
   >     auto field = schema->extract_field(raw_value, pegasus::value_field_type::EXPIRE_TIMESTAMP); // type deduction #1
   >     auto expire_ts_field = static_cast<expire_timestamp_field *>(segment.get()); // type deduction #2
   >     return expire_ts_field->expire_ts;
   > ```
   > 
   > There're 2 deductions here. I'm not indicating that this code will definitely perform poorly. But technically, I would prefer to write code without potential slow points.
   > 
   > Using a code generator can also avoid bugs so long as the tool itself is stable. So when we add another field in the future, we only need to modify the template file, rather than introduce more code.
   
   I don't think it's a good idea
   1.  we should add a different interface for each filed, for example: `extract_version_from_value`. This means you should add a interface if you add a filed. And In this interface, you should deal with all of these version even if this version does't support it. The code is not elegant enough if we implement it like this.
   2. `static_cast` doesn't cost too much time. And these is only one type deduction, not two.
   3. It's too complex to implement a tool to generate these code.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Smityz commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
Smityz commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r619768162



##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       ```
   std::map<value_field_type, std::unique_ptr<value_field>> fields;
   ```
   =>
   ```
   std::array<std::unique_ptr<value_field>> fields;
   fields[value_field_type::TIME_TAG] = dsn::make_unique<time_tag_field>(time_tag);
   ```
   ---
   ```
      ...
      auto expire_iter = params.fields.find(value_field_type::EXPIRE_TIMESTAMP);
       if (... || user_data_iter == params.fields.end())) {
           dassert_f(false, "USER_DATA or EXPIRE_TIMESTAMP is not provided");
           return {nullptr, 0};
       }
   ```
   =>
   ```
      ...
     // make sure std::array fields has enough spaces
      auto expire_iter = params.fields[value_field_type::EXPIRE_TIMESTAMP];
       if (... || user_data_iter == nullptr)) {
           dassert_f(false, "USER_DATA or EXPIRE_TIMESTAMP is not provided");
           return {nullptr, 0};
       }
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] neverchanje commented on pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
neverchanje commented on pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#issuecomment-826623454


   If your intention of this PR is to add new value schema more conveniently, I can give a simple solution:
   
   ```cpp
   struct value_schema_v2
   {
       static constexpr size_t version_field_offset = 0;
       typedef version_type = uint8_t;
   
       static constexpr size_t expire_ts_field_offset = 1;
       typedef expire_ts_type = uint32_t;
   
       static constexpr size_t timetag_field_offset = 5;
       typedef timetag_type = uint64_t;
   
       value_schema_v2() {
         _write_buf.resize(sizeof(uint8_t) + sizeof(uint32_t) + sizeof(uint64_t));
         _write_slices.resize(2);
       }
   
       rocksdb::SliceParts generate_value(uint8_t version, uint32_t expire_ts, uint64_t timetag, dsn::string_view user_data) {
         data_output out(_write_buf);
         out.write<uint8_t>(version);
         out.write<uint32_t>(expire_ts);
         out.write<uint64_t>(user_data);
   
         _write_buf[0] = rocksdb::Slice(_write_buf.data, )
         return {&_write_slices[0], static_cast<int>(_write_slices.size())};
       }
   
       static uint8_t extract_version(string_view val);
       static uint32_t extract_expire_ts(string_view val);
       static uint64_t extract_timetag(string_view val);
       static blob:: extract_user_data(std::string&& val);
   
   private:
       std::string _write_buf;
       std::vector<rocksdb::Slice> _write_slices;
   };
   
   void extract_expire_ts_from_value(int version, string_view user_data);
   void extract_timetag_from_value(int version, string_view user_data);
   void extract_version_from_value(int version, string_view user_data);
   ```
   
   As you can see, this code is very similar to our thrift-generated code. It can be generated with a very simple template language, for example, yaml.
   
   ```yaml
   0:
     - expire_ts : 
         type: uint32_t
         size: 4
   
   1:
     - expire_ts : 
         type: uint32_t
         size: 4
     - timetag : 
         type: uint64_t
         size: 8
   
   2:
     - version : 
         type: uint8_t
         size: 1
     - expire_ts : 
         type: uint32_t
         size: 4
     - timetag : 
         type: uint64_t
         size: 8
   ```
   
   I would suggest implementing a tool that can generate the above code according to the template, instead of using class inheritance.
   The value encoding/decoding is extremely critical and requires efficiency. I think we should avoid runtime type deduction as much as we can, for example:
   
   ```cpp
   void extract_expire_ts_from_value(value_schema *schema, string_view user_data) {
       auto field = schema->extract_field(raw_value, pegasus::value_field_type::EXPIRE_TIMESTAMP); // type deduction #1
       auto expire_ts_field = static_cast<expire_timestamp_field *>(segment.get()); // type deduction #2
       return expire_ts_field->expire_ts;
   ```
   
   There're 2 deductions here. I'm not indicating that this code will definitely perform poorly. But technically, I would prefer to write code without potential slow points.
   
   Using a code generator can also avoid bugs so long as the tool itself is stable. So when we add another field in the future, we only need to modify the template file, rather than introduce more code.
   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Smityz commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
Smityz commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r619768162



##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       ```
   std::map<value_field_type, std::unique_ptr<value_field>> fields;
   ```
   =>
   ```
   std::array<std::unique_ptr<value_field>> fields;
   fields[value_field_type::TIME_TAG] = dsn::make_unique<time_tag_field>(time_tag);
   ```
   
   ```
      ...
      auto expire_iter = params.fields.find(value_field_type::EXPIRE_TIMESTAMP);
       if (... || user_data_iter == params.fields.end())) {
           dassert_f(false, "USER_DATA or EXPIRE_TIMESTAMP is not provided");
           return {nullptr, 0};
       }
   ```
   =>
   ```
      ...
     // make sure std::array fields has enough spaces
      auto expire_iter = params.fields[value_field_type::EXPIRE_TIMESTAMP];
       if (... || user_data_iter == nullptr)) {
           dassert_f(false, "USER_DATA or EXPIRE_TIMESTAMP is not provided");
           return {nullptr, 0};
       }
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Smityz commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
Smityz commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r619768162



##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       ```
   std::map<value_field_type, std::unique_ptr<value_field>> fields;
   ```
   =>
   ```
   std::array<std::unique_ptr<value_field>> fields;
   fields[value_field_type::TIME_TAG] = dsn::make_unique<time_tag_field>(time_tag);
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Smityz commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
Smityz commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r619768162



##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       ```
   std::map<value_field_type, std::unique_ptr<value_field>> fields;
   ```
   =>
   ```
   std::array<std::unique_ptr<value_field>> fields;
   fields[value_field_type] = dsn::make_unique<time_tag_field>(time_tag);
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] neverchanje commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
neverchanje commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r617356875



##########
File path: src/base/value_schema_v0.h
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#pragma once
+
+#include "pegasus_value_schema.h"
+
+#include <dsn/utility/singleton.h>
+
+namespace pegasus {
+/**
+ *  rocksdb value: |- expire_ts(4bytes) -|- user value(bytes) -|
+ */
+class value_schema_v0 : public value_schema
+{
+public:
+    value_schema_v0() = default;
+
+    std::unique_ptr<value_field> extract_field(dsn::string_view value, value_field_type type);
+    dsn::blob extract_user_data(std::string &&value);
+    void update_field(std::string &value, std::unique_ptr<value_field> segment);
+    rocksdb::SliceParts generate_value(const value_params &params);
+    data_version version() const { return VERSION_0; }

Review comment:
       ```suggestion
       std::unique_ptr<value_field> extract_field(dsn::string_view value, value_field_type type) override;
       dsn::blob extract_user_data(std::string &&value) override;
       void update_field(std::string &value, std::unique_ptr<value_field> segment) override;
       rocksdb::SliceParts generate_value(const value_params &params) override;
       data_version version() override const { return VERSION_0; }
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] levy5307 removed a comment on pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
levy5307 removed a comment on pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#issuecomment-827252922


   > ```yaml
   > uint64_t
   > ```
   
   I don't think it's a good idea. Since we should add a different interface for each filed, for example: `extract_version_from_value`. This means you should add a interface if you add a filed. And In this interface, you should deal with all of these version even if this version does't support 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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] neverchanje commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
neverchanje commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r619946506



##########
File path: src/base/test/value_schema_test.cpp
##########
@@ -0,0 +1,113 @@
+/*
+ * 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 "base/pegasus_value_schema.h"
+#include "base/value_schema_manager.h"
+
+#include <gtest/gtest.h>
+
+using namespace pegasus;
+
+uint32_t extract_expire_ts(value_schema *schema, const std::string &raw_value)
+{
+    auto segment = schema->extract_field(raw_value, pegasus::value_field_type::EXPIRE_TIMESTAMP);
+    auto expire_ts_segment = static_cast<expire_timestamp_field *>(segment.get());
+    return expire_ts_segment->expire_ts;

Review comment:
       ```suggestion
       auto field = schema->extract_field(raw_value, pegasus::value_field_type::EXPIRE_TIMESTAMP);
       auto expire_ts_field = static_cast<expire_timestamp_field *>(segment.get());
       return expire_ts_field->expire_ts;
   ```

##########
File path: src/base/value_schema_manager.cpp
##########
@@ -51,6 +52,7 @@ value_schema *value_schema_manager::get_latest_value_schema() const
  */
 void register_value_schemas()
 {
+    value_schema_manager::instance().register_schema(dsn::make_unique<value_schema_v0>());

Review comment:
       You don't have to expose `register_value_schemas` as an API. It's totally unnecessary. You can simply put all registration of schemas in the constructor of `value_schema_manager`.
   
   ```
   class value_schema_manager {
     value_schema_manager() {
       register_schema(dsn::make_unique<value_schema_v0>());
     }
   
   private:
     void register_schema(...);
   }
   ```
   

##########
File path: src/base/value_schema_v0.cpp
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 "value_schema_v0.h"
+
+#include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/smart_pointers.h>
+
+namespace pegasus {
+std::unique_ptr<value_field> value_schema_v0::extract_field(dsn::string_view value,
+                                                            value_field_type type)
+{
+    std::unique_ptr<value_field> segment = nullptr;
+    switch (type) {
+    case value_field_type::EXPIRE_TIMESTAMP:
+        segment = extract_timestamp(value);
+        break;
+    default:
+        dassert_f(false, "Unsupported segment type: {}", type);
+    }
+    return segment;
+}
+
+dsn::blob value_schema_v0::extract_user_data(std::string &&value)
+{
+    auto *s = new std::string(std::move(value));
+    dsn::data_input input(*s);
+    input.skip(sizeof(uint32_t));
+    dsn::string_view view = input.read_str();
+
+    // tricky code to avoid memory copy
+    dsn::blob user_data;
+    std::shared_ptr<char> buf(const_cast<char *>(view.data()), [s](char *) { delete s; });
+    user_data.assign(std::move(buf), 0, static_cast<unsigned int>(view.length()));
+    return user_data;

Review comment:
       ```suggestion
       auto ret = dsn::blob::create_from_bytes(std::move(value));
       ret.range(sizeof(uint32_t));
       return ret;
   ```
   
   You should be aware of the string copy. This is costly.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong merged pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
hycdong merged pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722


   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Smityz commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
Smityz commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r619764949



##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       Maybe we can replace it with `std::array" to get more efficiency by both time complexity(O(logN) vs O(1)) and [cache-friendly ](https://stackoverflow.com/questions/16699247/what-is-a-cache-friendly-code).




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] levy5307 edited a comment on pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
levy5307 edited a comment on pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#issuecomment-827254982


   > If your intention of this PR is to add new value schema more conveniently, I can give a simple solution:
   > 
   > ```c++
   > struct value_schema_v2
   > {
   >     static constexpr size_t version_field_offset = 0;
   >     typedef version_type = uint8_t;
   > 
   >     static constexpr size_t expire_ts_field_offset = 1;
   >     typedef expire_ts_type = uint32_t;
   > 
   >     static constexpr size_t timetag_field_offset = 5;
   >     typedef timetag_type = uint64_t;
   > 
   >     value_schema_v2() {
   >       _write_buf.resize(sizeof(uint8_t) + sizeof(uint32_t) + sizeof(uint64_t));
   >       _write_slices.resize(2);
   >     }
   > 
   >     rocksdb::SliceParts generate_value(uint8_t version, uint32_t expire_ts, uint64_t timetag, dsn::string_view user_data) {
   >       data_output out(_write_buf);
   >       out.write<uint8_t>(version);
   >       out.write<uint32_t>(expire_ts);
   >       out.write<uint64_t>(user_data);
   > 
   >       _write_buf[0] = rocksdb::Slice(_write_buf.data, )
   >       return {&_write_slices[0], static_cast<int>(_write_slices.size())};
   >     }
   > 
   >     static uint8_t extract_version(string_view val);
   >     static uint32_t extract_expire_ts(string_view val);
   >     static uint64_t extract_timetag(string_view val);
   >     static blob:: extract_user_data(std::string&& val);
   > 
   > private:
   >     std::string _write_buf;
   >     std::vector<rocksdb::Slice> _write_slices;
   > };
   > 
   > void extract_expire_ts_from_value(int version, string_view user_data);
   > void extract_timetag_from_value(int version, string_view user_data);
   > void extract_version_from_value(int version, string_view user_data);
   > ```
   > 
   > As you can see, this code is very similar to our thrift-generated code. It can be generated with a very simple template language, for example, yaml.
   > 
   > ```yaml
   > 0:
   >   - expire_ts : 
   >       type: uint32_t
   >       size: 4
   > 
   > 1:
   >   - expire_ts : 
   >       type: uint32_t
   >       size: 4
   >   - timetag : 
   >       type: uint64_t
   >       size: 8
   > 
   > 2:
   >   - version : 
   >       type: uint8_t
   >       size: 1
   >   - expire_ts : 
   >       type: uint32_t
   >       size: 4
   >   - timetag : 
   >       type: uint64_t
   >       size: 8
   > ```
   > 
   > I would suggest implementing a tool that can generate the above code according to the template, instead of using class inheritance.
   > The value encoding/decoding is extremely critical and requires efficiency. I think we should avoid runtime type deduction as much as we can, for example:
   > 
   > ```c++
   > void extract_expire_ts_from_value(value_schema *schema, string_view user_data) {
   >     auto field = schema->extract_field(raw_value, pegasus::value_field_type::EXPIRE_TIMESTAMP); // type deduction #1
   >     auto expire_ts_field = static_cast<expire_timestamp_field *>(segment.get()); // type deduction #2
   >     return expire_ts_field->expire_ts;
   > ```
   > 
   > There're 2 deductions here. I'm not indicating that this code will definitely perform poorly. But technically, I would prefer to write code without potential slow points.
   > 
   > Using a code generator can also avoid bugs so long as the tool itself is stable. So when we add another field in the future, we only need to modify the template file, rather than introduce more code.
   
   I don't think it's a good idea
   1.  we should add a different interface for each filed, for example: `extract_version_from_value`. This means you should add a interface if you add a filed. And In this interface, you should deal with all of these version even if this version does't support it. The code is not elegant enough if we implement it like this. To be honest, that's what I did initially, but all of us thought it wasn't good enough.
   2. `static_cast` doesn't cost too much time. And there is only one type deduction, not two.
   3. It's too complex to implement a tool to generate these code.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] levy5307 edited a comment on pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
levy5307 edited a comment on pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#issuecomment-827254982


   > If your intention of this PR is to add new value schema more conveniently, I can give a simple solution:
   > 
   > ```c++
   > struct value_schema_v2
   > {
   >     static constexpr size_t version_field_offset = 0;
   >     typedef version_type = uint8_t;
   > 
   >     static constexpr size_t expire_ts_field_offset = 1;
   >     typedef expire_ts_type = uint32_t;
   > 
   >     static constexpr size_t timetag_field_offset = 5;
   >     typedef timetag_type = uint64_t;
   > 
   >     value_schema_v2() {
   >       _write_buf.resize(sizeof(uint8_t) + sizeof(uint32_t) + sizeof(uint64_t));
   >       _write_slices.resize(2);
   >     }
   > 
   >     rocksdb::SliceParts generate_value(uint8_t version, uint32_t expire_ts, uint64_t timetag, dsn::string_view user_data) {
   >       data_output out(_write_buf);
   >       out.write<uint8_t>(version);
   >       out.write<uint32_t>(expire_ts);
   >       out.write<uint64_t>(user_data);
   > 
   >       _write_buf[0] = rocksdb::Slice(_write_buf.data, )
   >       return {&_write_slices[0], static_cast<int>(_write_slices.size())};
   >     }
   > 
   >     static uint8_t extract_version(string_view val);
   >     static uint32_t extract_expire_ts(string_view val);
   >     static uint64_t extract_timetag(string_view val);
   >     static blob:: extract_user_data(std::string&& val);
   > 
   > private:
   >     std::string _write_buf;
   >     std::vector<rocksdb::Slice> _write_slices;
   > };
   > 
   > void extract_expire_ts_from_value(int version, string_view user_data);
   > void extract_timetag_from_value(int version, string_view user_data);
   > void extract_version_from_value(int version, string_view user_data);
   > ```
   > 
   > As you can see, this code is very similar to our thrift-generated code. It can be generated with a very simple template language, for example, yaml.
   > 
   > ```yaml
   > 0:
   >   - expire_ts : 
   >       type: uint32_t
   >       size: 4
   > 
   > 1:
   >   - expire_ts : 
   >       type: uint32_t
   >       size: 4
   >   - timetag : 
   >       type: uint64_t
   >       size: 8
   > 
   > 2:
   >   - version : 
   >       type: uint8_t
   >       size: 1
   >   - expire_ts : 
   >       type: uint32_t
   >       size: 4
   >   - timetag : 
   >       type: uint64_t
   >       size: 8
   > ```
   > 
   > I would suggest implementing a tool that can generate the above code according to the template, instead of using class inheritance.
   > The value encoding/decoding is extremely critical and requires efficiency. I think we should avoid runtime type deduction as much as we can, for example:
   > 
   > ```c++
   > void extract_expire_ts_from_value(value_schema *schema, string_view user_data) {
   >     auto field = schema->extract_field(raw_value, pegasus::value_field_type::EXPIRE_TIMESTAMP); // type deduction #1
   >     auto expire_ts_field = static_cast<expire_timestamp_field *>(segment.get()); // type deduction #2
   >     return expire_ts_field->expire_ts;
   > ```
   > 
   > There're 2 deductions here. I'm not indicating that this code will definitely perform poorly. But technically, I would prefer to write code without potential slow points.
   > 
   > Using a code generator can also avoid bugs so long as the tool itself is stable. So when we add another field in the future, we only need to modify the template file, rather than introduce more code.
   
   I don't think it's a good idea
   1.  we should add a different interface for each filed, for example: `extract_version_from_value`. This means you should add a interface if you add a filed. And In this interface, you should deal with all of these version even if this version does't support it. The code is not elegant enough if we implement it like this. To be honest, that's what I did initially, but all of us thought it wasn't good enough.
   2. `static_cast` doesn't cost too much time. And these is only one type deduction, not two.
   3. It's too complex to implement a tool to generate these code.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r633069870



##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }

Review comment:
       ```suggestion
       explicit expire_timestamp_field(uint32_t timestamp) : expire_ts(timestamp) {}
   ```

##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
-    std::map<value_field_type, std::unique_ptr<value_field>> fields;
-    /// TBD(zlw)
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
+    std::array<std::unique_ptr<value_field>, FIELD_COUNT> fields;
+    // write_buf and write_slices are transferred from `pegasus_value_generator`, Which are used to

Review comment:
       ```suggestion
       // write_buf and write_slices are transferred from `pegasus_value_generator`, which are used to
   ```

##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }
+    value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; };

Review comment:
       ```suggestion
       value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; }
   ```

##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }
+    value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; };
+
+    uint32_t expire_ts;
+};
+
+struct time_tag_field : public value_field
+{
+    explicit time_tag_field(uint64_t tag) { time_tag = tag; }
+    value_field_type type() { return value_field_type::TIME_TAG; };

Review comment:
       ```suggestion
       value_field_type type() { return value_field_type::TIME_TAG; }
   ```

##########
File path: src/base/value_schema_v0.cpp
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "value_schema_v0.h"
+
+#include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/smart_pointers.h>
+
+namespace pegasus {
+std::unique_ptr<value_field> value_schema_v0::extract_field(dsn::string_view value,
+                                                            value_field_type type)
+{
+    std::unique_ptr<value_field> field = nullptr;
+    switch (type) {
+    case value_field_type::EXPIRE_TIMESTAMP:
+        field = extract_timestamp(value);
+        break;
+    default:
+        dassert_f(false, "Unsupported field type: {}", type);
+    }
+    return field;
+}
+
+dsn::blob value_schema_v0::extract_user_data(std::string &&value)
+{
+    auto ret = dsn::blob::create_from_bytes(std::move(value));
+    ret.range(sizeof(uint32_t));

Review comment:
       What does this line mean?

##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }
+    value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; };
+
+    uint32_t expire_ts;
+};
+
+struct time_tag_field : public value_field
+{
+    explicit time_tag_field(uint64_t tag) { time_tag = tag; }

Review comment:
       ```suggestion
       explicit time_tag_field(uint64_t tag) : time_tag(tag) {}
   ```

##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }
+    value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; };
+
+    uint32_t expire_ts;
+};
+
+struct time_tag_field : public value_field
+{
+    explicit time_tag_field(uint64_t tag) { time_tag = tag; }
+    value_field_type type() { return value_field_type::TIME_TAG; };
+
+    uint64_t time_tag;
+};
+
+struct user_data_field : public value_field
+{
+    explicit user_data_field(dsn::string_view data) { user_data = data; }

Review comment:
       ```suggestion
       explicit user_data_field(dsn::string_view data) : user_data(data) {}
   ```

##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }
+    value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; };
+
+    uint32_t expire_ts;
+};
+
+struct time_tag_field : public value_field
+{
+    explicit time_tag_field(uint64_t tag) { time_tag = tag; }
+    value_field_type type() { return value_field_type::TIME_TAG; };
+
+    uint64_t time_tag;
+};
+
+struct user_data_field : public value_field
+{
+    explicit user_data_field(dsn::string_view data) { user_data = data; }
+    value_field_type type() { return value_field_type::USER_DATA; };

Review comment:
       ```suggestion
       value_field_type type() { return value_field_type::USER_DATA; }
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] levy5307 commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
levy5307 commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r619770152



##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       done




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Smityz commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
Smityz commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r619768162



##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       ```
   std::map<value_field_type, std::unique_ptr<value_field>> fields;
   ```
   =>
   ```
   std::array<std::unique_ptr<value_field>> fields;
   // fields[value_field_type] = dsn::make_unique<time_tag_field>(time_tag);
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] Smityz commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
Smityz commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r619764949



##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       Maybe we can replace it with `std::array" to get more efficiency by both time complexity(O(logN) vs O(1)) and [cache-friendly ](https://stackoverflow.com/questions/16699247/what-is-a-cache-friendly-code).

##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       ```
   std::map<value_field_type, std::unique_ptr<value_field>> fields;
   ```
   =>
   ```
   std::array<std::unique_ptr<value_field>> fields;
   // fields[value_field_type] = dsn::make_unique<time_tag_field>(time_tag);
   ```

##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       ```
   std::map<value_field_type, std::unique_ptr<value_field>> fields;
   ```
   =>
   ```
   std::array<std::unique_ptr<value_field>> fields;
   fields[value_field_type] = dsn::make_unique<time_tag_field>(time_tag);
   ```

##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       ```
   std::map<value_field_type, std::unique_ptr<value_field>> fields;
   ```
   =>
   ```
   std::array<std::unique_ptr<value_field>> fields;
   fields[value_field_type::TIME_TAG] = dsn::make_unique<time_tag_field>(time_tag);
   ```

##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       ```
   std::map<value_field_type, std::unique_ptr<value_field>> fields;
   ```
   =>
   ```
   std::array<std::unique_ptr<value_field>> fields;
   fields[value_field_type::TIME_TAG] = dsn::make_unique<time_tag_field>(time_tag);
   ```
   
   ```
      ...
      auto expire_iter = params.fields.find(value_field_type::EXPIRE_TIMESTAMP);
       if (... || user_data_iter == params.fields.end())) {
           dassert_f(false, "USER_DATA or EXPIRE_TIMESTAMP is not provided");
           return {nullptr, 0};
       }
   ```
   =>
   ```
      ...
     // make sure std::array fields has enough spaces
      auto expire_iter = params.fields[value_field_type::EXPIRE_TIMESTAMP];
       if (... || user_data_iter == nullptr)) {
           dassert_f(false, "USER_DATA or EXPIRE_TIMESTAMP is not provided");
           return {nullptr, 0};
       }
   ```

##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       ```
   std::map<value_field_type, std::unique_ptr<value_field>> fields;
   ```
   =>
   ```
   std::array<std::unique_ptr<value_field>> fields;
   fields[value_field_type::TIME_TAG] = dsn::make_unique<time_tag_field>(time_tag);
   ```
   ---
   ```
      ...
      auto expire_iter = params.fields.find(value_field_type::EXPIRE_TIMESTAMP);
       if (... || user_data_iter == params.fields.end())) {
           dassert_f(false, "USER_DATA or EXPIRE_TIMESTAMP is not provided");
           return {nullptr, 0};
       }
   ```
   =>
   ```
      ...
     // make sure std::array fields has enough spaces
      auto expire_iter = params.fields[value_field_type::EXPIRE_TIMESTAMP];
       if (... || user_data_iter == nullptr)) {
           dassert_f(false, "USER_DATA or EXPIRE_TIMESTAMP is not provided");
           return {nullptr, 0};
       }
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r633069870



##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }

Review comment:
       ```suggestion
       explicit expire_timestamp_field(uint32_t timestamp) : expire_ts(timestamp) {}
   ```

##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
-    std::map<value_field_type, std::unique_ptr<value_field>> fields;
-    /// TBD(zlw)
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
+    std::array<std::unique_ptr<value_field>, FIELD_COUNT> fields;
+    // write_buf and write_slices are transferred from `pegasus_value_generator`, Which are used to

Review comment:
       ```suggestion
       // write_buf and write_slices are transferred from `pegasus_value_generator`, which are used to
   ```

##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }
+    value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; };

Review comment:
       ```suggestion
       value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; }
   ```

##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }
+    value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; };
+
+    uint32_t expire_ts;
+};
+
+struct time_tag_field : public value_field
+{
+    explicit time_tag_field(uint64_t tag) { time_tag = tag; }
+    value_field_type type() { return value_field_type::TIME_TAG; };

Review comment:
       ```suggestion
       value_field_type type() { return value_field_type::TIME_TAG; }
   ```

##########
File path: src/base/value_schema_v0.cpp
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "value_schema_v0.h"
+
+#include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/smart_pointers.h>
+
+namespace pegasus {
+std::unique_ptr<value_field> value_schema_v0::extract_field(dsn::string_view value,
+                                                            value_field_type type)
+{
+    std::unique_ptr<value_field> field = nullptr;
+    switch (type) {
+    case value_field_type::EXPIRE_TIMESTAMP:
+        field = extract_timestamp(value);
+        break;
+    default:
+        dassert_f(false, "Unsupported field type: {}", type);
+    }
+    return field;
+}
+
+dsn::blob value_schema_v0::extract_user_data(std::string &&value)
+{
+    auto ret = dsn::blob::create_from_bytes(std::move(value));
+    ret.range(sizeof(uint32_t));

Review comment:
       What does this line mean?

##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }
+    value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; };
+
+    uint32_t expire_ts;
+};
+
+struct time_tag_field : public value_field
+{
+    explicit time_tag_field(uint64_t tag) { time_tag = tag; }

Review comment:
       ```suggestion
       explicit time_tag_field(uint64_t tag) : time_tag(tag) {}
   ```

##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }
+    value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; };
+
+    uint32_t expire_ts;
+};
+
+struct time_tag_field : public value_field
+{
+    explicit time_tag_field(uint64_t tag) { time_tag = tag; }
+    value_field_type type() { return value_field_type::TIME_TAG; };
+
+    uint64_t time_tag;
+};
+
+struct user_data_field : public value_field
+{
+    explicit user_data_field(dsn::string_view data) { user_data = data; }

Review comment:
       ```suggestion
       explicit user_data_field(dsn::string_view data) : user_data(data) {}
   ```

##########
File path: src/base/value_field.h
##########
@@ -23,12 +23,39 @@ namespace pegasus {
 
 enum value_field_type
 {
-    /// TBD(zlw)
+    EXPIRE_TIMESTAMP = 0,
+    TIME_TAG,
+    USER_DATA,
+    FIELD_COUNT,
 };
 
 struct value_field
 {
     virtual ~value_field() = default;
     virtual value_field_type type() = 0;
 };
+
+struct expire_timestamp_field : public value_field
+{
+    explicit expire_timestamp_field(uint32_t timestamp) { expire_ts = timestamp; }
+    value_field_type type() { return value_field_type::EXPIRE_TIMESTAMP; };
+
+    uint32_t expire_ts;
+};
+
+struct time_tag_field : public value_field
+{
+    explicit time_tag_field(uint64_t tag) { time_tag = tag; }
+    value_field_type type() { return value_field_type::TIME_TAG; };
+
+    uint64_t time_tag;
+};
+
+struct user_data_field : public value_field
+{
+    explicit user_data_field(dsn::string_view data) { user_data = data; }
+    value_field_type type() { return value_field_type::USER_DATA; };

Review comment:
       ```suggestion
       value_field_type type() { return value_field_type::USER_DATA; }
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] levy5307 commented on a change in pull request #722: refactor: implement value_shcema_v0

Posted by GitBox <gi...@apache.org>.
levy5307 commented on a change in pull request #722:
URL: https://github.com/apache/incubator-pegasus/pull/722#discussion_r619770152



##########
File path: src/base/pegasus_value_schema.h
##########
@@ -238,13 +238,24 @@ class pegasus_value_generator
 
 enum data_version
 {
+    VERSION_0 = 0,
+    VERSION_COUNT,
+    VERSION_MAX = VERSION_0,
     /// TBD(zlw)
 };
 
 struct value_params
 {
+    value_params(std::string &buf, std::vector<rocksdb::Slice> &slices)
+        : write_buf(buf), write_slices(slices)
+    {
+    }
+
     std::map<value_field_type, std::unique_ptr<value_field>> fields;

Review comment:
       done




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org