Repository: kudu
Updated Branches:
  refs/heads/branch-1.0.x 6f6e49ca9 -> 40637ef8f


KUDU-1090: relax MemTracker uniqueness constraint

This patch relaxes the MemTracker uniqueness constraint to avoid certain
rare crashes (e.g. web UI takes a reference to a MemRowSet tracker during
Tablet::RewindSchemaForBootstrap). Details on these crashes can be found in
the associated bug report.

More specifically, the constraint is preserved, but it is only enforced
during FindTracker() or FindOrCreateTracker(). This way it is there for
MemTracker users that really should avoid duplicates (e.g. enforcing that
every tablet has just one MemTracker for all of its DeltaMemStores), but out
of the way for everyone else. With this change, we can remove the various
hacks and workarounds that were sprinkled in various tests.

Without the patch, the new test failed 2% of dist-test runs, though I could
not get it to fail locally at all.

I also removed a few test-only overrides of table creation timeout. They are
detrimental now that the default admin operation timeout is 30s (it used to
be 5s).

Change-Id: Iea8e3d383878e829188eaca65d639bb44d8b8146
Reviewed-on: http://gerrit.cloudera.org:8080/4394
Tested-by: Kudu Jenkins
Reviewed-by: Todd Lipcon <t...@apache.org>
(cherry picked from commit 4b9d2f6976f45ea57e9a2c2648f31b3a0941a569)
Reviewed-on: http://gerrit.cloudera.org:8080/4601
Reviewed-by: Adar Dembo <a...@cloudera.com>
Tested-by: Adar Dembo <a...@cloudera.com>


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

Branch: refs/heads/branch-1.0.x
Commit: 908f0206a8f2daba7a32c24c62aa9ccc25e38ff2
Parents: 6f6e49c
Author: Adar Dembo <a...@cloudera.com>
Authored: Mon Sep 12 16:36:03 2016 -0700
Committer: Dan Burkert <d...@cloudera.com>
Committed: Mon Oct 3 21:37:50 2016 +0000

----------------------------------------------------------------------
 src/kudu/consensus/consensus_queue-test.cc      |  2 -
 src/kudu/consensus/log_cache-test.cc            | 10 ++-
 src/kudu/consensus/log_cache.cc                 |  4 -
 src/kudu/fs/block_manager-stress-test.cc        |  2 -
 src/kudu/fs/block_manager-test.cc               |  2 -
 src/kudu/fs/file_block_manager.cc               |  1 -
 src/kudu/fs/fs_manager-test.cc                  |  3 -
 src/kudu/fs/log_block_manager.cc                |  1 -
 src/kudu/integration-tests/CMakeLists.txt       |  5 +-
 .../external_mini_cluster-test.cc               | 35 ++++----
 .../integration-tests/linked_list-test-util.h   |  7 +-
 .../integration-tests/master_failover-itest.cc  | 18 -----
 src/kudu/integration-tests/test_workload.cc     |  4 -
 src/kudu/integration-tests/ts_itest-base.h      |  4 -
 .../integration-tests/webserver-stress-itest.cc | 85 ++++++++++++++++++++
 src/kudu/server/server_base.cc                  |  1 -
 src/kudu/tablet/memrowset.cc                    |  1 -
 src/kudu/tablet/tablet.cc                       | 10 +--
 .../tablet/transactions/transaction_tracker.cc  |  3 -
 src/kudu/util/mem_tracker-test.cc               | 35 ++++----
 src/kudu/util/mem_tracker.cc                    | 34 ++++----
 src/kudu/util/mem_tracker.h                     | 55 ++++++-------
 22 files changed, 177 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/consensus/consensus_queue-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus_queue-test.cc 
