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 2022/08/09 13:36:07 UTC

[GitHub] [incubator-pegasus] WHBANG opened a new pull request, #1108: fix: rocksdb options not changed even if update in Pegasus config file

WHBANG opened a new pull request, #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108

   ### What problem does this PR solve? <!--add issue link with summary if exists-->
   https://github.com/apache/incubator-pegasus/issues/1025
   
   ### What is changed and how does it work?
       The specific reason is that if you open an existing rocksdb, the usage scenario will be set to normal first, and then set to the corresponding mode according to the actual usage scenario. This process will change the relevant options, causing rocksdb to execute flush and compact, so in order to avoid this situation (https://github.com/apache/incubator-pegasus/pull/587), rocksdb::LoadLatestOptions is called, and these parameters are initialized with the last value, so the last thing you see is that if the configuration file is modified, And without changing the usage scenario mode, these modified configurations will not take effect after restart.
   Options involved:
   ```
   level0_file_num_compaction_trigger
   level0_slowdown_writes_trigger
   level0_stop_writes_trigger
   soft_pending_compaction_bytes_limit
   hard_pending_compaction_bytes_limit
   disable_auto_compactions
   max_compaction_bytes
   write_buffer_size
   max_write_buffer_number
   ```
   Solution: 
   1. After opening rocksdb, calculate the value of the corresponding options according to the usage scenario mode in the environment variable, compare whether there is a change, and call setOpion() if there is a change to set the newly calculated value. 
   3. If the usage scenario mode has been changed during this process, the options will be the correct value, and nothing needs to be done.
   
   ##### Tests <!-- At least one of them must be included. -->
   
   - Unit test
   - Manual test (add detailed scripts or steps below)
   
   -


-- 
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: dev-unsubscribe@pegasus.apache.org

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 diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r944175767


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -1560,9 +1560,10 @@ dsn::error_code pegasus_server_impl::start(int argc, char **argv)
 
     ddebug_replica("start to open rocksDB's rdb({})", rdb_path);
 
-    // Here we create a `tmp_data_cf_opts` because we don't want to modify `_data_cf_opts`, which
+    // Here we create a `_tmp_data_cf_opts` because we don't want to modify `_data_cf_opts`, which
     // will be used elsewhere.
-    rocksdb::ColumnFamilyOptions tmp_data_cf_opts = _data_cf_opts;
+    _tmp_data_cf_opts = _data_cf_opts;

Review Comment:
   'tmp' is not good name for a member variable.
   hosw about rename '_data_cf_opts' to '_base_data_cf_opts', and rename '_tmp_data_cf_opts' to '_table_data_cf_opts' ?



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -2634,6 +2635,10 @@ void pegasus_server_impl::update_usage_scenario(const std::map<std::string, std:
                            old_usage_scenario,
                            new_usage_scenario);
         }
+    } else {
+        // When an old db is opened and the conf is changed, the options related to usage scenario
+        // need to be recalculated with new values.
+        recalculate_data_cf_options(_tmp_data_cf_opts);

Review Comment:
   If I didn't miss any thing, I found this function will not be called when open an exist db, right?



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,97 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_OPTION_IF_NEEDED(option, value, str_value)                                          \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        new_options[#option] = (str_value);                                                        \
+    }

Review Comment:
   How about:
   ```
   #define UPDATE_OPTION_IF_NEEDED(option, new_value)             \
       if (_data_cf_opts.option != cur_data_cf_opts.option) {     \
           new_options[#option] = #new_value;                     \
       }
   ```



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,97 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_OPTION_IF_NEEDED(option, value, str_value)                                          \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        new_options[#option] = (str_value);                                                        \
+    }
+
+    if (!_is_need_update_data_cf_opts)

Review Comment:
   What's the aim of this variable?



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

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

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 diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r947425771


##########
src/server/pegasus_server_impl.h:
##########
@@ -414,10 +426,14 @@ class pegasus_server_impl : public pegasus_read_service
     std::shared_ptr<rocksdb::Statistics> _statistics;
     rocksdb::DBOptions _db_opts;
     rocksdb::ColumnFamilyOptions _data_cf_opts;
+    rocksdb::ColumnFamilyOptions _table_data_cf_opts;

Review Comment:
   Add some comments to explaint the differece between `_data_cf_opts` and `_table_data_cf_opts`



##########
src/server/pegasus_server_impl.h:
##########
@@ -414,10 +426,14 @@ class pegasus_server_impl : public pegasus_read_service
     std::shared_ptr<rocksdb::Statistics> _statistics;
     rocksdb::DBOptions _db_opts;
     rocksdb::ColumnFamilyOptions _data_cf_opts;
+    rocksdb::ColumnFamilyOptions _table_data_cf_opts;
     rocksdb::ColumnFamilyOptions _meta_cf_opts;
     rocksdb::ReadOptions _data_cf_rd_opts;
     std::string _usage_scenario;
     std::string _user_specified_compaction;
+    // Whether it is necessary to update the current data_cf, it is required when opening the db,
+    // but not later
+    bool _is_need_update_data_cf_opts;

Review Comment:
   How aboout rename ot to `_table_data_cf_opts_recalculated` ? it would be closer with what you want to do.



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -2634,6 +2635,10 @@ void pegasus_server_impl::update_usage_scenario(const std::map<std::string, std:
                            old_usage_scenario,
                            new_usage_scenario);
         }
