You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2022/03/14 22:45:41 UTC

[kudu] branch master updated: [tablet] add tablet_id into trace of a slow write op

This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
     new 2a5e565  [tablet] add tablet_id into trace of a slow write op
2a5e565 is described below

commit 2a5e56594a9dac28df9b02365d340bbd6a8e480f
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Fri Mar 11 14:16:27 2022 -0800

    [tablet] add tablet_id into trace of a slow write op
    
    While troubleshooting one performance issue, I found that the traces
    on long running write operations didn't contain information on their
    tablet identifier.
    
    This patch address that, so now it's easier to tell what's the tablet
    where write operations run longer.  I also took the liberty to update
    the code around, making the style of the trace messages more uniform.
    
    Change-Id: I06b05e8be1f5376b60244228809be9d260ed8398
    Reviewed-on: http://gerrit.cloudera.org:8080/18315
    Tested-by: Kudu Jenkins
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
---
 src/kudu/tablet/ops/write_op.cc | 37 +++++++++++++++++++------------------
 src/kudu/tablet/ops/write_op.h  |  2 +-
 2 files changed, 20 insertions(+), 19 deletions(-)

diff --git a/src/kudu/tablet/ops/write_op.cc b/src/kudu/tablet/ops/write_op.cc
index 7f1defb..b5ae129 100644
--- a/src/kudu/tablet/ops/write_op.cc
+++ b/src/kudu/tablet/ops/write_op.cc
@@ -159,7 +159,9 @@ void WriteOp::NewReplicateMsg(unique_ptr<ReplicateMsg>* replicate_msg) {
 
 Status WriteOp::Prepare() {
   TRACE_EVENT0("op", "WriteOp::Prepare");
-  TRACE("PREPARE: Starting.");
+
+  Tablet* tablet = state()->tablet_replica()->tablet();
+  TRACE(Substitute("PREPARE: Starting on tablet $0", tablet->tablet_id()));
   // Decode everything first so that we give up if something major is wrong.
   Schema client_schema;
   RETURN_NOT_OK_PREPEND(SchemaFromPB(state_->request()->schema(), &client_schema),
@@ -172,8 +174,6 @@ Status WriteOp::Prepare() {
     return s;
   }
 
-  Tablet* tablet = state()->tablet_replica()->tablet();
-
   // Before taking any other locks, acquire the transaction state lock and
   // ensure it is open.
   Status s;
@@ -224,7 +224,7 @@ Status WriteOp::Prepare() {
   }
   RETURN_NOT_OK(tablet->AcquireRowLocks(state()));
 
-  TRACE("PREPARE: Finished.");
+  TRACE("PREPARE: Finished");
   return Status::OK();
 }
 
@@ -264,7 +264,7 @@ void WriteOp::UpdatePerRowMetricsAndErrors() {
 // it seems pointless to return a Status!
 Status WriteOp::Apply(CommitMsg** commit_msg) {
   TRACE_EVENT0("op", "WriteOp::Apply");
-  TRACE("APPLY: Starting.");
+  TRACE("APPLY: Starting");
 
   if (PREDICT_FALSE(ANNOTATE_UNPROTECTED_READ(
       FLAGS_tablet_inject_latency_on_apply_write_op_ms) > 0)) {
@@ -275,7 +275,7 @@ Status WriteOp::Apply(CommitMsg** commit_msg) {
 
   Tablet* tablet = state()->tablet_replica()->tablet();
   RETURN_NOT_OK(tablet->ApplyRowOperations(state()));
-  TRACE("APPLY: Finished.");
+  TRACE("APPLY: Finished");
 
   UpdatePerRowMetricsAndErrors();
 
@@ -293,29 +293,30 @@ void WriteOp::Finish(OpResult result) {
   state()->FinishApplyingOrAbort(result);
 
   if (PREDICT_FALSE(result == Op::ABORTED)) {
-    TRACE("FINISH: Op aborted.");
+    TRACE("FINISH: Op aborted");
     return;
   }
 
   DCHECK_EQ(result, Op::APPLIED);
 
-  TRACE("FINISH: Updating metrics.");
+  TRACE("FINISH: Updating metrics");
 
-  TabletMetrics* metrics = state_->tablet_replica()->tablet()->metrics();
-  if (metrics) {
+  if (auto* metrics = state_->tablet_replica()->tablet()->metrics();
+      PREDICT_TRUE(metrics != nullptr)) {
     // TODO(unknown): should we change this so it's actually incremented by the
     // Tablet code itself instead of this wrapper code?
-    metrics->rows_inserted->IncrementBy(state_->metrics().successful_inserts);
-    metrics->insert_ignore_errors->IncrementBy(state_->metrics().insert_ignore_errors);
-    metrics->rows_upserted->IncrementBy(state_->metrics().successful_upserts);
-    metrics->rows_updated->IncrementBy(state_->metrics().successful_updates);
-    metrics->update_ignore_errors->IncrementBy(state_->metrics().update_ignore_errors);
-    metrics->rows_deleted->IncrementBy(state_->metrics().successful_deletes);
-    metrics->delete_ignore_errors->IncrementBy(state_->metrics().delete_ignore_errors);
+    const auto& op_m = state_->metrics();
+    metrics->rows_inserted->IncrementBy(op_m.successful_inserts);
+    metrics->insert_ignore_errors->IncrementBy(op_m.insert_ignore_errors);
+    metrics->rows_upserted->IncrementBy(op_m.successful_upserts);
+    metrics->rows_updated->IncrementBy(op_m.successful_updates);
+    metrics->update_ignore_errors->IncrementBy(op_m.update_ignore_errors);
+    metrics->rows_deleted->IncrementBy(op_m.successful_deletes);
+    metrics->delete_ignore_errors->IncrementBy(op_m.delete_ignore_errors);
 
     if (type() == consensus::LEADER) {
       if (state()->external_consistency_mode() == COMMIT_WAIT) {
-        metrics->commit_wait_duration->Increment(state_->metrics().commit_wait_duration_usec);
+        metrics->commit_wait_duration->Increment(op_m.commit_wait_duration_usec);
       }
       uint64_t op_duration_usec =
           (MonoTime::Now() - start_time_).ToMicroseconds();
diff --git a/src/kudu/tablet/ops/write_op.h b/src/kudu/tablet/ops/write_op.h
index 4751d39..ea61887 100644
--- a/src/kudu/tablet/ops/write_op.h
+++ b/src/kudu/tablet/ops/write_op.h
@@ -30,6 +30,7 @@
 
 #include "kudu/common/row_operations.h"
 #include "kudu/common/row_operations.pb.h"
+#include "kudu/common/schema.h"
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/ref_counted.h"
@@ -44,7 +45,6 @@
 
 namespace kudu {
 
-class Schema;
 class rw_semaphore;
 
 namespace rpc {