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 2020/11/11 06:17:44 UTC

[GitHub] [incubator-pegasus] hycdong commented on a change in pull request #641: fix(hotspot): replace partition_resolver to ddl_client

hycdong commented on a change in pull request #641:
URL: https://github.com/apache/incubator-pegasus/pull/641#discussion_r521133715



##########
File path: src/server/info_collector.cpp
##########
@@ -316,7 +316,8 @@ info_collector::get_hotspot_calculator(const std::string &app_name, const int pa
     if (iter != _hotspot_calculator_store.end()) {
         return iter->second;
     }
-    auto calculator = std::make_shared<hotspot_partition_calculator>(app_name, partition_count);
+    auto calculator =
+        std::make_shared<hotspot_partition_calculator>(app_name, partition_count, _shell_context);

Review comment:
       I think ddl_client is necessary for `hotspot_partition_calculator`, you can create a new ddl client instance for calculator, not share the shell_context with info_collector.

##########
File path: src/server/info_collector.cpp
##########
@@ -57,9 +57,9 @@ info_collector::info_collector()
 
     _cluster_name = dsn::replication::get_current_cluster_name();
 
-    _shell_context.current_cluster_name = _cluster_name;
-    _shell_context.meta_list = meta_servers;
-    _shell_context.ddl_client.reset(new replication_ddl_client(meta_servers));
+    _shell_context->current_cluster_name = _cluster_name;
+    _shell_context->meta_list = meta_servers;
+    _shell_context->ddl_client.reset(new replication_ddl_client(meta_servers));

Review comment:
       It seems that `_shell_context` is not initialized.

##########
File path: src/server/hotspot_partition_calculator.cpp
##########
@@ -180,44 +177,39 @@ void hotspot_partition_calculator::detect_hotkey_in_hotpartition(int data_type)
     const dsn::replication::detect_action::type action)
 {
     FAIL_POINT_INJECT_F("send_detect_hotkey_request", [](dsn::string_view) {});
-    auto request = dsn::make_unique<dsn::replication::detect_hotkey_request>();
-    request->type = hotkey_type;
-    request->action = action;
+
+    dsn::replication::detect_hotkey_request req;
+    req.type = hotkey_type;
+    req.action = action;
+    dsn::replication::detect_hotkey_response resp;
+
     ddebug_f("{} {} hotkey detection in {}.{}",
              (action == dsn::replication::detect_action::STOP) ? "Stop" : "Start",
              (hotkey_type == dsn::replication::hotkey_type::WRITE) ? "write" : "read",
              app_name,
              partition_index);
-    dsn::rpc_address meta_server;
-    meta_server.assign_group("meta-servers");
-    std::vector<dsn::rpc_address> meta_servers;
-    replica_helper::load_meta_servers(meta_servers);
-    for (const auto &address : meta_servers) {
-        meta_server.group_address()->add(address);
+
+    int app_id;
+    int partition_count;
+    std::vector<dsn::partition_configuration> partitions;
+    _shell_context->ddl_client->list_app(app_name, app_id, partition_count, partitions);
+    auto target_address = partitions[partition_index].primary;
+

Review comment:
       Move the debug log here, and add `target_address` in this log.

##########
File path: src/server/hotspot_partition_calculator.h
##########
@@ -37,7 +37,9 @@ typedef std::vector<std::array<dsn::perf_counter_wrapper, 2>> hot_partition_coun
 class hotspot_partition_calculator
 {
 public:
-    hotspot_partition_calculator(const std::string &app_name, int partition_count)
+    hotspot_partition_calculator(const std::string &app_name,
+                                 int partition_count,
+                                 std::shared_ptr<shell_context> context)

Review comment:
       It seems that _shell_context is not initialized.

##########
File path: src/server/hotspot_partition_calculator.cpp
##########
@@ -180,44 +177,39 @@ void hotspot_partition_calculator::detect_hotkey_in_hotpartition(int data_type)
     const dsn::replication::detect_action::type action)
 {
     FAIL_POINT_INJECT_F("send_detect_hotkey_request", [](dsn::string_view) {});
-    auto request = dsn::make_unique<dsn::replication::detect_hotkey_request>();
-    request->type = hotkey_type;
-    request->action = action;
+
+    dsn::replication::detect_hotkey_request req;
+    req.type = hotkey_type;
+    req.action = action;
+    dsn::replication::detect_hotkey_response resp;
+
     ddebug_f("{} {} hotkey detection in {}.{}",
              (action == dsn::replication::detect_action::STOP) ? "Stop" : "Start",
              (hotkey_type == dsn::replication::hotkey_type::WRITE) ? "write" : "read",
              app_name,
              partition_index);
-    dsn::rpc_address meta_server;
-    meta_server.assign_group("meta-servers");
-    std::vector<dsn::rpc_address> meta_servers;
-    replica_helper::load_meta_servers(meta_servers);
-    for (const auto &address : meta_servers) {
-        meta_server.group_address()->add(address);
+
+    int app_id;
+    int partition_count;
+    std::vector<dsn::partition_configuration> partitions;
+    _shell_context->ddl_client->list_app(app_name, app_id, partition_count, partitions);
+    auto target_address = partitions[partition_index].primary;

Review comment:
       It seems that you don't validate the `partition_index`, will the caller ensure that the `partition_index` is valid?

##########
File path: src/server/info_collector.cpp
##########
@@ -143,7 +143,7 @@ void info_collector::on_app_stat()
 {
     ddebug("start to stat apps");
     std::map<std::string, std::vector<row_data>> all_rows;
-    if (!get_app_partition_stat(&_shell_context, all_rows)) {
+    if (!get_app_partition_stat(_shell_context.get(), all_rows)) {

Review comment:
       +1, and I recommend that raise another pull request to do it, including update `_shell_context` to std::shared_ptr, and update functions like `get_app_partition_stat`




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