+    } else {
+        // When an old db is opened and the conf is changed, the options related to usage scenario

Review Comment:
   How about change 'the conf is changed' to 'the rocksDB related configs in server config.ini has been changed' ?



-- 
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: dev-unsubscribe@pegasus.apache.org

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] WHBANG commented on a diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
WHBANG commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r942343588


##########
src/server/pegasus_server_impl.h:
##########
@@ -325,6 +328,15 @@ class pegasus_server_impl : public pegasus_read_service
         return dsn::rand::next_u64(base_value - gap, base_value + gap);
     }
 
+    // return true if value in range of [0.75, 1.25] * base_value
+    bool check_value_if_nearby(uint64_t base_value, uint64_t check_value)
+    {
+        uint64_t gap = base_value / 4;
+        uint64_t actual_gap =
+            (base_value < check_value) ? check_value - base_value : base_value - check_value;

Review Comment:
    in this case, the following error will be reported:
   ```
   error: call of overloaded ‘abs(uint64_t)’ is ambiguous
   ```



-- 
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: dev-unsubscribe@pegasus.apache.org

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] WHBANG commented on a diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
WHBANG commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r944220277


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,97 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_OPTION_IF_NEEDED(option, value, str_value)                                          \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        new_options[#option] = (str_value);                                                        \
+    }
+
+    if (!_is_need_update_data_cf_opts)

Review Comment:
   Avoid `recalculate_data_cf_options` being called all the time



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

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

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 diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r945111380


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,97 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_OPTION_IF_NEEDED(option, value, str_value)                                          \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        new_options[#option] = (str_value);                                                        \
+    }

Review Comment:
   I meant value and str_value are always the same, you can use one instead.



