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 2023/01/19 06:47:26 UTC

[GitHub] [incubator-pegasus] WHBANG opened a new pull request, #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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

   ### What problem does this PR solve? <!--add issue link with summary if exists-->
   https://github.com/apache/incubator-pegasus/issues/1305
   
   This patch aim to use LOG_ERROR_F instead of LOG_ERROR for all the remain parts:
   
   Remove the defination of LOG_ERROR
   Use LOG_ERROR_F instead of LOG_ERROR
   Some format fixes
   


-- 
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 #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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


##########
src/meta/meta_backup_service.cpp:
##########
@@ -625,10 +625,10 @@ void policy_context::sync_backup_to_remote_storage_unlocked(const backup_info &b
                 LOG_WARNING_F("{}: empty callback", _policy.policy_name);
             }
         } else if (ERR_TIMEOUT == err) {
-            LOG_ERROR("%s: sync backup info(" PRId64
-                      ") to remote storage got timeout, retry it later",
-                      _policy.policy_name.c_str(),
-                      b_info.backup_id);
+            LOG_ERROR_F(
+                "{}: sync backup info({:#018x}) to remote storage got timeout, retry it later",

Review Comment:
   fixed



##########
src/nfs/nfs_server_impl.cpp:
##########
@@ -208,9 +207,9 @@ void nfs_service_impl::on_get_file_size(
 
             struct stat st;
             if (0 != ::stat(file_path.c_str(), &st)) {
-                LOG_ERROR("{nfs_service} get stat of file %s failed, err = %s",
-                          file_path.c_str(),
-                          strerror(errno));
+                LOG_ERROR_F("[nfs_service] get stat of file {} failed, err = {}",
+                            file_path,
+                            strerror(errno));

Review Comment:
   done



##########
src/runtime/rpc/asio_net_provider.cpp:
##########
@@ -148,7 +148,7 @@ void asio_network_provider::do_accept()
         if (!ec) {
             auto remote = socket->remote_endpoint(ec);
             if (ec) {
-                LOG_ERROR("failed to get the remote endpoint: %s", ec.message().data());
+                LOG_ERROR_F("failed to get the remote endpoint: {}", ec.message().data());

Review Comment:
   done



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

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 #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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


##########
src/failure_detector/failure_detector.cpp:
##########
@@ -227,13 +227,13 @@ void failure_detector::check_all_records()
                 is_time_greater_than(now, record.last_send_time_for_beacon_with_ack) &&
                 now + _check_interval_milliseconds - record.last_send_time_for_beacon_with_ack >
                     _lease_milliseconds) {
-                LOG_ERROR("master %s disconnected, now=%" PRId64 ", last_send_time=%" PRId64
-                          ", now+check_interval-last_send_time=%" PRId64,
-                          record.node.to_string(),
-                          now,
-                          record.last_send_time_for_beacon_with_ack,
-                          now + _check_interval_milliseconds -
-                              record.last_send_time_for_beacon_with_ack);
+                LOG_ERROR_F("master {} disconnected, now={:#018x}, last_send_time={:#018x}, "

Review Comment:
   is it necessary to use `:#018x`?



##########
src/client/partition_resolver_simple.cpp:
##########
@@ -311,28 +311,28 @@ void partition_resolver_simple::query_config_reply(error_code err,
                 }
             }
         } else if (resp.err == ERR_OBJECT_NOT_FOUND) {
-            LOG_ERROR("%s.client: query config reply, gpid = %d.%d, err = %s",
-                      _app_name.c_str(),
-                      _app_id,
-                      partition_index,
-                      resp.err.to_string());
+            LOG_ERROR_F("{}.client: query config reply, gpid = {}.{}, err = {}",

Review Comment:
   You can use `LOG_ERROR_PREFIX` macro instead.



##########
src/meta/meta_backup_service.cpp:
##########
@@ -625,10 +625,10 @@ void policy_context::sync_backup_to_remote_storage_unlocked(const backup_info &b
                 LOG_WARNING_F("{}: empty callback", _policy.policy_name);
             }
         } else if (ERR_TIMEOUT == err) {
-            LOG_ERROR("%s: sync backup info(" PRId64
-                      ") to remote storage got timeout, retry it later",
-                      _policy.policy_name.c_str(),
-                      b_info.backup_id);
+            LOG_ERROR_F(
+                "{}: sync backup info({:#018x}) to remote storage got timeout, retry it later",

Review Comment:
   why use `:#018x` ?



##########
src/runtime/rpc/asio_net_provider.cpp:
##########
@@ -148,7 +148,7 @@ void asio_network_provider::do_accept()
         if (!ec) {
             auto remote = socket->remote_endpoint(ec);
             if (ec) {
-                LOG_ERROR("failed to get the remote endpoint: %s", ec.message().data());
+                LOG_ERROR_F("failed to get the remote endpoint: {}", ec.message().data());

Review Comment:
   ec.message() is ok?



##########
src/nfs/nfs_server_impl.cpp:
##########
@@ -208,9 +207,9 @@ void nfs_service_impl::on_get_file_size(
 
             struct stat st;
             if (0 != ::stat(file_path.c_str(), &st)) {
-                LOG_ERROR("{nfs_service} get stat of file %s failed, err = %s",
-                          file_path.c_str(),
-                          strerror(errno));
+                LOG_ERROR_F("[nfs_service] get stat of file {} failed, err = {}",
+                            file_path,
+                            strerror(errno));

Review Comment:
   it's recommend to use `dsn::utils::safe_strerror` 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] WHBANG commented on a diff in pull request #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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


##########
src/server/pegasus_server_impl.cpp:
##########
@@ -365,11 +359,9 @@ void pegasus_server_impl::on_multi_get(multi_get_rpc rpc)
     }
 
     if (!is_filter_type_supported(request.sort_key_filter_type)) {
-        LOG_ERROR("%s: invalid argument for multi_get from %s: "
-                  "sort key filter type %d not supported",
-                  replica_name(),
-                  rpc.remote_address().to_string(),
-                  request.sort_key_filter_type);
+        LOG_ERROR_F("invalid argument for multi_get from {}: sort key filter type {} not supported",

Review Comment:
   done



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

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 #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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


##########
src/meta/meta_backup_service.cpp:
##########
@@ -1363,10 +1357,10 @@ void backup_service::do_update_policy_to_remote_storage(
                            p.policy_name);
                 p_context_ptr->set_policy(p);
             } else if (err == ERR_TIMEOUT) {
-                LOG_ERROR("update backup policy to remote storage failed, policy_name = %s, retry "
-                          "after %" PRId64 "(ms)",
-                          p.policy_name.c_str(),
-                          _opt.meta_retry_delay_ms.count());
+                LOG_ERROR_F("update backup policy to remote storage failed, policy_name = {}, "
+                            "retry after {:#018x} (ms)",

Review Comment:
   done



##########
src/meta/meta_backup_service.cpp:
##########
@@ -1329,9 +1324,8 @@ void backup_service::do_add_policy(dsn::message_ex *req,
                 }
                 p->start();
             } else if (err == ERR_TIMEOUT) {
-                LOG_ERROR("create backup policy on remote storage timeout, retry after %" PRId64
-                          "(ms)",
-                          _opt.meta_retry_delay_ms.count());
+                LOG_ERROR_F("create backup policy on remote storage timeout, retry after {} (ms)",

Review Comment:
   done



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

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 #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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


##########
src/meta/meta_backup_service.cpp:
##########
@@ -1363,10 +1357,10 @@ void backup_service::do_update_policy_to_remote_storage(
                            p.policy_name);
                 p_context_ptr->set_policy(p);
             } else if (err == ERR_TIMEOUT) {
-                LOG_ERROR("update backup policy to remote storage failed, policy_name = %s, retry "
-                          "after %" PRId64 "(ms)",
-                          p.policy_name.c_str(),
-                          _opt.meta_retry_delay_ms.count());
+                LOG_ERROR_F("update backup policy to remote storage failed, policy_name = {}, "
+                            "retry after {:#018x} ms",

Review Comment:
   ```suggestion
                               "retry after {} ms",
   ```



-- 
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 merged pull request #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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


-- 
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 #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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


##########
src/runtime/rpc/network.cpp:
##########
@@ -418,8 +418,8 @@ bool rpc_session::on_recv_message(message_ex *msg, int delay_ms)
         // - the remote address is not listened, which means the remote port is not occupied
         // - operating system chooses the remote port as client's ephemeral port
         if (is_client() && msg->header->from_address == _net.engine()->primary_address()) {
-            LOG_ERROR("self connection detected, address = %s",
-                      msg->header->from_address.to_string());
+            LOG_ERROR_F("self connection detected, address = {}",
+                        msg->header->from_address.to_string());

Review Comment:
   ```suggestion
                           msg->header->from_address);
   ```



##########
src/utils/filesystem.cpp:
##########
@@ -747,10 +746,10 @@ error_code md5sum(const std::string &file_path, /*out*/ std::string &result)
                 break;
             } else {
                 int err = ferror(fp);
-                LOG_ERROR("md5sum error: read file %s failed: errno = %d (%s)",
-                          file_path.c_str(),
-                          err,
-                          safe_strerror(err).c_str());
+                LOG_ERROR_F("md5sum error: read file {} failed: errno = %d ({})",

Review Comment:
   ```suggestion
                   LOG_ERROR_F("md5sum error: read file {} failed: errno = {} ({})",
   ```



##########
src/meta/meta_data.cpp:
##########
@@ -424,22 +424,21 @@ bool config_context::check_order()
         return true;
     for (unsigned int i = 0; i < dropped.size() - 1; ++i) {
         if (dropped_cmp(dropped[i], dropped[i + 1]) > 0) {
-            LOG_ERROR("check dropped order for gpid(%d.%d) failed, [%s,%llu,%lld,%lld,%lld@%d] vs "
-                      "[%s,%llu,%lld,%lld,%lld@%d]",
-                      config_owner->pid.get_app_id(),
-                      config_owner->pid.get_partition_index(),
-                      dropped[i].node.to_string(),
-                      dropped[i].time,
-                      dropped[i].ballot,
-                      dropped[i].last_committed_decree,
-                      dropped[i].last_prepared_decree,
-                      i,
-                      dropped[i].node.to_string(),
-                      dropped[i].time,
-                      dropped[i].ballot,
-                      dropped[i].last_committed_decree,
-                      dropped[i].last_prepared_decree,
-                      i + 1);
+            LOG_ERROR_F("check dropped order for gpid({}) failed, [{},{},{},{},{}@{}] vs "
+                        "[{},{},{},{},{}@{}]",
+                        config_owner->pid,
+                        dropped[i].node,
+                        dropped[i].time,
+                        dropped[i].ballot,
+                        dropped[i].last_committed_decree,
+                        dropped[i].last_prepared_decree,
+                        i,
+                        dropped[i].node,
+                        dropped[i].time,
+                        dropped[i].ballot,
+                        dropped[i].last_committed_decree,
+                        dropped[i].last_prepared_decree,

Review Comment:
   ```suggestion
                           dropped[i + 1].node,
                           dropped[i + 1].time,
                           dropped[i + 1].ballot,
                           dropped[i + 1].last_committed_decree,
                           dropped[i + 1].last_prepared_decree,
   ```



##########
src/nfs/nfs_server_impl.cpp:
##########
@@ -33,6 +33,7 @@
 
 #include "aio/disk_engine.h"
 #include "runtime/task/async_calls.h"
+#include "utils/safe_strerror_posix.h"
 #include "utils/filesystem.h"
 #include "utils/string_conv.h"

Review Comment:
   ```suggestion
   #include "utils/filesystem.h"
   #include "utils/safe_strerror_posix.h"
   #include "utils/string_conv.h"
   ```



-- 
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 #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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


##########
src/client/partition_resolver_simple.cpp:
##########
@@ -311,28 +311,18 @@ void partition_resolver_simple::query_config_reply(error_code err,
                 }
             }
         } else if (resp.err == ERR_OBJECT_NOT_FOUND) {
-            LOG_ERROR("%s.client: query config reply, gpid = %d.%d, err = %s",
-                      _app_name.c_str(),
-                      _app_id,
-                      partition_index,
-                      resp.err.to_string());
+            LOG_ERROR_PREFIX(
+                "query config reply, gpid = {}.{}, err = {}", _app_id, partition_index, resp.err);
 
             client_err = ERR_APP_NOT_EXIST;
         } else {
-            LOG_ERROR("%s.client: query config reply, gpid = %d.%d, err = %s",
-                      _app_name.c_str(),
-                      _app_id,
-                      partition_index,
-                      resp.err.to_string());
+            LOG_ERROR_PREFIX(
+                "query config reply, gpid = {}.{}, err = {}", _app_id, partition_index, resp.err);
 
             client_err = resp.err;
         }
     } else {
-        LOG_ERROR("%s.client: query config reply, gpid = %d.%d, err = %s",
-                  _app_name.c_str(),
-                  _app_id,
-                  partition_index,
-                  err.to_string());
+        LOG_ERROR_F("query config reply, gpid = {}.{}, err = {}", _app_id, partition_index, err);

Review Comment:
   done



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

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 #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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


##########
src/client/partition_resolver_simple.cpp:
##########
@@ -311,28 +311,28 @@ void partition_resolver_simple::query_config_reply(error_code err,
                 }
             }
         } else if (resp.err == ERR_OBJECT_NOT_FOUND) {
-            LOG_ERROR("%s.client: query config reply, gpid = %d.%d, err = %s",
-                      _app_name.c_str(),
-                      _app_id,
-                      partition_index,
-                      resp.err.to_string());
+            LOG_ERROR_F("{}.client: query config reply, gpid = {}.{}, err = {}",

Review Comment:
   done



##########
src/failure_detector/failure_detector.cpp:
##########
@@ -227,13 +227,13 @@ void failure_detector::check_all_records()
                 is_time_greater_than(now, record.last_send_time_for_beacon_with_ack) &&
                 now + _check_interval_milliseconds - record.last_send_time_for_beacon_with_ack >
                     _lease_milliseconds) {
-                LOG_ERROR("master %s disconnected, now=%" PRId64 ", last_send_time=%" PRId64
-                          ", now+check_interval-last_send_time=%" PRId64,
-                          record.node.to_string(),
-                          now,
-                          record.last_send_time_for_beacon_with_ack,
-                          now + _check_interval_milliseconds -
-                              record.last_send_time_for_beacon_with_ack);
+                LOG_ERROR_F("master {} disconnected, now={:#018x}, last_send_time={:#018x}, "

Review Comment:
   fixed



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

To unsubscribe, e-mail: 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 #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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


##########
src/meta/meta_backup_service.cpp:
##########
@@ -1363,10 +1357,10 @@ void backup_service::do_update_policy_to_remote_storage(
                            p.policy_name);
                 p_context_ptr->set_policy(p);
             } else if (err == ERR_TIMEOUT) {
-                LOG_ERROR("update backup policy to remote storage failed, policy_name = %s, retry "
-                          "after %" PRId64 "(ms)",
-                          p.policy_name.c_str(),
-                          _opt.meta_retry_delay_ms.count());
+                LOG_ERROR_F("update backup policy to remote storage failed, policy_name = {}, "
+                            "retry after {:#018x} ms",

Review Comment:
   fixed



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

To unsubscribe, e-mail: 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 #1320: refactor(log): use LOG_ERROR_F instead of LOG_ERROR (2/2)

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


##########
src/meta/meta_backup_service.cpp:
##########
@@ -1363,10 +1357,10 @@ void backup_service::do_update_policy_to_remote_storage(
                            p.policy_name);
                 p_context_ptr->set_policy(p);
             } else if (err == ERR_TIMEOUT) {
-                LOG_ERROR("update backup policy to remote storage failed, policy_name = %s, retry "
-                          "after %" PRId64 "(ms)",
-                          p.policy_name.c_str(),
-                          _opt.meta_retry_delay_ms.count());
+                LOG_ERROR_F("update backup policy to remote storage failed, policy_name = {}, "
+                            "retry after {:#018x} (ms)",

Review Comment:
   ```suggestion
                               "retry after {} ms",
   ```



##########
src/meta/meta_backup_service.cpp:
##########
@@ -1329,9 +1324,8 @@ void backup_service::do_add_policy(dsn::message_ex *req,
                 }
                 p->start();
             } else if (err == ERR_TIMEOUT) {
-                LOG_ERROR("create backup policy on remote storage timeout, retry after %" PRId64
-                          "(ms)",
-                          _opt.meta_retry_delay_ms.count());
+                LOG_ERROR_F("create backup policy on remote storage timeout, retry after {} (ms)",

Review Comment:
   ```suggestion
                   LOG_ERROR_F("create backup policy on remote storage timeout, retry after {} ms",
   ```



##########
src/server/pegasus_server_impl.cpp:
##########
@@ -365,11 +359,9 @@ void pegasus_server_impl::on_multi_get(multi_get_rpc rpc)
     }
 
     if (!is_filter_type_supported(request.sort_key_filter_type)) {
-        LOG_ERROR("%s: invalid argument for multi_get from %s: "
-                  "sort key filter type %d not supported",
-                  replica_name(),
-                  rpc.remote_address().to_string(),
-                  request.sort_key_filter_type);
+        LOG_ERROR_F("invalid argument for multi_get from {}: sort key filter type {} not supported",

Review Comment:
   LOG_ERROR_PREFIX



##########
src/client/partition_resolver_simple.cpp:
##########
@@ -311,28 +311,18 @@ void partition_resolver_simple::query_config_reply(error_code err,
                 }
             }
         } else if (resp.err == ERR_OBJECT_NOT_FOUND) {
-            LOG_ERROR("%s.client: query config reply, gpid = %d.%d, err = %s",
-                      _app_name.c_str(),
-                      _app_id,
-                      partition_index,
-                      resp.err.to_string());
+            LOG_ERROR_PREFIX(
+                "query config reply, gpid = {}.{}, err = {}", _app_id, partition_index, resp.err);
 
             client_err = ERR_APP_NOT_EXIST;
         } else {
-            LOG_ERROR("%s.client: query config reply, gpid = %d.%d, err = %s",
-                      _app_name.c_str(),
-                      _app_id,
-                      partition_index,
-                      resp.err.to_string());
+            LOG_ERROR_PREFIX(
+                "query config reply, gpid = {}.{}, err = {}", _app_id, partition_index, resp.err);
 
             client_err = resp.err;
         }
     } else {
-        LOG_ERROR("%s.client: query config reply, gpid = %d.%d, err = %s",
-                  _app_name.c_str(),
-                  _app_id,
-                  partition_index,
-                  err.to_string());
+        LOG_ERROR_F("query config reply, gpid = {}.{}, err = {}", _app_id, partition_index, err);

Review Comment:
   LOG_ERROR_PREFIX



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