You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@kvrocks.apache.org by "chrisxu333 (via GitHub)" <gi...@apache.org> on 2023/12/18 15:01:05 UTC

[PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

chrisxu333 opened a new pull request, #1943:
URL: https://github.com/apache/kvrocks/pull/1943

   To expose essential perf_context and iostats_context from rocksdb, a new command ANALYZE is introduced.
   
   A typical usage would be as follows:
   ![perf](https://github.com/apache/kvrocks/assets/44099579/44ef020b-90e0-48f5-8d0c-e7d8e46d2582)
   


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1436061484


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,65 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      command_args_.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) override {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(command_args_[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(command_args_);
+
+    auto s = cmd->Parse(command_args_);

Review Comment:
   Should we check the arity before `Parse`? Since wrong `command_args_` might make our program crash



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1868908519

   Would you mind fix the lint first?


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1865724713

   > do we still need to do this in this patch
   
   I agree it's not neccessary in this patch. Personally I think metrics can be widely used, but lets first get it in used...
   
   cc @PragmaTwice @git-hulk This command looks like SQL "PROFILE" with rocksdb io/perf stats enabled and output. It can be used in some cases, but for some blocking command I don't know whether it's ok. Would you mind take a look?


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1436295085


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,71 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      command_args_.push_back(args[i]);
+    }

Review Comment:
   nit: we can directly `insert` rather than push back



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1432200926


##########
src/server/redis_reply.cc:
##########
@@ -34,6 +34,29 @@ std::string BulkString(const std::string &data) { return "$" + std::to_string(da
 
 std::string NilString() { return "$-1" CRLF; }
 
+std::string ExtractData(const std::string &reply) {

Review Comment:
   Emmm, what about the multi bulk string like?



##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,60 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args);
+
+    auto s = cmd->Parse(true_args);
+
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+
+    std::string tmp_output;
+    s = cmd->Execute(srv, conn, &tmp_output);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    if (tmp_output[0] == '-') {
+      *output = tmp_output;
+      return s;
+    }
+
+    std::string perf_context = rocksdb::get_perf_context()->ToString(true);
+    std::string iostats_context = rocksdb::get_iostats_context()->ToString(true);
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kDisable);
+    if (perf_context.empty()) return Status::OK();  // request without db operation
+
+    tmp_output = redis::ExtractData(tmp_output);
+
+    *output = redis::MultiBulkString(
+        {tmp_output, "(perf_context): " + perf_context, "(iostats_context): " + iostats_context});

Review Comment:
   ```suggestion
           {std::move(tmp_output), "(perf_context): " + perf_context, "(iostats_context): " + iostats_context});
   ```



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1869908754

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [1 New issue](https://sonarcloud.io/project/issues?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   No data about Coverage  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_kvrocks&pullRequest=1943&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943)
   
   


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1868931109

    
   
   > Would you mind fix the lint first?
   
   Sounds good I just did.


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1868844965

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [4 New issues](https://sonarcloud.io/project/issues?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   No data about Coverage  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_kvrocks&pullRequest=1943&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943)
   
   


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk merged PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1873555628

   @chrisxu333 Thanks for your follow-up. This PR looks good to me, to see if @PragmaTwice has any comments, if not, we can move forward to merge.


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1865693330

   > > > Maybe it's better to first having a RAII guard like TiKV: https://github.com/tikv/tikv/blob/master/src/coprocessor/tracker.rs ( We can be easier than them) ?
   > > 
   > > 
   > > Can you elaborate more on this? I didn't quite get the point.
   > 
   > Hmm we can just leave it here in this patch. I believe something like a :
   > 
   > ```
   > class StoragePerfContext {
   >   
   >   void close();
   >   ~StoragePerfContext() {
   >       if (!reset_) {
   >          rocksdb::Perfcontext // resume
   >       }
   >    }
   > private:
   >   Storage* // or rocksdb?
   >   bool reseted_ = false;
   > };
   > ```
   > 
   > Might be better? If other command want to debugging, they can also uses this tool? Also we can considering RAII here.
   
   I see what you meant. So just to be clear, do we still need to do this in this patch? Or we wait until when it's actually needed to do so?


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1435941719


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,75 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args_.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args_[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args_);
+
+    auto s = cmd->Parse(true_args_);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    auto prev_perf_level = rocksdb::GetPerfLevel();
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+
+    std::string tmp_output;
+    s = cmd->Execute(srv, conn, &tmp_output);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    if (tmp_output[0] == '-') {
+      *output = tmp_output;
+      return s;
+    }
+
+    std::string perf_context = rocksdb::get_perf_context()->ToString(true);
+    std::string iostats_context = rocksdb::get_iostats_context()->ToString(true);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+    rocksdb::SetPerfLevel(prev_perf_level);
+    if (perf_context.empty()) {
+      *output = tmp_output;
+      return Status::OK();  // request without db operation
+    }
+
+    if (tmp_output.at(0) == '*') {
+      // if multi bulkstring, append directly
+      tmp_output.append(redis::BulkString(perf_context));
+      tmp_output.append(redis::BulkString(iostats_context));
+      *output = tmp_output;
+    } else {
+      tmp_output = redis::ExtractData(tmp_output);
+      *output = redis::MultiBulkString(
+          {std::move(tmp_output), "(perf_context): " + perf_context, "(iostats_context): " + iostats_context});
+    }

Review Comment:
   Can also remove function `ExtractData` and rename `tmp_output` to `command_output`



##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,75 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args_.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args_[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args_);
+
+    auto s = cmd->Parse(true_args_);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    auto prev_perf_level = rocksdb::GetPerfLevel();
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+
+    std::string tmp_output;
+    s = cmd->Execute(srv, conn, &tmp_output);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    if (tmp_output[0] == '-') {
+      *output = tmp_output;
+      return s;
+    }
+
+    std::string perf_context = rocksdb::get_perf_context()->ToString(true);
+    std::string iostats_context = rocksdb::get_iostats_context()->ToString(true);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+    rocksdb::SetPerfLevel(prev_perf_level);
+    if (perf_context.empty()) {
+      *output = tmp_output;
+      return Status::OK();  // request without db operation
+    }
+
+    if (tmp_output.at(0) == '*') {
+      // if multi bulkstring, append directly
+      tmp_output.append(redis::BulkString(perf_context));
+      tmp_output.append(redis::BulkString(iostats_context));
+      *output = tmp_output;
+    } else {
+      tmp_output = redis::ExtractData(tmp_output);
+      *output = redis::MultiBulkString(
+          {std::move(tmp_output), "(perf_context): " + perf_context, "(iostats_context): " + iostats_context});
+    }

Review Comment:
   ```suggestion
       *output = redis::MultiLen(3); // command output + perf context + iostats context
       *output += std::move(tmp_output);
       *output += redis::BulkString(perf_context);
       *output += redis::BulkString(iostats_context);
   ```
   
   This implementation isn't correct. For the multi-bulk strings, the first element is the number of array elements, so you can't simply append the element.



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1435978400


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,75 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args_.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args_[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args_);
+
+    auto s = cmd->Parse(true_args_);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    auto prev_perf_level = rocksdb::GetPerfLevel();
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+
+    std::string tmp_output;
+    s = cmd->Execute(srv, conn, &tmp_output);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    if (tmp_output[0] == '-') {
+      *output = tmp_output;
+      return s;
+    }
+
+    std::string perf_context = rocksdb::get_perf_context()->ToString(true);
+    std::string iostats_context = rocksdb::get_iostats_context()->ToString(true);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+    rocksdb::SetPerfLevel(prev_perf_level);
+    if (perf_context.empty()) {
+      *output = tmp_output;
+      return Status::OK();  // request without db operation
+    }
+
+    if (tmp_output.at(0) == '*') {
+      // if multi bulkstring, append directly
+      tmp_output.append(redis::BulkString(perf_context));
+      tmp_output.append(redis::BulkString(iostats_context));
+      *output = tmp_output;
+    } else {
+      tmp_output = redis::ExtractData(tmp_output);
+      *output = redis::MultiBulkString(
+          {std::move(tmp_output), "(perf_context): " + perf_context, "(iostats_context): " + iostats_context});
+    }

Review Comment:
   Sounds good I've changed them accordingly :)



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1436076227


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,65 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      command_args_.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) override {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(command_args_[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(command_args_);
+
+    auto s = cmd->Parse(command_args_);

Review Comment:
   We need to reuse the code in ExecuteCommand rather than write them again here.



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1860739299

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [3 New issues](https://sonarcloud.io/project/issues?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   No data about Coverage  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_kvrocks&pullRequest=1943&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943)
   
   


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1436279808


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,65 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      command_args_.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) override {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(command_args_[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(command_args_);
+
+    auto s = cmd->Parse(command_args_);

Review Comment:
   I just pushed a newer version which added the arity checking logic. As for reusing / refactoring the code in `ExecuteCommand`, maybe it's better to do it in a separate pr? As the `ExecuteCommand` code needs to be dissected into several chunks for reusing purpose. @mapleFU @PragmaTwice 



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1435971235


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,75 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args_.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args_[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args_);
+
+    auto s = cmd->Parse(true_args_);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    auto prev_perf_level = rocksdb::GetPerfLevel();
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+
+    std::string tmp_output;
+    s = cmd->Execute(srv, conn, &tmp_output);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    if (tmp_output[0] == '-') {
+      *output = tmp_output;
+      return s;
+    }
+
+    std::string perf_context = rocksdb::get_perf_context()->ToString(true);
+    std::string iostats_context = rocksdb::get_iostats_context()->ToString(true);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+    rocksdb::SetPerfLevel(prev_perf_level);
+    if (perf_context.empty()) {
+      *output = tmp_output;
+      return Status::OK();  // request without db operation
+    }
+
+    if (tmp_output.at(0) == '*') {
+      // if multi bulkstring, append directly
+      tmp_output.append(redis::BulkString(perf_context));
+      tmp_output.append(redis::BulkString(iostats_context));
+      *output = tmp_output;
+    } else {
+      tmp_output = redis::ExtractData(tmp_output);
+      *output = redis::MultiBulkString(
+          {std::move(tmp_output), "(perf_context): " + perf_context, "(iostats_context): " + iostats_context});
+    }

Review Comment:
   What do you mean by saying remove `ExtractData` function?



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1868918884

   @chrisxu333  `./x.py format` should do the trick.


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1868831085

   @chrisxu333 Can rename `true_args_` to `command_args_`, rest are good to me. Sorry for missing this point.


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1430296589


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,60 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args);
+
+    auto s = cmd->Parse(true_args);

Review Comment:
   Should we check the status here?



##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,60 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args);
+
+    auto s = cmd->Parse(true_args);
+
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+
+    std::string tmp_output;
+    s = cmd->Execute(srv, conn, &tmp_output);

Review Comment:
   If we have hundreds/thousands of operations per second, how do we know that the perf context and its data are related for this particular command?



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1862181099

   > Maybe it's better to first having a RAII guard like TiKV: https://github.com/tikv/tikv/blob/master/src/coprocessor/tracker.rs ( We can be easier than them) ?
   
   Can you elaborate more on this? I didn't quite get the point.


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1433636642


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,60 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args);
+
+    auto s = cmd->Parse(true_args);
+
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+
+    std::string tmp_output;
+    s = cmd->Execute(srv, conn, &tmp_output);

Review Comment:
   This might related to context, it uses ThreadLocal storage, and metric in the thread local context



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1868794112

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [4 New issues](https://sonarcloud.io/project/issues?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   No data about Coverage  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_kvrocks&pullRequest=1943&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943)
   
   


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1436279808


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,65 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) override {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      command_args_.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) override {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(command_args_[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(command_args_);
+
+    auto s = cmd->Parse(command_args_);

Review Comment:
   I just pushed a newer version which added the arity checking logic. As for reusing / refactoring the code in `ExecuteCommand`, maybe it's better to do it in a separate pr? As the `ExecuteCommand` code needs to be dissected into several chunks for reusing purpose.



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1430918692


##########
src/commands/cmd_server.cc:
##########
@@ -1157,6 +1214,7 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandAuth>("auth", 2, "read-only ok-loadin
                         MakeCmdAttr<CommandFlushBackup>("flushbackup", 1, "read-only no-script", 0, 0, 0),
                         MakeCmdAttr<CommandSlaveOf>("slaveof", 3, "read-only exclusive no-script", 0, 0, 0),
                         MakeCmdAttr<CommandStats>("stats", 1, "read-only", 0, 0, 0),
-                        MakeCmdAttr<CommandRdb>("rdb", -3, "write exclusive", 0, 0, 0), )
+                        MakeCmdAttr<CommandRdb>("rdb", -3, "write exclusive", 0, 0, 0),
+                        MakeCmdAttr<CommandAnalyze>("analyze", -1, "", 0, 0, 0), )

Review Comment:
   Sorry maybe I made it wrong, since PerfContext is a tls in implemention, maybe current implemention doesn't need to considering exclusive



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1430888093


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,60 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args);
+
+    auto s = cmd->Parse(true_args);
+
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+
+    std::string tmp_output;
+    s = cmd->Execute(srv, conn, &tmp_output);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    if (tmp_output[0] == '-') {
+      *output = tmp_output;
+      return s;
+    }
+
+    std::string perf_context = rocksdb::get_perf_context()->ToString(true);
+    std::string iostats_context = rocksdb::get_iostats_context()->ToString(true);
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kDisable);

Review Comment:
   That's a good point I'll fix it accordingly :)



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1869306993

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [1 New issue](https://sonarcloud.io/project/issues?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   No data about Coverage  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_kvrocks&pullRequest=1943&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943)
   
   


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1869908845

   > @chrisxu333 `./x.py format` should do the trick.
   
   Hi @torwig, could you help review this pr whenever you're able to? Thx!


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1435971235


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,75 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args_.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args_[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args_);
+
+    auto s = cmd->Parse(true_args_);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    auto prev_perf_level = rocksdb::GetPerfLevel();
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+
+    std::string tmp_output;
+    s = cmd->Execute(srv, conn, &tmp_output);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    if (tmp_output[0] == '-') {
+      *output = tmp_output;
+      return s;
+    }
+
+    std::string perf_context = rocksdb::get_perf_context()->ToString(true);
+    std::string iostats_context = rocksdb::get_iostats_context()->ToString(true);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+    rocksdb::SetPerfLevel(prev_perf_level);
+    if (perf_context.empty()) {
+      *output = tmp_output;
+      return Status::OK();  // request without db operation
+    }
+
+    if (tmp_output.at(0) == '*') {
+      // if multi bulkstring, append directly
+      tmp_output.append(redis::BulkString(perf_context));
+      tmp_output.append(redis::BulkString(iostats_context));
+      *output = tmp_output;
+    } else {
+      tmp_output = redis::ExtractData(tmp_output);
+      *output = redis::MultiBulkString(
+          {std::move(tmp_output), "(perf_context): " + perf_context, "(iostats_context): " + iostats_context});
+    }

Review Comment:
   What do you mean by saying remove `ExtractData` function?



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1868895131

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [4 New issues](https://sonarcloud.io/project/issues?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   No data about Coverage  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_kvrocks&pullRequest=1943&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943)
   
   


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1862097186

   Maybe it's better to first having a RAII guard like TiKV: https://github.com/tikv/tikv/blob/master/src/coprocessor/tracker.rs ( We can be easier than them) ?


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1863792815

   > > Maybe it's better to first having a RAII guard like TiKV: https://github.com/tikv/tikv/blob/master/src/coprocessor/tracker.rs ( We can be easier than them) ?
   > 
   > Can you elaborate more on this? I didn't quite get the point.
   
   Hmm we can just leave it here in this patch. I believe something like a :
   
   ```
   class StoragePerfContext {
     
     void close();
     ~StoragePerfContext() {
         if (!reset_) {
            rocksdb::Perfcontext // resume
         }
      }
   private:
     Storage* // or rocksdb?
     bool reseted_ = false;
   };
   ```
   
   Might be better? If other command want to debugging, they can also uses this tool? Also we can considering RAII here.


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#discussion_r1430292799


##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,60 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args);
+
+    auto s = cmd->Parse(true_args);
+
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+
+    std::string tmp_output;
+    s = cmd->Execute(srv, conn, &tmp_output);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    if (tmp_output[0] == '-') {
+      *output = tmp_output;
+      return s;
+    }
+
+    std::string perf_context = rocksdb::get_perf_context()->ToString(true);
+    std::string iostats_context = rocksdb::get_iostats_context()->ToString(true);
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kDisable);
+    if (perf_context.empty()) return Status::OK();  // request without db operation
+
+    tmp_output = redis::ExtractData(tmp_output);
+
+    *output = redis::MultiBulkString(
+        {tmp_output, "(perf_context): " + perf_context, "(iostats_context): " + iostats_context});
+    return Status::OK();
+  }
+
+ private:
+  std::vector<std::string> true_args;

Review Comment:
   ```suggestion
     std::vector<std::string> true_args_;
   ```



##########
src/commands/cmd_server.cc:
##########
@@ -1157,6 +1214,7 @@ REDIS_REGISTER_COMMANDS(MakeCmdAttr<CommandAuth>("auth", 2, "read-only ok-loadin
                         MakeCmdAttr<CommandFlushBackup>("flushbackup", 1, "read-only no-script", 0, 0, 0),
                         MakeCmdAttr<CommandSlaveOf>("slaveof", 3, "read-only exclusive no-script", 0, 0, 0),
                         MakeCmdAttr<CommandStats>("stats", 1, "read-only", 0, 0, 0),
-                        MakeCmdAttr<CommandRdb>("rdb", -3, "write exclusive", 0, 0, 0), )
+                        MakeCmdAttr<CommandRdb>("rdb", -3, "write exclusive", 0, 0, 0),
+                        MakeCmdAttr<CommandAnalyze>("analyze", -1, "", 0, 0, 0), )

Review Comment:
   at least, it should be an exclusive call?



##########
src/commands/cmd_server.cc:
##########
@@ -1123,6 +1126,60 @@ class CommandRdb : public Commander {
   uint32_t db_index_ = 0;
 };
 
+class CommandAnalyze : public Commander {
+ public:
+  Status Parse(const std::vector<std::string> &args) {
+    if (args.size() <= 1) return {Status::RedisExecErr, errInvalidSyntax};
+    for (int i = 1; i < args.size(); ++i) {
+      true_args.push_back(args[i]);
+    }
+    return Status::OK();
+  }
+  Status Execute(Server *srv, Connection *conn, std::string *output) {
+    auto commands = redis::CommandTable::Get();
+    auto cmd_iter = commands->find(util::ToLower(true_args[0]));
+    if (cmd_iter == commands->end()) {
+      // unsupported redis command
+      return {Status::RedisExecErr, errInvalidSyntax};
+    }
+    auto redis_cmd = cmd_iter->second;
+    auto cmd = redis_cmd->factory();
+    cmd->SetAttributes(redis_cmd);
+    cmd->SetArgs(true_args);
+
+    auto s = cmd->Parse(true_args);
+
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);
+    rocksdb::get_perf_context()->Reset();
+    rocksdb::get_iostats_context()->Reset();
+
+    std::string tmp_output;
+    s = cmd->Execute(srv, conn, &tmp_output);
+    if (!s.IsOK()) {
+      return s;
+    }
+
+    if (tmp_output[0] == '-') {
+      *output = tmp_output;
+      return s;
+    }
+
+    std::string perf_context = rocksdb::get_perf_context()->ToString(true);
+    std::string iostats_context = rocksdb::get_iostats_context()->ToString(true);
+    rocksdb::SetPerfLevel(rocksdb::PerfLevel::kDisable);

Review Comment:
   at least we should record pervious perf level, and resume it after this function( even if the status is error after execution )



-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1865693587

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [3 New issues](https://sonarcloud.io/project/issues?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   No data about Coverage  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_kvrocks&pullRequest=1943&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943)
   
   


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1868701243

   > > do we still need to do this in this patch
   > 
   > I agree it's not neccessary in this patch. Personally I think metrics can be widely used, but lets first get it in used...
   > 
   > cc @PragmaTwice @git-hulk This command looks like SQL "PROFILE" with rocksdb io/perf stats enabled and output. It can be used in some cases, but for some blocking command I don't know whether it's ok. Would you mind take a look?
   
   @git-hulk @PragmaTwice Hello guys, if you have time could you help review this pr? Thank you :)


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] expose rocksdb perf/iostat context with ANALYZE command [kvrocks]

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1868710819

   Sure, sorry for missing this message. I will take a look recently.


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1868931789

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [1 New issue](https://sonarcloud.io/project/issues?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   No data about Coverage  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_kvrocks&pullRequest=1943&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943)
   
   


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "sonarcloud[bot] (via GitHub)" <gi...@apache.org>.
sonarcloud[bot] commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1873660288

   ## [![Quality Gate Passed](https://sonarsource.github.io/sonarcloud-github-static-resources/v2/checks/QualityGateBadge/qg-passed-20px.png 'Quality Gate Passed')](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943) **Quality Gate passed**  
   The SonarCloud Quality Gate passed, but some issues were introduced.
   
   [1 New issue](https://sonarcloud.io/project/issues?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   [0 Security Hotspots](https://sonarcloud.io/project/security_hotspots?id=apache_kvrocks&pullRequest=1943&resolved=false&inNewCodePeriod=true)  
   No data about Coverage  
   [0.0% Duplication on New Code](https://sonarcloud.io/component_measures?id=apache_kvrocks&pullRequest=1943&metric=new_duplicated_lines_density&view=list)  
     
   [See analysis details on SonarCloud](https://sonarcloud.io/dashboard?id=apache_kvrocks&pullRequest=1943)
   
   


-- 
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: issues-unsubscribe@kvrocks.apache.org

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


Re: [PR] Add the support of ANALYZE command to inspect the performance of RocksDB [kvrocks]

Posted by "chrisxu333 (via GitHub)" <gi...@apache.org>.
chrisxu333 commented on PR #1943:
URL: https://github.com/apache/kvrocks/pull/1943#issuecomment-1873554183

   Hi @git-hulk @PragmaTwice,
   
   Could you help review and approve this pr if possible? Thx!


-- 
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: issues-unsubscribe@kvrocks.apache.org

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