-- 
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: dev-unsubscribe@pegasus.apache.org

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 diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r952116308


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,98 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_NUMBER_OPTION_IF_NEEDED(option, value)                                              \
+    do {                                                                                           \
+        if ((value) != cur_data_cf_opts.option) {                                                  \
+            new_options[#option] = std::to_string((value));                                        \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_BOOL_OPTION_IF_NEEDED(option, value)                                                \
+    do {                                                                                           \
+        if ((value) != cur_data_cf_opts.option) {                                                  \
+            if ((value)) {                                                                         \
+                new_options[#option] = "true";                                                     \
+            } else {                                                                               \
+                new_options[#option] = "false";                                                    \
+            }                                                                                      \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_OPTION_IF_NEEDED(option) UPDATE_NUMBER_OPTION_IF_NEEDED(option, _data_cf_opts.option)
+
+    if (_table_data_cf_opts_recalculated)
+        return;
+    std::unordered_map<std::string, std::string> new_options;
+    if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario ||
+        ROCKSDB_ENV_USAGE_SCENARIO_PREFER_WRITE == _usage_scenario) {
+        if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario) {
+            if (!check_value_if_nearby(_data_cf_opts.write_buffer_size,
+                                       cur_data_cf_opts.write_buffer_size)) {
+                new_options["write_buffer_size"] =
+                    std::to_string(get_random_nearby(_data_cf_opts.write_buffer_size));
+            }
+            UPDATE_OPTION_IF_NEEDED(level0_file_num_compaction_trigger);
+        } else {
+            uint64_t buffer_size = dsn::rand::next_u64(_data_cf_opts.write_buffer_size,
+                                                       _data_cf_opts.write_buffer_size * 2);
+            if (!(cur_data_cf_opts.write_buffer_size >= _data_cf_opts.write_buffer_size &&
+                  cur_data_cf_opts.write_buffer_size <= _data_cf_opts.write_buffer_size * 2)) {
+                new_options["write_buffer_size"] = std::to_string(buffer_size);
+                uint64_t max_size = get_random_nearby(_data_cf_opts.max_bytes_for_level_base);
+                new_options["level0_file_num_compaction_trigger"] =
+                    std::to_string(std::max<uint64_t>(4UL, max_size / buffer_size));
+            } else if (!check_value_if_nearby(_data_cf_opts.max_bytes_for_level_base,
+                                              cur_data_cf_opts.max_bytes_for_level_base)) {
+                uint64_t max_size = get_random_nearby(_data_cf_opts.max_bytes_for_level_base);
+                new_options["level0_file_num_compaction_trigger"] =
+                    std::to_string(std::max<uint64_t>(4UL, max_size / buffer_size));
+            }
+        }
+        UPDATE_OPTION_IF_NEEDED(level0_slowdown_writes_trigger);
+        UPDATE_OPTION_IF_NEEDED(level0_stop_writes_trigger);
+        UPDATE_OPTION_IF_NEEDED(soft_pending_compaction_bytes_limit);
+        UPDATE_OPTION_IF_NEEDED(hard_pending_compaction_bytes_limit);
+        UPDATE_BOOL_OPTION_IF_NEEDED(disable_auto_compactions, false);
+        UPDATE_OPTION_IF_NEEDED(max_compaction_bytes);
+        UPDATE_OPTION_IF_NEEDED(max_write_buffer_number);
+    } else {
+        // ROCKSDB_ENV_USAGE_SCENARIO_BULK_LOAD
+        UPDATE_NUMBER_OPTION_IF_NEEDED(level0_file_num_compaction_trigger, 1000000000);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(level0_slowdown_writes_trigger, 1000000000);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(level0_stop_writes_trigger, 1000000000);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(soft_pending_compaction_bytes_limit, 0);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(hard_pending_compaction_bytes_limit, 0);
+        UPDATE_BOOL_OPTION_IF_NEEDED(disable_auto_compactions, true);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(max_compaction_bytes, static_cast<uint64_t>(1) << 60);
+        if (!check_value_if_nearby(_data_cf_opts.write_buffer_size * 4,
+                                   cur_data_cf_opts.write_buffer_size)) {
+            new_options["write_buffer_size"] =
+                std::to_string(get_random_nearby(_data_cf_opts.write_buffer_size * 4));
+        }
+        if (cur_data_cf_opts.max_write_buffer_number !=
+            std::max(_data_cf_opts.max_write_buffer_number, 6)) {
+            new_options["max_write_buffer_number"] =
+                std::to_string(std::max(_data_cf_opts.max_write_buffer_number, 6));
+        }

Review Comment:
   Can be replaced by `UPDATE_NUMBER_OPTION_IF_NEEDED(max_write_buffer_number, std::max(_data_cf_opts.max_write_buffer_number, 6))` ?



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,98 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_NUMBER_OPTION_IF_NEEDED(option, value)                                              \
+    do {                                                                                           \
+        if ((value) != cur_data_cf_opts.option) {                                                  \

Review Comment:
   better to add a line:
   ```
   auto _v = (value);
   ```
   then use `_v` instead of `value`.



-- 
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: dev-unsubscribe@pegasus.apache.org

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] WHBANG commented on a diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
WHBANG commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r944198301


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -2634,6 +2635,10 @@ void pegasus_server_impl::update_usage_scenario(const std::map<std::string, std:
                            old_usage_scenario,
                            new_usage_scenario);
         }
+    } else {
+        // When an old db is opened and the conf is changed, the options related to usage scenario
+        // need to be recalculated with new values.
+        recalculate_data_cf_options(_tmp_data_cf_opts);

Review Comment:
   No, no matter what the last `usage scenario` is, it will change from 'normal' to the corresponding mode, so it will be executed once.
   here is a cyclic task, it will get the current `usage scenario`, in order to only execute once when the db is opened, so `is_need_update_data_cf_opts` variable is added to mark whether `recalculate_data_cf_options` has been called



-- 
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: dev-unsubscribe@pegasus.apache.org

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] WHBANG commented on a diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
WHBANG commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r944198301


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -2634,6 +2635,10 @@ void pegasus_server_impl::update_usage_scenario(const std::map<std::string, std:
                            old_usage_scenario,
                            new_usage_scenario);
         }
+    } else {
+        // When an old db is opened and the conf is changed, the options related to usage scenario
+        // need to be recalculated with new values.
+        recalculate_data_cf_options(_tmp_data_cf_opts);

Review Comment:
   No, no matter what the last `usage scenario` is, it will change from 'normal' to the corresponding mode, `usage_scenario == _usage_scenario`, so it will be executed once.
   here is a cyclic task, it will get the current `usage scenario`, in order to only execute once when the db is opened, so `is_need_update_data_cf_opts` variable is added to mark whether `recalculate_data_cf_options` has been called



-- 
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: dev-unsubscribe@pegasus.apache.org

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 diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r942289900


##########
src/server/pegasus_server_impl.h:
##########
@@ -325,6 +328,15 @@ class pegasus_server_impl : public pegasus_read_service
         return dsn::rand::next_u64(base_value - gap, base_value + gap);
     }
 
+    // return true if value in range of [0.75, 1.25] * base_value
+    bool check_value_if_nearby(uint64_t base_value, uint64_t check_value)
+    {
+        uint64_t gap = base_value / 4;
+        uint64_t actual_gap =
+            (base_value < check_value) ? check_value - base_value : base_value - check_value;

Review Comment:
   std::abs(check_value - base_value);



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -2634,6 +2634,10 @@ void pegasus_server_impl::update_usage_scenario(const std::map<std::string, std:
                            old_usage_scenario,
                            new_usage_scenario);
         }
+    } else {
+        // When an old db is opened and the conf is changed, the options related to usage scenario
+        // need to be recalculated with new values.
+        recalculate_usage_scenario(_tmp_data_cf_opts);

Review Comment:
   Usage scenario is not changed, maybe naming as recalculate_cf_options?



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3011,112 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_usage_scenario(const rocksdb::ColumnFamilyOptions &cur_opts)
+{
+    std::unordered_map<std::string, std::string> new_options;
+    if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario ||
+        ROCKSDB_ENV_USAGE_SCENARIO_PREFER_WRITE == _usage_scenario) {
+        if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario) {
+            if (!check_value_if_nearby(_data_cf_opts.write_buffer_size,
+                                       cur_opts.write_buffer_size)) {
+                new_options["write_buffer_size"] =
+                    std::to_string(get_random_nearby(_data_cf_opts.write_buffer_size));
+            }

Review Comment:
   There are many duplicate code, use a macro here will be easier to read.
   ```
   #define UPDATE_OPTION_IF_NEEDED(option, new_value) \
     ...
   ```
   `#` and `##` can be used in this macro.



-- 
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: dev-unsubscribe@pegasus.apache.org

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] WHBANG commented on a diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
WHBANG commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r945116285


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,97 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_OPTION_IF_NEEDED(option, value, str_value)                                          \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        new_options[#option] = (str_value);                                                        \
+    }

Review Comment:
   when `new_options[#option] = #new_value;`run  the following statement:
   ```
   UPDATE_OPTION_IF_NEEDED(level0_slowdown_writes_trigger,_data_cf_opts.level0_slowdown_writes_trigger)
   ```
   result is:
   ```
   new_options["level0_slowdown_writes_trigger"] = "_data_cf_opts.level0_slowdown_writes_trigger"
   ```



-- 
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: dev-unsubscribe@pegasus.apache.org

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 merged pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
acelyc111 merged PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108


-- 
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: dev-unsubscribe@pegasus.apache.org

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] empiredan commented on a diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
empiredan commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r953794855


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,96 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_NUMBER_OPTION_IF_NEEDED(option, value)                                              \
+    do {                                                                                           \
+        auto _v = (value);                                                                         \
+        if (_v != cur_data_cf_opts.option) {                                                       \
+            new_options[#option] = std::to_string(_v);                                             \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_BOOL_OPTION_IF_NEEDED(option, value)                                                \
+    do {                                                                                           \
+        if ((value) != cur_data_cf_opts.option) {                                                  \

Review Comment:
   Should `(value)` be assigned to a variable (e.g. `auto _v = (value);`), just like what has been done in `UPDATE_NUMBER_OPTION_IF_NEEDED`, to avoid evaluating 2 times ?



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,96 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_NUMBER_OPTION_IF_NEEDED(option, value)                                              \
+    do {                                                                                           \
+        auto _v = (value);                                                                         \
+        if (_v != cur_data_cf_opts.option) {                                                       \
+            new_options[#option] = std::to_string(_v);                                             \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_BOOL_OPTION_IF_NEEDED(option, value)                                                \
+    do {                                                                                           \
+        if ((value) != cur_data_cf_opts.option) {                                                  \
+            if ((value)) {                                                                         \
+                new_options[#option] = "true";                                                     \
+            } else {                                                                               \
+                new_options[#option] = "false";                                                    \
+            }                                                                                      \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_OPTION_IF_NOT_NEARBY(option, value)                                                 \
+    do {                                                                                           \
+        auto _v = (value);                                                                         \
+        if (!check_value_if_nearby(_v, cur_data_cf_opts.option)) {                                 \
+            new_options[#option] = std::to_string(get_random_nearby(_v));                          \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_OPTION_IF_NEEDED(option) UPDATE_NUMBER_OPTION_IF_NEEDED(option, _data_cf_opts.option)
+
+    if (_table_data_cf_opts_recalculated)
+        return;

Review Comment:
   ```c++
   if (_table_data_cf_opts_recalculated) {
       return;
   }
   ```



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,96 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_NUMBER_OPTION_IF_NEEDED(option, value)                                              \
+    do {                                                                                           \
+        auto _v = (value);                                                                         \
+        if (_v != cur_data_cf_opts.option) {                                                       \
+            new_options[#option] = std::to_string(_v);                                             \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_BOOL_OPTION_IF_NEEDED(option, value)                                                \
+    do {                                                                                           \
+        if ((value) != cur_data_cf_opts.option) {                                                  \
+            if ((value)) {                                                                         \
+                new_options[#option] = "true";                                                     \
+            } else {                                                                               \
+                new_options[#option] = "false";                                                    \
+            }                                                                                      \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_OPTION_IF_NOT_NEARBY(option, value)                                                 \
+    do {                                                                                           \
+        auto _v = (value);                                                                         \
+        if (!check_value_if_nearby(_v, cur_data_cf_opts.option)) {                                 \
+            new_options[#option] = std::to_string(get_random_nearby(_v));                          \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_OPTION_IF_NEEDED(option) UPDATE_NUMBER_OPTION_IF_NEEDED(option, _data_cf_opts.option)
+
+    if (_table_data_cf_opts_recalculated)
+        return;
+    std::unordered_map<std::string, std::string> new_options;
+    if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario ||
+        ROCKSDB_ENV_USAGE_SCENARIO_PREFER_WRITE == _usage_scenario) {
+        if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario) {
+            UPDATE_OPTION_IF_NOT_NEARBY(write_buffer_size, _data_cf_opts.write_buffer_size);
+            UPDATE_OPTION_IF_NEEDED(level0_file_num_compaction_trigger);
+        } else {
+            uint64_t buffer_size = dsn::rand::next_u64(_data_cf_opts.write_buffer_size,
+                                                       _data_cf_opts.write_buffer_size * 2);
+            if (!(cur_data_cf_opts.write_buffer_size >= _data_cf_opts.write_buffer_size &&
+                  cur_data_cf_opts.write_buffer_size <= _data_cf_opts.write_buffer_size * 2)) {

Review Comment:
   ```c++ 
   if (cur_data_cf_opts.write_buffer_size < _data_cf_opts.write_buffer_size ||
       cur_data_cf_opts.write_buffer_size > _data_cf_opts.write_buffer_size * 2)
   ```



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

To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org

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 diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r945110249


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -2634,6 +2635,10 @@ void pegasus_server_impl::update_usage_scenario(const std::map<std::string, std:
                            old_usage_scenario,
                            new_usage_scenario);
         }
+    } else {
+        // When an old db is opened and the conf is changed, the options related to usage scenario
+        // need to be recalculated with new values.
+        recalculate_data_cf_options(_tmp_data_cf_opts);

Review Comment:
   I see



-- 
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: dev-unsubscribe@pegasus.apache.org

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] WHBANG commented on a diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
WHBANG commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r945116285


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,97 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_OPTION_IF_NEEDED(option, value, str_value)                                          \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        new_options[#option] = (str_value);                                                        \
+    }

Review Comment:
   when `new_options[#option] = #new_value;`run  the following statement:
   ```
   UPDATE_OPTION_IF_NEEDED(level0_slowdown_writes_trigger,_data_cf_opts.level0_slowdown_writes_trigger)
   ```
   result is bad:
   ```
   new_options["level0_slowdown_writes_trigger"] = "_data_cf_opts.level0_slowdown_writes_trigger"
   ```



-- 
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: dev-unsubscribe@pegasus.apache.org

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 diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r947432931


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,93 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_NUMBER_OPTION_IF_NEEDED(option, value)                                              \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        new_options[#option] = std::to_string((value));                                            \
+    }
+#define UPDATE_BOOL_OPTION_IF_NEEDED(option, value)                                                \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        if ((value))                                                                               \
+            new_options[#option] = "true";                                                         \
+        else                                                                                       \
+            new_options[#option] = "false";                                                        \
+    }
+
+    if (!_is_need_update_data_cf_opts)
+        return;
+    std::unordered_map<std::string, std::string> new_options;
+    if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario ||
+        ROCKSDB_ENV_USAGE_SCENARIO_PREFER_WRITE == _usage_scenario) {
+        if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario) {
+            if (!check_value_if_nearby(_data_cf_opts.write_buffer_size,
+                                       cur_data_cf_opts.write_buffer_size)) {
+                new_options["write_buffer_size"] =
+                    std::to_string(get_random_nearby(_data_cf_opts.write_buffer_size));
+            }
+            UPDATE_NUMBER_OPTION_IF_NEEDED(level0_file_num_compaction_trigger,
+                                           _data_cf_opts.level0_file_num_compaction_trigger);
+        } else {
+            uint64_t buffer_size = dsn::rand::next_u64(_data_cf_opts.write_buffer_size,
+                                                       _data_cf_opts.write_buffer_size * 2);
+            if (!(cur_data_cf_opts.write_buffer_size >= _data_cf_opts.write_buffer_size &&
+                  cur_data_cf_opts.write_buffer_size <= _data_cf_opts.write_buffer_size * 2)) {
+                new_options["write_buffer_size"] = std::to_string(buffer_size);
+                uint64_t max_size = get_random_nearby(_data_cf_opts.max_bytes_for_level_base);
+                new_options["level0_file_num_compaction_trigger"] =
+                    std::to_string(std::max<uint64_t>(4UL, max_size / buffer_size));
+            } else if (!check_value_if_nearby(_data_cf_opts.max_bytes_for_level_base,
+                                              cur_data_cf_opts.max_bytes_for_level_base)) {
+                uint64_t max_size = get_random_nearby(_data_cf_opts.max_bytes_for_level_base);
+                new_options["level0_file_num_compaction_trigger"] =
+                    std::to_string(std::max<uint64_t>(4UL, max_size / buffer_size));
+            }
+        }
+        UPDATE_NUMBER_OPTION_IF_NEEDED(level0_slowdown_writes_trigger,
+                                       _data_cf_opts.level0_slowdown_writes_trigger);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(level0_stop_writes_trigger,
+                                       _data_cf_opts.level0_stop_writes_trigger);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(soft_pending_compaction_bytes_limit,
+                                       _data_cf_opts.soft_pending_compaction_bytes_limit);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(hard_pending_compaction_bytes_limit,
+                                       _data_cf_opts.hard_pending_compaction_bytes_limit);

Review Comment:
   how about define another macro, to leave only one parameter, change "XXX(xxx, _data_cf_opts.xxx)" to "XXX(xxx)"?



-- 
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: dev-unsubscribe@pegasus.apache.org

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 diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r949849415


##########
src/server/pegasus_server_impl.h:
##########
@@ -413,11 +425,18 @@ class pegasus_server_impl : public pegasus_read_service
     std::shared_ptr<KeyWithTTLCompactionFilterFactory> _key_ttl_compaction_filter_factory;
     std::shared_ptr<rocksdb::Statistics> _statistics;
     rocksdb::DBOptions _db_opts;
+    // The value of option in data_cf according to conf template file config.ini
     rocksdb::ColumnFamilyOptions _data_cf_opts;
+    // Dynamically calculate the value of current data_cf option according to the conf module file
+    // and usage mode

Review Comment:
   ```suggestion
       // and usage scenario
   ```



##########
src/server/pegasus_server_impl.h:
##########
@@ -413,11 +425,18 @@ class pegasus_server_impl : public pegasus_read_service
     std::shared_ptr<KeyWithTTLCompactionFilterFactory> _key_ttl_compaction_filter_factory;
     std::shared_ptr<rocksdb::Statistics> _statistics;
     rocksdb::DBOptions _db_opts;
+    // The value of option in data_cf according to conf template file config.ini
     rocksdb::ColumnFamilyOptions _data_cf_opts;
+    // Dynamically calculate the value of current data_cf option according to the conf module file
+    // and usage mode
+    rocksdb::ColumnFamilyOptions _table_data_cf_opts;
     rocksdb::ColumnFamilyOptions _meta_cf_opts;
     rocksdb::ReadOptions _data_cf_rd_opts;
     std::string _usage_scenario;
     std::string _user_specified_compaction;
+    // Whether it is necessary to update the current data_cf, it is required when opening the db,
+    // but not later
+    bool _table_data_cf_opts_recalculated;

Review Comment:
   the name `_table_data_cf_opts_recalculated` means option has been **recalculated**, it's different with the meaning you defined before, that is to say, true and flase should reverse.



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,97 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_NUMBER_OPTION_IF_NEEDED(option, value)                                              \
+    do {                                                                                           \
+        if ((value) != cur_data_cf_opts.option) {                                                  \
+            new_options[#option] = std::to_string((value));                                        \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_BOOL_OPTION_IF_NEEDED(option, value)                                                \
+    do {                                                                                           \
+        if ((value) != cur_data_cf_opts.option) {                                                  \
+            if ((value))                                                                           \

Review Comment:
   it's recommend to keep `{}` for `if..else..` even though there is only one line.
   some other places are the same.



##########
src/server/pegasus_server_impl.h:
##########
@@ -413,11 +425,18 @@ class pegasus_server_impl : public pegasus_read_service
     std::shared_ptr<KeyWithTTLCompactionFilterFactory> _key_ttl_compaction_filter_factory;
     std::shared_ptr<rocksdb::Statistics> _statistics;
     rocksdb::DBOptions _db_opts;
+    // The value of option in data_cf according to conf template file config.ini
     rocksdb::ColumnFamilyOptions _data_cf_opts;
+    // Dynamically calculate the value of current data_cf option according to the conf module file
+    // and usage mode
+    rocksdb::ColumnFamilyOptions _table_data_cf_opts;
     rocksdb::ColumnFamilyOptions _meta_cf_opts;
     rocksdb::ReadOptions _data_cf_rd_opts;
     std::string _usage_scenario;
     std::string _user_specified_compaction;
+    // Whether it is necessary to update the current data_cf, it is required when opening the db,
+    // but not later

Review Comment:
   ```suggestion
       // Whether it is necessary to update the current data_cf, it is required when opening the db at the first time,
       // but not later
   ```



-- 
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: dev-unsubscribe@pegasus.apache.org

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] WHBANG commented on a diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
WHBANG commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r944218454


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,97 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_OPTION_IF_NEEDED(option, value, str_value)                                          \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        new_options[#option] = (str_value);                                                        \
+    }

Review Comment:
   `if (_data_cf_opts.option != cur_data_cf_opts.option) { ` is wrong
   use level0_file_num_compaction_trigger as an example: 
   1. start_onebox, do not change the config template
   ```
   cur_data_cf_opts.level0_file_num_compaction_trigger = 4
   _data_cf_opts.level0_file_num_compaction_trigger = 4
   ```
   2. set `usage scenario`is `bulk load`
   ```
   cur_data_cf_opts.level0_file_num_compaction_trigger = 1000000000
   _data_cf_opts.level0_file_num_compaction_trigger = 4
   ```
   3. restart the db, will call LoadLatestOptions() 
   ```
   cur_data_cf_opts.level0_file_num_compaction_trigger = 1000000000
   _data_cf_opts.level0_file_num_compaction_trigger = 4
   ```
   At this time `_data_cf_opts.option != cur_data_cf_opts.option`, but in fact the value of option is correct, nothing needs to be done, so need `(value) != cur_data_cf_opts.option) `



-- 
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: dev-unsubscribe@pegasus.apache.org

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 diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r952118712


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,98 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_NUMBER_OPTION_IF_NEEDED(option, value)                                              \
+    do {                                                                                           \
+        if ((value) != cur_data_cf_opts.option) {                                                  \
+            new_options[#option] = std::to_string((value));                                        \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_BOOL_OPTION_IF_NEEDED(option, value)                                                \
+    do {                                                                                           \
+        if ((value) != cur_data_cf_opts.option) {                                                  \
+            if ((value)) {                                                                         \
+                new_options[#option] = "true";                                                     \
+            } else {                                                                               \
+                new_options[#option] = "false";                                                    \
+            }                                                                                      \
+        }                                                                                          \
+    } while (0)
+
+#define UPDATE_OPTION_IF_NEEDED(option) UPDATE_NUMBER_OPTION_IF_NEEDED(option, _data_cf_opts.option)
+
+    if (_table_data_cf_opts_recalculated)
+        return;
+    std::unordered_map<std::string, std::string> new_options;
+    if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario ||
+        ROCKSDB_ENV_USAGE_SCENARIO_PREFER_WRITE == _usage_scenario) {
+        if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario) {
+            if (!check_value_if_nearby(_data_cf_opts.write_buffer_size,
+                                       cur_data_cf_opts.write_buffer_size)) {
+                new_options["write_buffer_size"] =
+                    std::to_string(get_random_nearby(_data_cf_opts.write_buffer_size));
+            }

Review Comment:
   how about add another mnacro like `UPDATE_OPTION_IF_NOT_NEARBY`?



-- 
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: dev-unsubscribe@pegasus.apache.org

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] empiredan commented on a diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
empiredan commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r948879532


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,88 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_NUMBER_OPTION_IF_NEEDED(option, value)                                              \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        new_options[#option] = std::to_string((value));                                            \
+    }

Review Comment:
   better to use `do { ... } while (0)` ?



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -3007,6 +3013,88 @@ void pegasus_server_impl::reset_usage_scenario_options(
     target_opts->max_write_buffer_number = base_opts.max_write_buffer_number;
 }
 
+void pegasus_server_impl::recalculate_data_cf_options(
+    const rocksdb::ColumnFamilyOptions &cur_data_cf_opts)
+{
+#define UPDATE_NUMBER_OPTION_IF_NEEDED(option, value)                                              \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        new_options[#option] = std::to_string((value));                                            \
+    }
+#define UPDATE_BOOL_OPTION_IF_NEEDED(option, value)                                                \
+    if ((value) != cur_data_cf_opts.option) {                                                      \
+        if ((value))                                                                               \
+            new_options[#option] = "true";                                                         \
+        else                                                                                       \
+            new_options[#option] = "false";                                                        \
+    }
+#define UPDATE_OPTION_IF_NEEDED(option) UPDATE_NUMBER_OPTION_IF_NEEDED(option, _data_cf_opts.option)
+
+    if (!_table_data_cf_opts_recalculated)
+        return;
+    std::unordered_map<std::string, std::string> new_options;
+    if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario ||
+        ROCKSDB_ENV_USAGE_SCENARIO_PREFER_WRITE == _usage_scenario) {
+        if (ROCKSDB_ENV_USAGE_SCENARIO_NORMAL == _usage_scenario) {
+            if (!check_value_if_nearby(_data_cf_opts.write_buffer_size,
+                                       cur_data_cf_opts.write_buffer_size)) {
+                new_options["write_buffer_size"] =
+                    std::to_string(get_random_nearby(_data_cf_opts.write_buffer_size));
+            }
+            UPDATE_OPTION_IF_NEEDED(level0_file_num_compaction_trigger);
+        } else {
+            uint64_t buffer_size = dsn::rand::next_u64(_data_cf_opts.write_buffer_size,
+                                                       _data_cf_opts.write_buffer_size * 2);
+            if (!(cur_data_cf_opts.write_buffer_size >= _data_cf_opts.write_buffer_size &&
+                  cur_data_cf_opts.write_buffer_size <= _data_cf_opts.write_buffer_size * 2)) {
+                new_options["write_buffer_size"] = std::to_string(buffer_size);
+                uint64_t max_size = get_random_nearby(_data_cf_opts.max_bytes_for_level_base);
+                new_options["level0_file_num_compaction_trigger"] =
+                    std::to_string(std::max<uint64_t>(4UL, max_size / buffer_size));
+            } else if (!check_value_if_nearby(_data_cf_opts.max_bytes_for_level_base,
+                                              cur_data_cf_opts.max_bytes_for_level_base)) {
+                uint64_t max_size = get_random_nearby(_data_cf_opts.max_bytes_for_level_base);
+                new_options["level0_file_num_compaction_trigger"] =
+                    std::to_string(std::max<uint64_t>(4UL, max_size / buffer_size));
+            }
+        }
+        UPDATE_OPTION_IF_NEEDED(level0_slowdown_writes_trigger);
+        UPDATE_OPTION_IF_NEEDED(level0_stop_writes_trigger);
+        UPDATE_OPTION_IF_NEEDED(soft_pending_compaction_bytes_limit);
+        UPDATE_OPTION_IF_NEEDED(hard_pending_compaction_bytes_limit);
+        UPDATE_BOOL_OPTION_IF_NEEDED(disable_auto_compactions, false);
+        UPDATE_OPTION_IF_NEEDED(max_compaction_bytes);
+        UPDATE_OPTION_IF_NEEDED(max_write_buffer_number);
+    } else {
+        // ROCKSDB_ENV_USAGE_SCENARIO_BULK_LOAD
+        UPDATE_NUMBER_OPTION_IF_NEEDED(level0_file_num_compaction_trigger, 1000000000);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(level0_slowdown_writes_trigger, 1000000000);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(level0_stop_writes_trigger, 1000000000);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(soft_pending_compaction_bytes_limit, 0);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(hard_pending_compaction_bytes_limit, 0);
+        UPDATE_BOOL_OPTION_IF_NEEDED(disable_auto_compactions, true);
+        UPDATE_NUMBER_OPTION_IF_NEEDED(max_compaction_bytes, static_cast<uint64_t>(1) << 60);
+        if (!check_value_if_nearby(_data_cf_opts.write_buffer_size * 4,
+                                   cur_data_cf_opts.write_buffer_size)) {
+            new_options["write_buffer_size"] =
+                std::to_string(get_random_nearby(_data_cf_opts.write_buffer_size * 4));
+        }
+        if (cur_data_cf_opts.max_write_buffer_number !=
+            std::max(_data_cf_opts.max_write_buffer_number, 6)) {
+            new_options["max_write_buffer_number"] =
+                std::to_string(std::max(_data_cf_opts.max_write_buffer_number, 6));
+        }
+    }
+    if (new_options.size() > 0) {
+        if (set_options(new_options)) {
+            ddebug_replica(
+                "{}: recalculate the value of the options related to usage scenario \"{}\"",
+                replica_name(),
+                _usage_scenario);
+        }
+    }
+    _table_data_cf_opts_recalculated = false;

Review Comment:
   ```c++
   #undef UPDATE_OPTION_IF_NEEDED
   #undef UPDATE_BOOL_OPTION_IF_NEEDED
   #undef UPDATE_NUMBER_OPTION_IF_NEEDED
   ```



-- 
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: dev-unsubscribe@pegasus.apache.org

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] WHBANG commented on a diff in pull request #1108: fix: rocksdb options not changed even if update in Pegasus config file

Posted by GitBox <gi...@apache.org>.
WHBANG commented on code in PR #1108:
URL: https://github.com/apache/incubator-pegasus/pull/1108#discussion_r944228802


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -1560,9 +1560,10 @@ dsn::error_code pegasus_server_impl::start(int argc, char **argv)
 
     ddebug_replica("start to open rocksDB's rdb({})", rdb_path);
 
-    // Here we create a `tmp_data_cf_opts` because we don't want to modify `_data_cf_opts`, which
+    // Here we create a `_tmp_data_cf_opts` because we don't want to modify `_data_cf_opts`, which
     // will be used elsewhere.
-    rocksdb::ColumnFamilyOptions tmp_data_cf_opts = _data_cf_opts;
+    _tmp_data_cf_opts = _data_cf_opts;

Review Comment:
   `_data_cf_opts` involves too many files, not necessary.
   i will rename `_tmp_data_cf_opts ` to `_table_data_cf_opts `



-- 
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: dev-unsubscribe@pegasus.apache.org

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