b/src/kudu/consensus/consensus_queue-test.cc
index ff5051b..69e99c7 100644
--- a/src/kudu/consensus/consensus_queue-test.cc
+++ b/src/kudu/consensus/consensus_queue-test.cc
@@ -73,8 +73,6 @@ class ConsensusQueueTest : public KuduTest {
   }
 
   void CloseAndReopenQueue() {
-    // Blow away the memtrackers before creating the new queue.
-    queue_.reset();
     queue_.reset(new PeerMessageQueue(metric_entity_,
                                       log_.get(),
                                       FakeRaftPeerPB(kLeaderUuid),

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/consensus/log_cache-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log_cache-test.cc 
b/src/kudu/consensus/log_cache-test.cc
index d581f4f..2f764e0 100644
--- a/src/kudu/consensus/log_cache-test.cc
+++ b/src/kudu/consensus/log_cache-test.cc
@@ -74,9 +74,6 @@ class LogCacheTest : public KuduTest {
   }
 
   void CloseAndReopenCache(const OpId& preceding_id) {
-    // Blow away the memtrackers before creating the new cache.
-    cache_.reset();
-
     cache_.reset(new LogCache(metric_entity_,
                               log_.get(),
                               kPeerUuid,
@@ -271,6 +268,11 @@ TEST_F(LogCacheTest, TestMemoryLimit) {
 }
 
 TEST_F(LogCacheTest, TestGlobalMemoryLimit) {
+  // Need to force the global cache memtracker to be destroyed before calling
+  // CloseAndreopenCache(), otherwise it'll just be reused instead of recreated
+  // with a new limit.
+  cache_.reset();
+
   FLAGS_global_log_cache_size_limit_mb = 4;
   CloseAndReopenCache(MinimumOpId());
 
@@ -292,7 +294,7 @@ TEST_F(LogCacheTest, TestGlobalMemoryLimit) {
 // consumption wasn't properly managed when messages were replaced.
 TEST_F(LogCacheTest, TestReplaceMessages) {
   const int kPayloadSize = 128 * 1024;
-  shared_ptr<MemTracker> tracker = cache_->tracker_;;
+  shared_ptr<MemTracker> tracker = cache_->tracker_;
   ASSERT_EQ(0, tracker->consumption());
 
   ASSERT_OK(AppendReplicateMessagesToCache(1, 1, kPayloadSize));

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/consensus/log_cache.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/log_cache.cc b/src/kudu/consensus/log_cache.cc
index 6619a22..914d3da 100644
--- a/src/kudu/consensus/log_cache.cc
+++ b/src/kudu/consensus/log_cache.cc
@@ -104,10 +104,6 @@ LogCache::LogCache(const scoped_refptr<MetricEntity>& 
metric_entity,
 LogCache::~LogCache() {
   tracker_->Release(tracker_->consumption());
   cache_.clear();
-
-  // Don't need to unregister parent_tracker_ because it is reused in each
-  // LogCache, not duplicated.
-  tracker_->UnregisterFromParent();
 }
 
 void LogCache::Init(const OpId& preceding_op) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/fs/block_manager-stress-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/block_manager-stress-test.cc 
b/src/kudu/fs/block_manager-stress-test.cc
index bcdada9..0c128a1 100644
--- a/src/kudu/fs/block_manager-stress-test.cc
+++ b/src/kudu/fs/block_manager-stress-test.cc
@@ -388,8 +388,6 @@ TYPED_TEST(BlockManagerStressTest, StressTest) {
   LOG(INFO) << "Running on fresh block manager";
   this->RunTest(FLAGS_test_duration_secs / 2);
   LOG(INFO) << "Running on populated block manager";
-  // Blow away old memtrackers before creating new block manager.
-  this->bm_.reset();
   this->bm_.reset(this->CreateBlockManager());
   ASSERT_OK(this->bm_->Open());
   this->RunTest(FLAGS_test_duration_secs / 2);

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/fs/block_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/block_manager-test.cc 
b/src/kudu/fs/block_manager-test.cc
index 47fba51..b922eab 100644
--- a/src/kudu/fs/block_manager-test.cc
+++ b/src/kudu/fs/block_manager-test.cc
@@ -111,8 +111,6 @@ class BlockManagerTest : public KuduTest {
                             const shared_ptr<MemTracker>& parent_mem_tracker,
                             const vector<string>& paths,
                             bool create) {
-    // Blow away old memtrackers first.
-    bm_.reset();
     bm_.reset(CreateBlockManager(metric_entity, parent_mem_tracker, paths));
     if (create) {
       RETURN_NOT_OK(bm_->Create());

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/fs/file_block_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/file_block_manager.cc 
b/src/kudu/fs/file_block_manager.cc
index 53d0352..75de798 100644
--- a/src/kudu/fs/file_block_manager.cc
+++ b/src/kudu/fs/file_block_manager.cc
@@ -522,7 +522,6 @@ FileBlockManager::FileBlockManager(Env* env, const 
BlockManagerOptions& opts)
 
 FileBlockManager::~FileBlockManager() {
   STLDeleteValues(&root_paths_by_idx_);
-  mem_tracker_->UnregisterFromParent();
 }
 
 Status FileBlockManager::Create() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/fs/fs_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/fs_manager-test.cc b/src/kudu/fs/fs_manager-test.cc
index 21bb460..747352e 100644
--- a/src/kudu/fs/fs_manager-test.cc
+++ b/src/kudu/fs/fs_manager-test.cc
@@ -49,9 +49,6 @@ class FsManagerTestBase : public KuduTest {
   }
 
   void ReinitFsManager(const string& wal_path, const vector<string>& 
data_paths) {
-    // Blow away the old memtrackers first.
-    fs_manager_.reset();
-
     FsManagerOpts opts;
     opts.wal_path = wal_path;
     opts.data_paths = data_paths;

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/fs/log_block_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/fs/log_block_manager.cc b/src/kudu/fs/log_block_manager.cc
index 594c756..02a1176 100644
--- a/src/kudu/fs/log_block_manager.cc
+++ b/src/kudu/fs/log_block_manager.cc
@@ -1147,7 +1147,6 @@ LogBlockManager::~LogBlockManager() {
   STLDeleteElements(&all_containers_);
   STLDeleteValues(&thread_pools_by_root_path_);
   STLDeleteValues(&instances_by_root_path_);
-  mem_tracker_->UnregisterFromParent();
 }
 
 static const char kHolePunchErrorMsg[] =

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/integration-tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/CMakeLists.txt 
b/src/kudu/integration-tests/CMakeLists.txt
index aa6a673..b786ecf 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -51,6 +51,7 @@ ADD_KUDU_TEST(create-table-itest)
 ADD_KUDU_TEST(create-table-stress-test)
 ADD_KUDU_TEST(delete_table-test)
 ADD_KUDU_TEST(disk_reservation-itest)
+ADD_KUDU_TEST(exactly_once_writes-itest)
 ADD_KUDU_TEST(external_mini_cluster-test RESOURCE_LOCK "master-rpc-ports")
 ADD_KUDU_TEST(fuzz-itest)
 ADD_KUDU_TEST(linked_list-test RESOURCE_LOCK "master-rpc-ports")
@@ -68,11 +69,11 @@ ADD_KUDU_TEST(tablet_history_gc-itest)
 ADD_KUDU_TEST(tablet_replacement-itest)
 ADD_KUDU_TEST(ts_recovery-itest)
 ADD_KUDU_TEST(ts_tablet_manager-itest)
+ADD_KUDU_TEST(webserver-stress-itest)
 ADD_KUDU_TEST(write_throttling-itest)
-ADD_KUDU_TEST(exactly_once_writes-itest)
 
 # Some tests have additional dependencies
-set(KUDU_TEST_LINK_LIBS kudu_client kudu_tools_util ${KUDU_TEST_LINK_LIBS})
+set(KUDU_TEST_LINK_LIBS kudu_tools_util ${KUDU_TEST_LINK_LIBS})
 ADD_KUDU_TEST(flex_partitioning-itest)
 ADD_KUDU_TEST(full_stack-insert-scan-test RUN_SERIAL true)
 ADD_KUDU_TEST(update_scan_delta_compact-test RUN_SERIAL true)

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/integration-tests/external_mini_cluster-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster-test.cc 
b/src/kudu/integration-tests/external_mini_cluster-test.cc
index cb3d37b..944e958 100644
--- a/src/kudu/integration-tests/external_mini_cluster-test.cc
+++ b/src/kudu/integration-tests/external_mini_cluster-test.cc
@@ -15,13 +15,16 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <glog/logging.h>
-#include <gtest/gtest.h>
 #include <sys/types.h>
 #include <unistd.h>
-#include <vector>
+
+#include <string>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
 
 #include "kudu/integration-tests/external_mini_cluster.h"
+#include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
 #include "kudu/util/metrics.h"
@@ -33,24 +36,18 @@ METRIC_DECLARE_gauge_uint64(threads_running);
 
 namespace kudu {
 
-class EMCTest : public KuduTest {
- public:
-  EMCTest() {
-    // Hard-coded RPC ports for the masters. This is safe, as this unit test
-    // runs under a resource lock (see CMakeLists.txt in this directory).
-    // TODO we should have a generic method to obtain n free ports.
-    master_peer_ports_ = { 11010, 11011, 11012 };
-  }
-
- protected:
-  std::vector<uint16_t> master_peer_ports_;
-};
+using std::string;
+using strings::Substitute;
 
-TEST_F(EMCTest, TestBasicOperation) {
+TEST_F(KuduTest, TestBasicOperation) {
   ExternalMiniClusterOptions opts;
-  opts.num_masters = master_peer_ports_.size();
+
+  // Hard-coded RPC ports for the masters. This is safe, as this unit test
+  // runs under a resource lock (see CMakeLists.txt in this directory).
+  // TODO we should have a generic method to obtain n free ports.
+  opts.master_rpc_ports = { 11010, 11011, 11012 };
+  opts.num_masters = opts.master_rpc_ports.size();
   opts.num_tablet_servers = 3;
-  opts.master_rpc_ports = master_peer_ports_;
 
   ExternalMiniCluster cluster(opts);
   ASSERT_OK(cluster.Start());
@@ -80,7 +77,7 @@ TEST_F(EMCTest, TestBasicOperation) {
     SCOPED_TRACE(i);
     ExternalTabletServer* ts = CHECK_NOTNULL(cluster.tablet_server(i));
     HostPort ts_rpc = ts->bound_rpc_hostport();
-    string expected_prefix = strings::Substitute("$0:", 
cluster.GetBindIpForTabletServer(i));
+    string expected_prefix = Substitute("$0:", 
cluster.GetBindIpForTabletServer(i));
     EXPECT_NE(expected_prefix, "127.0.0.1") << "Should bind to unique 
per-server hosts";
     EXPECT_TRUE(HasPrefixString(ts_rpc.ToString(), expected_prefix)) << 
ts_rpc.ToString();
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/integration-tests/linked_list-test-util.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/linked_list-test-util.h 
b/src/kudu/integration-tests/linked_list-test-util.h
index 9b2175c..b3d9ea5 100644
--- a/src/kudu/integration-tests/linked_list-test-util.h
+++ b/src/kudu/integration-tests/linked_list-test-util.h
@@ -290,11 +290,16 @@ class PeriodicWebUIChecker {
     master_pages.push_back("/tables");
     master_pages.push_back("/dump-entities");
     master_pages.push_back("/tablet-servers");
+    master_pages.push_back("/mem-trackers");
 
     ts_pages.push_back("/metrics");
     ts_pages.push_back("/tablets");
-    ts_pages.push_back(strings::Substitute("/transactions?tablet_id=$0", 
tablet_id));
+    if (!tablet_id.empty()) {
+      ts_pages.push_back(strings::Substitute("/transactions?tablet_id=$0",
+                                             tablet_id));
+    }
     ts_pages.push_back("/maintenance-manager");
+    ts_pages.push_back("/mem-trackers");
 
     // Generate list of urls for each master and tablet server
     for (int i = 0; i < cluster.num_masters(); i++) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/integration-tests/master_failover-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/master_failover-itest.cc 
b/src/kudu/integration-tests/master_failover-itest.cc
index 104d1b4..5063350 100644
--- a/src/kudu/integration-tests/master_failover-itest.cc
+++ b/src/kudu/integration-tests/master_failover-itest.cc
@@ -29,7 +29,6 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
 #include "kudu/integration-tests/external_mini_cluster.h"
-#include "kudu/integration-tests/linked_list-test-util.h"
 #include "kudu/master/sys_catalog.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/random.h"
@@ -467,22 +466,5 @@ TEST_F(MasterFailoverTest, TestMasterPermanentFailure) {
   }
 }
 
-// Tests that accessing the web UI while the master is starting doesn't cause
-// any crashes.
-TEST_F(MasterFailoverTest, TestWebUIDuringStartup) {
-  PeriodicWebUIChecker checker(
-      *cluster_.get(),
-      "doesn't matter", // it'll ping a non-existent page
-      MonoDelta::FromMilliseconds(50));
-
-  for (int i = 0; i < 5; i++) {
-    for (int j = 0; j < cluster_->num_masters(); j++) {
-      cluster_->master(j)->Shutdown();
-      ASSERT_OK(cluster_->master(j)->Restart());
-      ASSERT_OK(cluster_->master(j)->WaitForCatalogManager());
-    }
-  }
-}
-
 } // namespace client
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/integration-tests/test_workload.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/test_workload.cc 
b/src/kudu/integration-tests/test_workload.cc
index e7c474a..94bcb7e 100644
--- a/src/kudu/integration-tests/test_workload.cc
+++ b/src/kudu/integration-tests/test_workload.cc
@@ -204,10 +204,6 @@ void TestWorkload::Setup() {
              .num_replicas(num_replicas_)
              .set_range_partition_columns({ "key" })
              .split_rows(splits)
-             // NOTE: this is quite high as a timeout, but the default (5 sec) 
does not
-             // seem to be high enough in some cases (see KUDU-550). We should 
remove
-             // this once that ticket is addressed.
-             .timeout(MonoDelta::FromSeconds(20))
              .Create());
   } else {
     LOG(INFO) << "TestWorkload: Skipping table creation because table "

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/integration-tests/ts_itest-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/ts_itest-base.h 
b/src/kudu/integration-tests/ts_itest-base.h
index 4b8f905..35ded6a 100644
--- a/src/kudu/integration-tests/ts_itest-base.h
+++ b/src/kudu/integration-tests/ts_itest-base.h
@@ -441,10 +441,6 @@ class TabletServerIntegrationTestBase : public 
TabletServerTestBase {
              .schema(&client_schema)
              .set_range_partition_columns({ "key" })
              .num_replicas(FLAGS_num_replicas)
-             // NOTE: this is quite high as a timeout, but the default (5 sec) 
does not
-             // seem to be high enough in some cases (see KUDU-550). We should 
remove
-             // this once that ticket is addressed.
-             .timeout(MonoDelta::FromSeconds(20))
              .Create());
     ASSERT_OK(client_->OpenTable(kTableId, &table_));
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/integration-tests/webserver-stress-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/webserver-stress-itest.cc 
b/src/kudu/integration-tests/webserver-stress-itest.cc
new file mode 100644
index 0000000..903e3bb
--- /dev/null
+++ b/src/kudu/integration-tests/webserver-stress-itest.cc
@@ -0,0 +1,85 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "kudu/integration-tests/external_mini_cluster.h"
+#include "kudu/integration-tests/linked_list-test-util.h"
+#include "kudu/integration-tests/test_workload.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+namespace kudu {
+
+using std::string;
+using std::unique_ptr;
+using std::vector;
+
+// Tests that pounding the web UI doesn't cause any crashes.
+TEST_F(KuduTest, TestWebUIDoesNotCrashCluster) {
+#ifdef THREAD_SANITIZER
+  // When using TSAN, more than one checker is too much load on the cluster.
+  const int kWebUICheckers = 1;
+#else
+  const int kWebUICheckers = 10;
+#endif
+  const int kNumTablets = 50;
+
+  ExternalMiniClusterOptions opts;
+  opts.extra_master_flags.push_back("--never_fsync");
+  opts.extra_tserver_flags.push_back("--never_fsync");
+  ExternalMiniCluster cluster(opts);
+  ASSERT_OK(cluster.Start());
+
+  // Start pounding the master and tserver's web UIs.
+  vector<unique_ptr<PeriodicWebUIChecker>> checkers;
+  for (int i = 0; i < kWebUICheckers; i++) {
+    checkers.emplace_back(new PeriodicWebUIChecker(
+        cluster,
+        "doesn't matter", // will ping a non-existent page
+        MonoDelta::FromMilliseconds(1)));
+  }
+
+  // Create a table and write to it. Just a few rows, so that there's something
+  // in the tablets' WALs.
+  TestWorkload work(&cluster);
+  work.set_timeout_allowed(true);
+  work.set_num_replicas(1);
+  work.set_num_tablets(kNumTablets);
+  work.Setup();
+  work.Start();
+  while (work.rows_inserted() < 100) {
+    SleepFor(MonoDelta::FromMilliseconds(10));
+  }
+  work.StopAndJoin();
+
+  // Restart the cluster.
+  NO_FATALS(cluster.AssertNoCrashes());
+  cluster.Shutdown();
+  ASSERT_OK(cluster.Restart());
+  ASSERT_OK(cluster.WaitForTabletsRunning(cluster.tablet_server(0),
+                                          kNumTablets,
+                                          MonoDelta::FromSeconds(30)));
+  NO_FATALS(cluster.AssertNoCrashes());
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/server/server_base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index c05d8a9..4d5fb46 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -120,7 +120,6 @@ ServerBase::ServerBase(string name, const 
ServerBaseOptions& options,
 
 ServerBase::~ServerBase() {
   Shutdown();
-  mem_tracker_->UnregisterFromParent();
 }
 
 Sockaddr ServerBase::first_rpc_address() const {

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/tablet/memrowset.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/memrowset.cc b/src/kudu/tablet/memrowset.cc
index 57c599d..d30f4a1 100644
--- a/src/kudu/tablet/memrowset.cc
+++ b/src/kudu/tablet/memrowset.cc
@@ -105,7 +105,6 @@ MemRowSet::MemRowSet(int64_t id,
 }
 
 MemRowSet::~MemRowSet() {
-  mem_tracker_->UnregisterFromParent();
 }
 
 Status MemRowSet::DebugDump(vector<string> *lines) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/tablet/tablet.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index 34e1c91..508aef7 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -204,8 +204,6 @@ Tablet::Tablet(const scoped_refptr<TabletMetadata>& 
metadata,
 
 Tablet::~Tablet() {
   Shutdown();
-  dms_mem_tracker_->UnregisterFromParent();
-  mem_tracker_->UnregisterFromParent();
 }
 
 Status Tablet::Open() {
@@ -896,13 +894,7 @@ Status Tablet::RewindSchemaForBootstrap(const Schema& 
new_schema,
     shared_ptr<MemRowSet> old_mrs = components_->memrowset;
     shared_ptr<RowSetTree> old_rowsets = components_->rowsets;
     CHECK(old_mrs->empty());
-    int64_t old_mrs_id = old_mrs->mrs_id();
-    // We have to reset the components here before creating the new MemRowSet,
-    // or else the new MRS will end up trying to claim the same MemTracker ID
-    // as the old one.
-    components_.reset();
-    old_mrs.reset();
-    shared_ptr<MemRowSet> new_mrs(new MemRowSet(old_mrs_id, new_schema,
+    shared_ptr<MemRowSet> new_mrs(new MemRowSet(old_mrs->mrs_id(), new_schema,
                                                 log_anchor_registry_.get(), 
mem_tracker_));
     components_ = new TabletComponents(new_mrs, old_rowsets);
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/tablet/transactions/transaction_tracker.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/transactions/transaction_tracker.cc 
b/src/kudu/tablet/transactions/transaction_tracker.cc
index f2bc6c4..95f9f1e 100644
--- a/src/kudu/tablet/transactions/transaction_tracker.cc
+++ b/src/kudu/tablet/transactions/transaction_tracker.cc
@@ -88,9 +88,6 @@ TransactionTracker::TransactionTracker() {
 TransactionTracker::~TransactionTracker() {
   std::lock_guard<simple_spinlock> l(lock_);
   CHECK_EQ(pending_txns_.size(), 0);
-  if (mem_tracker_) {
-    mem_tracker_->UnregisterFromParent();
-  }
 }
 
 Status TransactionTracker::Add(TransactionDriver* driver) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/util/mem_tracker-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/mem_tracker-test.cc 
b/src/kudu/util/mem_tracker-test.cc
index 4c1b244..22a0bcd 100644
--- a/src/kudu/util/mem_tracker-test.cc
+++ b/src/kudu/util/mem_tracker-test.cc
@@ -309,34 +309,37 @@ TEST(MemTrackerTest, TcMallocRootTracker) {
 }
 #endif
 
-TEST(MemTrackerTest, UnregisterFromParent) {
+TEST(MemTrackerTest, CollisionDetection) {
   shared_ptr<MemTracker> p = MemTracker::CreateTracker(-1, "parent");
   shared_ptr<MemTracker> c = MemTracker::CreateTracker(-1, "child", p);
-  vector<shared_ptr<MemTracker> > all;
+  vector<shared_ptr<MemTracker>> all;
 
   // Three trackers: root, parent, and child.
   MemTracker::ListTrackers(&all);
   ASSERT_EQ(3, all.size());
 
-  c->UnregisterFromParent();
-
-  // Now only two because the child cannot be found from the root, though it is
-  // still alive.
+  // Now only two because the child has been destroyed.
+  c.reset();
   MemTracker::ListTrackers(&all);
   ASSERT_EQ(2, all.size());
   shared_ptr<MemTracker> not_found;
   ASSERT_FALSE(MemTracker::FindTracker("child", &not_found, p));
 
-  // We can also recreate the child with the same name without colliding
-  // with the old one.
-  shared_ptr<MemTracker> c2 = MemTracker::CreateTracker(-1, "child", p);
-
-  // We should still able to walk up to the root from the unregistered child
-  // without crashing.
-  LOG(INFO) << c->ToString();
-
-  // And this should no-op.
-  c->UnregisterFromParent();
+  // Let's duplicate the parent. It's not recommended, but it's allowed.
+  shared_ptr<MemTracker> p2 = MemTracker::CreateTracker(-1, "parent");
+  ASSERT_EQ(p->ToString(), p2->ToString());
+
+  // Only when we do a Find() operation do we crash.
+#ifndef NDEBUG
+  const string kDeathMsg = "Multiple memtrackers with same id";
+  EXPECT_DEATH({
+    shared_ptr<MemTracker> found;
+    MemTracker::FindTracker("parent", &found);
+  }, kDeathMsg);
+  EXPECT_DEATH({
+    MemTracker::FindOrCreateTracker(-1, "parent");
+  }, kDeathMsg);
+#endif
 }
 
 TEST(MemTrackerTest, TestMultiThreadedRegisterAndDestroy) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/util/mem_tracker.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/mem_tracker.cc b/src/kudu/util/mem_tracker.cc
index f5d7adc..51ee211 100644
--- a/src/kudu/util/mem_tracker.cc
+++ b/src/kudu/util/mem_tracker.cc
@@ -188,16 +188,12 @@ MemTracker::~MemTracker() {
     DCHECK(consumption() == 0) << "Memory tracker " << ToString()
         << " has unreleased consumption " << consumption();
     parent_->Release(consumption());
-    UnregisterFromParent();
-  }
-}
 
-void MemTracker::UnregisterFromParent() {
-  DCHECK(parent_);
-  MutexLock l(parent_->child_trackers_lock_);
-  if (child_tracker_it_ != parent_->child_trackers_.end()) {
-    parent_->child_trackers_.erase(child_tracker_it_);
-    child_tracker_it_ = parent_->child_trackers_.end();
+    MutexLock l(parent_->child_trackers_lock_);
+    if (child_tracker_it_ != parent_->child_trackers_.end()) {
+      parent_->child_trackers_.erase(child_tracker_it_);
+      child_tracker_it_ = parent_->child_trackers_.end();
+    }
   }
 }
 
@@ -227,13 +223,23 @@ bool MemTracker::FindTrackerUnlocked(const string& id,
                                      const shared_ptr<MemTracker>& parent) {
   DCHECK(parent != NULL);
   parent->child_trackers_lock_.AssertAcquired();
+  vector<shared_ptr<MemTracker>> found;
   for (const auto& child_weak : parent->child_trackers_) {
     shared_ptr<MemTracker> child = child_weak.lock();
     if (child && child->id() == id) {
-      *tracker = std::move(child);
-      return true;
+      found.emplace_back(std::move(child));
     }
   }
+  if (PREDICT_TRUE(found.size() == 1)) {
+    *tracker = found[0];
+    return true;
+  } else if (found.size() > 1) {
+    LOG(DFATAL) <<
+        Substitute("Multiple memtrackers with same id ($0) found on parent $1",
+                   id, parent->ToString());
+    *tracker = found[0];
+    return true;
+  }
   return false;
 }
 
@@ -551,12 +557,6 @@ void MemTracker::Init() {
 
 void MemTracker::AddChildTrackerUnlocked(const shared_ptr<MemTracker>& 
tracker) {
   child_trackers_lock_.AssertAcquired();
-#ifndef NDEBUG
-  shared_ptr<MemTracker> found;
-  CHECK(!FindTrackerUnlocked(tracker->id(), &found, shared_from_this()))
-    << Substitute("Duplicate memory tracker (id $0) on parent $1",
-                  tracker->id(), ToString());
-#endif
   tracker->child_tracker_it_ = child_trackers_.insert(child_trackers_.end(), 
tracker);
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/908f0206/src/kudu/util/mem_tracker.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/mem_tracker.h b/src/kudu/util/mem_tracker.h
index 6cd980f..ae6de74 100644
--- a/src/kudu/util/mem_tracker.h
+++ b/src/kudu/util/mem_tracker.h
@@ -44,16 +44,16 @@ class MemTracker;
 // ancestor to all MemTrackers. All operations that discover MemTrackers begin
 // at the root and work their way down the tree, while operations that deal
 // with adjusting memory consumption begin at a particular MemTracker and work
-// their way up the tree to the root. The tree structure is strictly enforced:
-// all MemTrackers (except the root) must have a parent, and all children
-// belonging to a parent must have unique ids.
+// their way up the tree to the root. All MemTrackers (except the root) must
+// have a parent. As a rule, all children belonging to a parent should have
+// unique ids, but this is only enforced during a Find() operation to allow for
+// transient duplicates (e.g. the web UI grabbing very short-lived references
+// to all MemTrackers while rendering a web page). This also means id
+// uniqueness only exists where it's actually needed.
 //
 // When a MemTracker begins its life, it has a strong reference to its parent
-// and the parent has a weak reference to it. The strong reference remains for
-// the lifetime of the MemTracker, but the weak reference can be dropped via
-// UnregisterFromParent(). A MemTracker in this state may continue servicing
-// memory consumption operations while allowing a new MemTracker with the same
-// id to be created on the old parent.
+// and the parent has a weak reference to it. Both remain for the lifetime of
+// the MemTracker.
 //
 // By default, memory consumption is tracked via calls to Consume()/Release(), 
either to
 // the tracker itself or to one of its descendents. Alternatively, a 
consumption function
@@ -95,35 +95,24 @@ class MemTracker : public 
std::enable_shared_from_this<MemTracker> {
 
   ~MemTracker();
 
-  // Removes this tracker from its parent's children. This tracker retains its
-  // link to its parent. Must be called on a tracker with a parent.
-  //
-  // Automatically called in the MemTracker destructor, but should be called
-  // explicitly when an object is destroyed if that object is also the "primary
-  // owner" of a tracker (i.e. the object that originally created the tracker).
-  // This orphans the tracker so that if the object is recreated, its new
-  // tracker won't collide with the now orphaned tracker.
-  //
-  // Is thread-safe on the parent but not the child. Meaning, multiple trackers
-  // that share the same parent can all UnregisterFromParent() at the same
-  // time, but all UnregisterFromParent() calls on a given tracker must be
-  // externally synchronized.
-  void UnregisterFromParent();
-
   // Creates and adds the tracker to the tree so that it can be retrieved with
   // FindTracker/FindOrCreateTracker.
   //
   // byte_limit < 0 means no limit; 'id' is a used as a label for LogUsage()
-  // and web UI and must be unique for the given parent. Use the two-argument
-  // form if there is no parent.
+  // and web UI. Use the two-argument form if there is no parent.
   static std::shared_ptr<MemTracker> CreateTracker(
       int64_t byte_limit,
       const std::string& id,
       const std::shared_ptr<MemTracker>& parent = 
std::shared_ptr<MemTracker>());
 
   // If a tracker with the specified 'id' and 'parent' exists in the tree, sets
-  // 'tracker' to reference that instance. Use the two-argument form if there
-  // is no parent. Returns false if no such tracker exists.
+  // 'tracker' to reference that instance. Returns false if no such tracker
+  // exists.
+  //
+  // Use the two-argument form if there is no parent.
+  //
+  // Note: this function will enforce that 'id' is unique amongst the children
+  // of 'parent'.
   static bool FindTracker(
       const std::string& id,
       std::shared_ptr<MemTracker>* tracker,
@@ -131,8 +120,12 @@ class MemTracker : public 
std::enable_shared_from_this<MemTracker> {
 
   // If a tracker with the specified 'id' and 'parent' exists in the tree,
   // returns a shared_ptr to that instance. Otherwise, creates a new
-  // MemTracker with the specified byte_limit, id, and parent. Use the two
-  // argument form if there is no parent.
+  // MemTracker with the specified byte_limit, id, and parent.
+  //
+  // Use the two argument form if there is no parent.
+  //
+  // Note: this function will enforce that 'id' is unique amongst the children
+  // of 'parent'.
   static std::shared_ptr<MemTracker> FindOrCreateTracker(
       int64_t byte_limit,
       const std::string& id,
@@ -228,8 +221,8 @@ class MemTracker : public 
std::enable_shared_from_this<MemTracker> {
     log_stack_ = log_stack;
   }
 
-  // Returns a textual representation of the tracker that is guaranteed to be
-  // globally unique.
+  // Returns a textual representation of the tracker that is likely (but not
+  // guaranteed) to be globally unique.
   std::string ToString() const;
 
  private:

Reply via email to