You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2018/01/11 22:16:23 UTC

[1/3] kudu git commit: [webui] Make tombstone tablet info less confusing

Repository: kudu
Updated Branches:
  refs/heads/master f2adee004 -> 56107ac80


[webui] Make tombstone tablet info less confusing

Previously, when a tombstone tablet was reloaded at server startup,
the last status message was "Tablet initializing...". This was
confusing as it set the expectation that something more was going
to happen to the tombstoned tablet. The message is now simply
"Tombstoned".

Also, now that tombstoned tablets can vote, they retain cmeta
despite not participating in non-election Raft operations.
Their list of peers is not updated and not usually relevant. It
might be confusing to see it on the web ui. This patch suppresses
it.

Change-Id: I5c879cc7ff634e5b434fc33374d3010cf1f262cb
Reviewed-on: http://gerrit.cloudera.org:8080/8981
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 1cf30effc13827ec4f2e739e7ca3afdbc0051437
Parents: f2adee0
Author: Will Berkeley <wd...@apache.org>
Authored: Tue Jan 9 11:23:09 2018 -0800
Committer: Will Berkeley <wd...@gmail.com>
Committed: Thu Jan 11 02:22:40 2018 +0000

----------------------------------------------------------------------
 src/kudu/tablet/tablet_replica.h          |  5 ++++
 src/kudu/tserver/tablet_server-test.cc    | 33 ++++++++++++++++++++++++++
 src/kudu/tserver/ts_tablet_manager.cc     |  1 +
 src/kudu/tserver/tserver_path_handlers.cc | 23 +++++++++++++-----
 4 files changed, 56 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/1cf30eff/src/kudu/tablet/tablet_replica.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_replica.h b/src/kudu/tablet/tablet_replica.h
index bf3476d..253e9c0 100644
--- a/src/kudu/tablet/tablet_replica.h
+++ b/src/kudu/tablet/tablet_replica.h
@@ -187,6 +187,11 @@ class TabletReplica : public RefCountedThreadSafe<TabletReplica>,
 
   std::string StateName() const;
 
+  const TabletDataState data_state() const {
+    std::lock_guard<simple_spinlock> lock(lock_);
+    return meta_->tablet_data_state();
+  }
+
   // Returns the current Raft configuration.
   const consensus::RaftConfigPB RaftConfig() const;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/1cf30eff/src/kudu/tserver/tablet_server-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server-test.cc b/src/kudu/tserver/tablet_server-test.cc
index de498dc..411b982 100644
--- a/src/kudu/tserver/tablet_server-test.cc
+++ b/src/kudu/tserver/tablet_server-test.cc
@@ -385,6 +385,39 @@ TEST_F(TabletServerTest, TestFailedTabletsOnWebUI) {
   ASSERT_STR_CONTAINS(buf.ToString(), "Tombstoned Tablets");
 }
 
+// Test that tombstoned tablets are displayed correctly in the web ui:
+// - After restart, status message of "Tombstoned" instead of "Tablet initializing...".
+// - No consensus configuration.
+TEST_F(TabletServerTest, TestTombstonedTabletOnWebUI) {
+  TSTabletManager* tablet_manager = mini_server_->server()->tablet_manager();
+  TabletServerErrorPB::Code error_code;
+  ASSERT_OK(tablet_manager->DeleteTablet(kTabletId,
+                                         tablet::TABLET_DATA_TOMBSTONED, boost::none, &error_code));
+
+  // Restart the server. We drop the tablet_replica_ reference since it becomes
+  // invalid when the server shuts down.
+  tablet_replica_.reset();
+  mini_server_->Shutdown();
+  ASSERT_OK(mini_server_->Restart());
+  ASSERT_OK(mini_server_->WaitStarted());
+
+  EasyCurl c;
+  faststring buf;
+  const string addr = mini_server_->bound_http_addr().ToString();
+  ASSERT_OK(c.FetchURL(Substitute("http://$0/tablets", addr), &buf));
+
+  // Ensure the html contains the "Tombstoned Tablets" header and
+  // a table entry with the proper status message.
+  string s = buf.ToString();
+  ASSERT_STR_CONTAINS(s, "Tombstoned Tablets");
+  ASSERT_STR_CONTAINS(s, "<td>Tombstoned</td>");
+  ASSERT_STR_NOT_CONTAINS(s, "<td>Tablet initializing...</td>");
+
+  // Since the consensus config shouldn't be displayed, the html should not
+  // contain the server's RPC address.
+  ASSERT_STR_NOT_CONTAINS(s, mini_server_->bound_rpc_addr().ToString());
+}
+
 class TabletServerDiskFailureTest : public TabletServerTestBase {
  public:
   virtual void SetUp() override {

http://git-wip-us.apache.org/repos/asf/kudu/blob/1cf30eff/src/kudu/tserver/ts_tablet_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/ts_tablet_manager.cc b/src/kudu/tserver/ts_tablet_manager.cc
index bef2d9d..a3c2887 100644
--- a/src/kudu/tserver/ts_tablet_manager.cc
+++ b/src/kudu/tserver/ts_tablet_manager.cc
@@ -1256,6 +1256,7 @@ Status TSTabletManager::HandleNonReadyTabletOnStartup(const scoped_refptr<Tablet
   if (data_state == TABLET_DATA_TOMBSTONED) {
     scoped_refptr<TabletReplica> dummy;
     RETURN_NOT_OK(CreateAndRegisterTabletReplica(meta, NEW_REPLICA, &dummy));
+    dummy->SetStatusMessage("Tombstoned");
   }
 
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/1cf30eff/src/kudu/tserver/tserver_path_handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tserver_path_handlers.cc b/src/kudu/tserver/tserver_path_handlers.cc
index 7cfc935..19b8bb6 100644
--- a/src/kudu/tserver/tserver_path_handlers.cc
+++ b/src/kudu/tserver/tserver_path_handlers.cc
@@ -207,6 +207,10 @@ string TabletLink(const string& id) {
                     EscapeForHtmlToString(id));
 }
 
+bool IsTombstoned(const scoped_refptr<TabletReplica>& replica) {
+  return replica->data_state() == tablet::TABLET_DATA_TOMBSTONED;
+}
+
 } // anonymous namespace
 
 void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& /*req*/,
@@ -282,7 +286,12 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& /*
                                  .PartitionDebugString(replica->tablet_metadata()->partition(),
                                                        replica->tablet_metadata()->schema());
 
+      // We don't show the config if it's a tombstone because it's misleading.
       shared_ptr<consensus::RaftConsensus> consensus = replica->shared_consensus();
+      string consensus_state_html =
+          consensus && !IsTombstoned(replica) ? ConsensusStatePBToHtml(consensus->ConsensusState())
+                                              : "";
+
       *output << Substitute(
           // Table name, tablet id, partition
           "<tr><td>$0</td><td>$1</td><td>$2</td>"
@@ -292,7 +301,7 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& /*
           tablet_id_or_link, // $1
           EscapeForHtmlToString(partition), // $2
           EscapeForHtmlToString(replica->HumanReadableState()), mem_bytes, n_bytes, // $3, $4, $5
-          consensus ? ConsensusStatePBToHtml(consensus->ConsensusState()) : "", // $6
+          consensus_state_html, // $6
           EscapeForHtmlToString(status.last_status())); // $7
     }
     *output << "<tbody></table>\n</div>\n";
@@ -301,10 +310,10 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& /*
   vector<scoped_refptr<TabletReplica>> live_replicas;
   vector<scoped_refptr<TabletReplica>> tombstoned_replicas;
   for (const scoped_refptr<TabletReplica>& replica : replicas) {
-    if (replica->HumanReadableState().find("TABLET_DATA_TOMBSTONED") == string::npos) {
-      live_replicas.push_back(replica);
-    } else {
+    if (IsTombstoned(replica)) {
       tombstoned_replicas.push_back(replica);
+    } else {
+      live_replicas.push_back(replica);
     }
   }
 
@@ -314,8 +323,10 @@ void TabletServerPathHandlers::HandleTabletsPage(const Webserver::WebRequest& /*
   }
   if (!tombstoned_replicas.empty()) {
     *output << "<h3>Tombstoned Tablets</h3>\n";
-    *output << "<p><small>Tombstoned tablets are tablets that previously "
-               "stored a replica on this server.</small></p>";
+    *output << "<p><small>Tombstone tablets are necessary for correct operation "
+               "of Kudu. These tablets have had all of their data removed from "
+               "disk and do not consume significant resources, and must not be "
+               "deleted.</small></p>";
     generate_table(tombstoned_replicas, output);
   }
 }


[3/3] kudu git commit: KUDU-2251: rowset size can overflow int in RowSetInfo

Posted by da...@apache.org.
KUDU-2251: rowset size can overflow int in RowSetInfo

This overflow causes a CHECK failure from rowset compaction planning in
tablets with rowsets with more than 2GiB of REDO deltafiles:

*** SIGABRT (@0x3ce00007614) received by PID 30228 (TID 0x7fbb52a5e700) from PID 30228; stack trace: ***
    @     0x7fbb977cb100 (unknown)
    @     0x7fbb95a985f7 __GI_raise
    @     0x7fbb95a99ce8 __GI_abort
    @          0x1af56d9 (unknown)
    @           0x8baf3d google::LogMessage::Fail()
    @           0x8bce93 google::LogMessage::SendToLog()
    @           0x8baa99 google::LogMessage::Flush()
    @           0x8bd81f google::LogMessageFatal::~LogMessageFatal()
    @           0x9f71d6 kudu::tablet::RowSetInfo::CollectOrdered()
    @           0x9d42d9 kudu::tablet::BudgetedCompactionPolicy::SetupKnapsackInput()
    @           0x9d5a3a kudu::tablet::BudgetedCompactionPolicy::PickRowSets()
    @           0x98e28f kudu::tablet::Tablet::UpdateCompactionStats()
    @           0x9aff08 kudu::tablet::CompactRowSetsOp::UpdateStats()
    @          0x1ae02b5 kudu::MaintenanceManager::FindBestOp()
    @          0x1ae2bce kudu::MaintenanceManager::RunSchedulerThread()
    @          0x1b27eda kudu::Thread::SuperviseThread()
    @     0x7fbb977c3dc5 start_thread
    @     0x7fbb95b5921d __clone
    @                0x0 (unknown)

Testing: included is a targeted unit-test which reproduces the overflow
quickly and deterministically. I also reproduced the issue using an
integration test, however that test exposed other issues which need to
be addressed before it can land (KUDU-2253). I'll be working on that in
a follow-up commit.

Change-Id: I74975cdab605b51617d93d1ae98ef72ce87e35cb
Reviewed-on: http://gerrit.cloudera.org:8080/8951
Tested-by: Kudu Jenkins
Reviewed-by: David Ribeiro Alves <da...@gmail.com>


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

Branch: refs/heads/master
Commit: 56107ac806a171db33f09c6bdce7909bb8c9bd4b
Parents: 236fc67
Author: Dan Burkert <da...@apache.org>
Authored: Thu Jan 4 10:14:55 2018 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Thu Jan 11 20:59:43 2018 +0000

----------------------------------------------------------------------
 src/kudu/tablet/compaction_policy-test.cc | 17 +++++++++++++++++
 src/kudu/tablet/mock-rowsets.h            |  2 +-
 src/kudu/tablet/rowset_info.h             | 11 ++++++-----
 3 files changed, 24 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/56107ac8/src/kudu/tablet/compaction_policy-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/compaction_policy-test.cc b/src/kudu/tablet/compaction_policy-test.cc
index 9b19230..fe548dd 100644
--- a/src/kudu/tablet/compaction_policy-test.cc
+++ b/src/kudu/tablet/compaction_policy-test.cc
@@ -200,5 +200,22 @@ TEST_F(TestCompactionPolicy, TestYcsbCompaction) {
       << qualities;
 }
 
+// Regression test for KUDU-2251 which ensures that large (> 2GiB) rowsets don't
+// cause integer overflow in compaction planning.
+TEST_F(TestCompactionPolicy, KUDU2251) {
+  RowSetVector vec = {
+    std::make_shared<MockDiskRowSet>("C", "c", 1L << 31),
+    std::make_shared<MockDiskRowSet>("B", "a", 1L << 32),
+    std::make_shared<MockDiskRowSet>("A", "b", 1L << 33)
+  };
+
+  const int kBudgetMb = 1L << 30; // enough to select all
+  unordered_set<RowSet*> picked;
+  double quality = 0;
+  NO_FATALS(RunTestCase(vec, kBudgetMb, &picked, &quality));
+  ASSERT_EQ(3, picked.size());
+  ASSERT_GE(quality, 1.0);
+}
+
 } // namespace tablet
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/56107ac8/src/kudu/tablet/mock-rowsets.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/mock-rowsets.h b/src/kudu/tablet/mock-rowsets.h
index f488302..51763d6 100644
--- a/src/kudu/tablet/mock-rowsets.h
+++ b/src/kudu/tablet/mock-rowsets.h
@@ -164,7 +164,7 @@ class MockRowSet : public RowSet {
 class MockDiskRowSet : public MockRowSet {
  public:
   MockDiskRowSet(std::string first_key, std::string last_key,
-                 int size = 1000000)
+                 uint64_t size = 1000000)
       : first_key_(std::move(first_key)),
         last_key_(std::move(last_key)),
         size_(size) {}

http://git-wip-us.apache.org/repos/asf/kudu/blob/56107ac8/src/kudu/tablet/rowset_info.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/rowset_info.h b/src/kudu/tablet/rowset_info.h
index 6cc5588..03fb9b7 100644
--- a/src/kudu/tablet/rowset_info.h
+++ b/src/kudu/tablet/rowset_info.h
@@ -17,11 +17,12 @@
 #ifndef KUDU_TABLET_ROWSET_INFO_H_
 #define KUDU_TABLET_ROWSET_INFO_H_
 
-#include "kudu/gutil/ref_counted.h"
-
+#include <cstdint>
 #include <string>
 #include <vector>
 
+#include "kudu/gutil/ref_counted.h"
+
 namespace kudu {
 namespace tablet {
 
@@ -43,7 +44,7 @@ class RowSetInfo {
                              std::vector<RowSetInfo>* min_key,
                              std::vector<RowSetInfo>* max_key);
 
-  int size_bytes() const { return extra_->size_bytes; }
+  uint64_t size_bytes() const { return extra_->size_bytes; }
   int size_mb() const { return size_mb_; }
 
   // Return the value of the CDF at the minimum key of this candidate.
@@ -97,8 +98,8 @@ class RowSetInfo {
   //
   // These are ref-counted so that RowSetInfo is copyable.
   struct ExtraData : public RefCounted<ExtraData> {
-    // Cached version of rowset_->OnDiskBaseDataSize().
-    int size_bytes;
+    // Cached version of rowset_->OnDiskBaseDataSizeWithRedos().
+    uint64_t size_bytes;
 
     // True if the RowSet has known bounds.
     // MemRowSets in particular do not.


[2/3] kudu git commit: data_dirs: fix logging message

Posted by da...@apache.org.
data_dirs: fix logging message

If fs_target_data_dirs_per_tablet is set to be greater than the number
of available directories, it will log a message that is dependent on
there being a configured metrics entity, which is not always available.
This patch avoids the potential nullptr access.

This patch also changes the logging to INFO-level instead of
WARNING-level, as the message doesn't necessarily indicate a problem.

Change-Id: I8135e0ad2411524fcb74b3e349bea7a5828a4237
Reviewed-on: http://gerrit.cloudera.org:8080/8999
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 236fc67e1c8168b72e19db29af45bdb7ef115e6c
Parents: 1cf30ef
Author: Andrew Wong <aw...@cloudera.com>
Authored: Wed Jan 10 14:03:14 2018 -0800
Committer: Andrew Wong <aw...@cloudera.com>
Committed: Thu Jan 11 18:52:02 2018 +0000

----------------------------------------------------------------------
 src/kudu/fs/data_dirs.cc | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/236fc67e/src/kudu/fs/data_dirs.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/data_dirs.cc b/src/kudu/fs/data_dirs.cc
index 7860caa..5f1c461 100644
--- a/src/kudu/fs/data_dirs.cc
+++ b/src/kudu/fs/data_dirs.cc
@@ -824,10 +824,14 @@ Status DataDirManager::CreateDataDirGroup(const string& tablet_id,
       return Status::IOError("All healthy data directories are full", "", ENOSPC);
     }
     if (PREDICT_FALSE(group_indices.size() < FLAGS_fs_target_data_dirs_per_tablet)) {
-      LOG(WARNING) << Substitute("Could only allocate $0 dirs of requested $1 for tablet $2 ($3 "
-                                 "dirs total, $4 full, $5 failed).", group_indices.size(),
-                                 FLAGS_fs_target_data_dirs_per_tablet, tablet_id, data_dirs_.size(),
-                                 metrics_->data_dirs_full.get(), metrics_->data_dirs_failed.get());
+      string msg = Substitute("Could only allocate $0 dirs of requested $1 for tablet "
+                              "$2. $3 dirs total", group_indices.size(),
+                              FLAGS_fs_target_data_dirs_per_tablet, tablet_id, data_dirs_.size());
+      if (metrics_) {
+        msg = Substitute("$0, $1 dirs full, $2 dirs failed", msg,
+                         metrics_->data_dirs_full.get(), metrics_->data_dirs_failed.get());
+      }
+      LOG(INFO) << Substitute(msg);
     }
   }
   InsertOrDie(&group_by_tablet_map_, tablet_id, DataDirGroup(group_indices));