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/09 13:40:43 UTC

[GitHub] [incubator-pegasus] Smityz opened a new pull request #641: fix(hotspot): replace partition_resolver to ddl_client

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


   After `detect_hotkey_request` was removed from rrdb_types, `partition_resolver` needs to be replaced to send RPC correctly.


----------------------------------------------------------------
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 #641: fix(hotspot): replace partition_resolver to ddl_client

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



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

Review comment:
       +1




----------------------------------------------------------------
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 #641: fix(hotspot): replace partition_resolver to ddl_client

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



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

Review comment:
       Move closer to the place where you use it, just before `_shell_context->ddl_client->detect_hotkey `

##########
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;
+
+    auto error = _shell_context->ddl_client->detect_hotkey(target_address, req, resp);
+

Review comment:
       ```suggestion
   ```




----------------------------------------------------------------
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 #641: fix(hotspot): replace partition_resolver to ddl_client

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



##########
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:
       I will solve it in the other PR




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

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 #641: fix(hotspot): replace partition_resolver to ddl_client

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



##########
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:
       Yes, it will ensure




----------------------------------------------------------------
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 #641: fix(hotspot): replace partition_resolver to ddl_client

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



##########
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:
       There is no necessary to replace the raw pointer to shared_ptr, because `_shell_context` will not be destructed under any circumstances.
   




----------------------------------------------------------------
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 commented on a change in pull request #641: fix(hotspot): replace partition_resolver to ddl_client

Posted by GitBox <gi...@apache.org>.
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


[GitHub] [incubator-pegasus] acelyc111 merged pull request #641: fix(hotspot): replace partition_resolver to ddl_client

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


   


----------------------------------------------------------------
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 #641: fix(hotspot): replace partition_resolver to ddl_client

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



##########
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:
       There will be a lot of `hotspot_partition_calculator`(s) in one database, I think it is better to share one instance




----------------------------------------------------------------
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 #641: fix(hotspot): replace partition_resolver to ddl_client

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



##########
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:
       you can change the type for first param of  get_app_partition_stat to std::shared_ptr. avoid to use raw pointer




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