You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2016/12/14 14:19:43 UTC

[1/2] kudu git commit: tablet_bootstrap: periodically update status during replay

Repository: kudu
Updated Branches:
  refs/heads/master 0259586af -> 2b0bb5116


tablet_bootstrap: periodically update status during replay

This makes the bootstrap code update its status once every 5 seconds
while replaying each segment. This is useful to watch during server
startup in particular.

Change-Id: If48e5bf8745be163baa8c512229708973747eb3a
Reviewed-on: http://gerrit.cloudera.org:8080/5472
Tested-by: Kudu Jenkins
Reviewed-by: Mike Percy <mp...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/a684b7aa
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/a684b7aa
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/a684b7aa

Branch: refs/heads/master
Commit: a684b7aa604eff8eaf4450198bae3e68cfbc0ff6
Parents: 0259586
Author: Todd Lipcon <to...@apache.org>
Authored: Mon Dec 12 09:57:49 2016 +0700
Committer: Mike Percy <mp...@apache.org>
Committed: Wed Dec 14 14:18:47 2016 +0000

----------------------------------------------------------------------
 src/kudu/consensus/log_util.h       |  5 +++++
 src/kudu/tablet/tablet_bootstrap.cc | 18 +++++++++++++++---
 2 files changed, 20 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a684b7aa/src/kudu/consensus/log_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log_util.h b/src/kudu/consensus/log_util.h
index 59b4826..e2b5a93 100644
--- a/src/kudu/consensus/log_util.h
+++ b/src/kudu/consensus/log_util.h
@@ -98,6 +98,11 @@ class LogEntryReader {
     return offset_;
   }
 
+  // Return the offset at which this reader will stop reading.
+  int64_t read_up_to_offset() const {
+    return read_up_to_;
+  }
+
  private:
   friend class ReadableLogSegment;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/a684b7aa/src/kudu/tablet/tablet_bootstrap.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_bootstrap.cc b/src/kudu/tablet/tablet_bootstrap.cc
index 80e163d..af7e494 100644
--- a/src/kudu/tablet/tablet_bootstrap.cc
+++ b/src/kudu/tablet/tablet_bootstrap.cc
@@ -37,6 +37,7 @@
 #include "kudu/fs/fs_manager.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/strings/human_readable.h"
 #include "kudu/gutil/strings/strcat.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
@@ -1041,7 +1042,10 @@ Status TabletBootstrap::PlaySegments(ConsensusBootstrapInfo* consensus_info) {
   // writing.
   RETURN_NOT_OK_PREPEND(OpenNewLog(), "Failed to open new log");
 
+  auto last_status_update = MonoTime::Now();
+  const auto kStatusUpdateInterval = MonoDelta::FromSeconds(5);
   int segment_count = 0;
+
   for (const scoped_refptr<ReadableLogSegment>& segment : segments) {
     log::LogEntryReader reader(segment.get());
 
@@ -1075,11 +1079,19 @@ Status TabletBootstrap::PlaySegments(ConsensusBootstrapInfo* consensus_info) {
 
       // If HandleEntry returns OK, then it has taken ownership of the entry.
       entry.release();
+
+      auto now = MonoTime::Now();
+      if (now - last_status_update > kStatusUpdateInterval) {
+        StatusMessage(Substitute("Bootstrap replaying log segment $0/$1 "
+                                 "($2/$3 this segment, stats: $4)",
+                                 segment_count + 1, log_reader_->num_segments(),
+                                 HumanReadableNumBytes::ToString(reader.offset()),
+                                 HumanReadableNumBytes::ToString(reader.read_up_to_offset()),
+                                 stats_.ToString()));
+        last_status_update = now;
+      }
     }
 
-    // TODO: could be more granular here and log during the segments as well,
-    // plus give info about number of MB processed, but this is better than
-    // nothing.
     StatusMessage(Substitute("Bootstrap replayed $0/$1 log segments. "
                              "Stats: $2. Pending: $3 replicates",
                              segment_count + 1, log_reader_->num_segments(),


[2/2] kudu git commit: Avoid flakiness in RaftConsensusITest.TestReplaceOperationStuckInPrepareQueue

Posted by mp...@apache.org.
Avoid flakiness in RaftConsensusITest.TestReplaceOperationStuckInPrepareQueue

Change-Id: I58027417a93c14db14ac9139f1bef530d552bba0
Reviewed-on: http://gerrit.cloudera.org:8080/5467
Tested-by: Kudu Jenkins
Reviewed-by: Mike Percy <mp...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/2b0bb511
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/2b0bb511
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/2b0bb511

Branch: refs/heads/master
Commit: 2b0bb51167666f9beb26a7fe442b3a744bf8cacf
Parents: a684b7a
Author: Todd Lipcon <to...@apache.org>
Authored: Sun Dec 11 23:01:55 2016 +0800
Committer: Mike Percy <mp...@apache.org>
Committed: Wed Dec 14 14:19:01 2016 +0000

----------------------------------------------------------------------
 src/kudu/integration-tests/raft_consensus-itest.cc | 14 ++++++++++----
 1 file changed, 10 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/2b0bb511/src/kudu/integration-tests/raft_consensus-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/raft_consensus-itest.cc b/src/kudu/integration-tests/raft_consensus-itest.cc
index 5c3f79f..4d284e4 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -1401,10 +1401,16 @@ TEST_F(RaftConsensusITest, TestReplaceOperationStuckInPrepareQueue) {
   ASSERT_FALSE(resp.has_error()) << resp.DebugString();
 
   // Ensure we can read the data.
-  vector<string> results;
-  NO_FATALS(WaitForRowCount(replica_ts->tserver_proxy.get(), 2, &results));
-  ASSERT_EQ("(int32 key=1, int32 int_val=3, string string_val=\"term: 3 index: 4\")", results[0]);
-  ASSERT_EQ("(int32 key=2, int32 int_val=3, string string_val=\"term: 3 index: 5\")", results[1]);
+  // We need to AssertEventually here because otherwise it's possible to read the old value
+  // of row '1', if the operation is still in flight.
+  AssertEventually([&]() {
+      vector<string> results;
+      NO_FATALS(WaitForRowCount(replica_ts->tserver_proxy.get(), 2, &results));
+      ASSERT_EQ("(int32 key=1, int32 int_val=3, string string_val=\"term: 3 index: 4\")",
+                results[0]);
+      ASSERT_EQ("(int32 key=2, int32 int_val=3, string string_val=\"term: 3 index: 5\")",
+                results[1]);
+    });
 }
 
 // Regression test for KUDU-644: