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 2017/10/15 04:10:30 UTC

[1/3] kudu git commit: KUDU-2184. Avoid allocations larger than 1MB

Repository: kudu
Updated Branches:
  refs/heads/master b6f727ea2 -> 24308aa1a


KUDU-2184. Avoid allocations larger than 1MB

tcmalloc doesn't perform well for allocations larger than 1MB[1]. We
currently use large allocations for full arenas (in particular the MRS)
which can cause large latency blips when a thread tries to allocate a
new arena component.

This just changes the max buffer size for all arenas to be 1MB and adds
a DCHECK so we avoid this problem in the future.

Additionally I increased the buffer size for the /memz output so that we
can get more stats on the occupancy of different allocator size classes.

[1] https://github.com/gperftools/gperftools/issues/535

Change-Id: I19c8ae9ce268e2623a89624e19db673931a093c0
Reviewed-on: http://gerrit.cloudera.org:8080/8239
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/eb5c7e2f
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/eb5c7e2f
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/eb5c7e2f

Branch: refs/heads/master
Commit: eb5c7e2f0c73729a4486bc71512739b0788b1574
Parents: b6f727e
Author: Todd Lipcon <to...@apache.org>
Authored: Mon Oct 9 14:22:48 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Oct 13 01:58:04 2017 +0000

----------------------------------------------------------------------
 src/kudu/cfile/binary_dict_block.cc          | 2 +-
 src/kudu/cfile/cfile-test-base.h             | 2 +-
 src/kudu/cfile/cfile-test.cc                 | 2 +-
 src/kudu/cfile/cfile_util.cc                 | 2 +-
 src/kudu/server/default-path-handlers.cc     | 7 ++++---
 src/kudu/tablet/concurrent_btree.h           | 5 +----
 src/kudu/tablet/deltamemstore.cc             | 2 +-
 src/kudu/tablet/memrowset.cc                 | 2 +-
 src/kudu/tablet/tablet-test-base.h           | 2 +-
 src/kudu/tablet/tablet_random_access-test.cc | 2 +-
 src/kudu/tablet/transactions/transaction.cc  | 2 +-
 src/kudu/util/memory/arena.cc                | 9 +++++----
 12 files changed, 19 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/cfile/binary_dict_block.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/binary_dict_block.cc b/src/kudu/cfile/binary_dict_block.cc
index 9e96163..f00d75d 100644
--- a/src/kudu/cfile/binary_dict_block.cc
+++ b/src/kudu/cfile/binary_dict_block.cc
@@ -52,7 +52,7 @@ namespace cfile {
 BinaryDictBlockBuilder::BinaryDictBlockBuilder(const WriterOptions* options)
     : options_(options),
       dict_block_(options_),
-      dictionary_strings_arena_(1024, 32*1024*1024),
+      dictionary_strings_arena_(1024, 1024*1024),
       mode_(kCodeWordMode) {
   data_builder_.reset(new BShufBlockBuilder<UINT32>(options_));
   // We use this invalid StringPiece for the "empty key". It's safe to build such

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/cfile/cfile-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile-test-base.h b/src/kudu/cfile/cfile-test-base.h
index f4f0c8f..64fd35d 100644
--- a/src/kudu/cfile/cfile-test-base.h
+++ b/src/kudu/cfile/cfile-test-base.h
@@ -467,7 +467,7 @@ void TimeReadFile(FsManager* fs_manager, const BlockId& block_id, size_t *count_
   ASSERT_OK(reader->NewIterator(&iter, CFileReader::CACHE_BLOCK));
   ASSERT_OK(iter->SeekToOrdinal(0));
 
-  Arena arena(8192, 8*1024*1024);
+  Arena arena(8192, 1024 * 1024);
   int count = 0;
   switch (reader->type_info()->physical_type()) {
     case UINT8:

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/cfile/cfile-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile-test.cc b/src/kudu/cfile/cfile-test.cc
index 364fb93..f7ffe85 100644
--- a/src/kudu/cfile/cfile-test.cc
+++ b/src/kudu/cfile/cfile-test.cc
@@ -200,7 +200,7 @@ class TestCFile : public CFileTestBase {
     gscoped_ptr<CFileIterator> iter;
     ASSERT_OK(reader->NewIterator(&iter, CFileReader::CACHE_BLOCK));
 
-    Arena arena(8192, 8*1024*1024);
+    Arena arena(8192, 1024 * 1024);
     ScopedColumnBlock<DataGeneratorType::kDataType> cb(10);
 
     SelectionVector sel(10);

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/cfile/cfile_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile_util.cc b/src/kudu/cfile/cfile_util.cc
index 2995407..31c5af3 100644
--- a/src/kudu/cfile/cfile_util.cc
+++ b/src/kudu/cfile/cfile_util.cc
@@ -46,7 +46,7 @@ Status DumpIterator(const CFileReader& reader,
                     int num_rows,
                     int indent) {
 
-  Arena arena(8192, 8*1024*1024);
+  Arena arena(8192, 1024 * 1024);
   uint8_t buf[kBufSize];
   const TypeInfo *type = reader.type_info();
   size_t max_rows = kBufSize/type->size();

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/server/default-path-handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/default-path-handlers.cc b/src/kudu/server/default-path-handlers.cc
index e71922b..8260ec8 100644
--- a/src/kudu/server/default-path-handlers.cc
+++ b/src/kudu/server/default-path-handlers.cc
@@ -152,10 +152,11 @@ static void MemUsageHandler(const Webserver::WebRequest& req,
 #ifndef TCMALLOC_ENABLED
   (*output) << "Memory tracking is not available unless tcmalloc is enabled.";
 #else
-  char buf[2048];
-  MallocExtension::instance()->GetStats(buf, 2048);
+  faststring buf;
+  buf.resize(32 * 1024);
+  MallocExtension::instance()->GetStats(reinterpret_cast<char*>(buf.data()), buf.size());
   // Replace new lines with <br> for html
-  string tmp(buf);
+  string tmp(reinterpret_cast<char*>(buf.data()));
   boost::replace_all(tmp, "\n", tags.line_break);
   (*output) << tmp << tags.end_pre_tag;
 #endif

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/tablet/concurrent_btree.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/concurrent_btree.h b/src/kudu/tablet/concurrent_btree.h
index 0e4dfa6..a80fa6b 100644
--- a/src/kudu/tablet/concurrent_btree.h
+++ b/src/kudu/tablet/concurrent_btree.h
@@ -952,10 +952,7 @@ class PreparedMutation {
 template<class Traits = BTreeTraits>
 class CBTree {
  public:
-  CBTree()
-    : arena_(new typename Traits::ArenaType(512*1024, 4*1024*1024)),
-      root_(NewLeaf(false)),
-      frozen_(false) {
+  CBTree() : CBTree(std::make_shared<typename Traits::ArenaType>(4 * 1024, 1024 * 1024)) {
   }
 
   explicit CBTree(std::shared_ptr<typename Traits::ArenaType> arena)

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/tablet/deltamemstore.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/deltamemstore.cc b/src/kudu/tablet/deltamemstore.cc
index d929fa8..8adfee5 100644
--- a/src/kudu/tablet/deltamemstore.cc
+++ b/src/kudu/tablet/deltamemstore.cc
@@ -54,7 +54,7 @@ using strings::Substitute;
 ////////////////////////////////////////////////////////////
 
 static const int kInitialArenaSize = 16;
-static const int kMaxArenaBufferSize = 5*1024*1024;
+static const int kMaxArenaBufferSize = 1024*1024;
 
 Status DeltaMemStore::Create(int64_t id,
                              int64_t rs_id,

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/tablet/memrowset.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/memrowset.cc b/src/kudu/tablet/memrowset.cc
index 61130e7..1bfa6c7 100644
--- a/src/kudu/tablet/memrowset.cc
+++ b/src/kudu/tablet/memrowset.cc
@@ -62,7 +62,7 @@ using log::LogAnchorRegistry;
 using strings::Substitute;
 
 static const int kInitialArenaSize = 16;
-static const int kMaxArenaBufferSize = 8*1024*1024;
+static const int kMaxArenaBufferSize = 1024*1024;
 
 bool MRSRow::IsGhost() const {
   bool is_ghost = false;

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/tablet/tablet-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet-test-base.h b/src/kudu/tablet/tablet-test-base.h
index 92ce1b6..d813aa7 100644
--- a/src/kudu/tablet/tablet-test-base.h
+++ b/src/kudu/tablet/tablet-test-base.h
@@ -310,7 +310,7 @@ class TabletTestBase : public KuduTabletTest {
     KuduTabletTest(TESTSETUP::CreateSchema(), clock_type),
     setup_(),
     max_rows_(setup_.GetMaxRows()),
-    arena_(1024, 4*1024*1024)
+    arena_(1024, 1024 * 1024)
   {}
 
   // Inserts "count" rows.

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/tablet/tablet_random_access-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_random_access-test.cc b/src/kudu/tablet/tablet_random_access-test.cc
index e31ae5d..4c25dec 100644
--- a/src/kudu/tablet/tablet_random_access-test.cc
+++ b/src/kudu/tablet/tablet_random_access-test.cc
@@ -279,7 +279,7 @@ class TestRandomAccess : public KuduTabletTest {
     optional<ExpectedKeyValueRow> ret;
     int n_results = 0;
 
-    Arena arena(1024, 4*1024*1024);
+    Arena arena(1024, 1024 * 1024);
     RowBlock block(schema, 100, &arena);
     while (iter->HasNext()) {
       arena.Reset();

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/tablet/transactions/transaction.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/transactions/transaction.cc b/src/kudu/tablet/transactions/transaction.cc
index 543dcd8..7a95914 100644
--- a/src/kudu/tablet/transactions/transaction.cc
+++ b/src/kudu/tablet/transactions/transaction.cc
@@ -34,7 +34,7 @@ TransactionState::TransactionState(TabletReplica* tablet_replica)
     : tablet_replica_(tablet_replica),
       completion_clbk_(new TransactionCompletionCallback()),
       timestamp_error_(0),
-      arena_(1024, 4 * 1024 * 1024),
+      arena_(1024, 1024 * 1024),
       external_consistency_mode_(CLIENT_PROPAGATED) {
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/eb5c7e2f/src/kudu/util/memory/arena.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/memory/arena.cc b/src/kudu/util/memory/arena.cc
index 6f65df4..490be0a 100644
--- a/src/kudu/util/memory/arena.cc
+++ b/src/kudu/util/memory/arena.cc
@@ -39,15 +39,16 @@ ArenaBase<THREADSAFE>::ArenaBase(
     : buffer_allocator_(buffer_allocator),
       max_buffer_size_(max_buffer_size),
       arena_footprint_(0) {
+  DCHECK_LE(max_buffer_size_, 1024 * 1024)
+      << "Should not use buffer sizes larger than 1MB due to tcmalloc inefficiencies";
   AddComponent(CHECK_NOTNULL(NewComponent(initial_buffer_size, 0)));
 }
 
 template <bool THREADSAFE>
 ArenaBase<THREADSAFE>::ArenaBase(size_t initial_buffer_size, size_t max_buffer_size)
-    : buffer_allocator_(HeapBufferAllocator::Get()),
-      max_buffer_size_(max_buffer_size),
-      arena_footprint_(0) {
-  AddComponent(CHECK_NOTNULL(NewComponent(initial_buffer_size, 0)));
+    : ArenaBase<THREADSAFE>(HeapBufferAllocator::Get(),
+                            initial_buffer_size,
+                            max_buffer_size) {
 }
 
 template <bool THREADSAFE>


[3/3] kudu git commit: tsan: flush TSAN memory every 10 seconds

Posted by al...@apache.org.
tsan: flush TSAN memory every 10 seconds

I'm seeing a TSAN precommit test fail in a hard-to-diagnose way because the
Linux OOM killer is killing tservers underneath it. It doesn't appear to
be any leak -- rather, TSAN is just using too much memory due to the
test doing a lot of allocation.

This enables a TSAN option to flush its memory every 10 seconds. This
has the downside of potentially missing races that happen far apart in
time (but those are unlikely to actually trigger in practice) but
reduces RSS significantly.

Change-Id: I721f126eb731c7f35afb732cebe48abd52c3ebd1
Reviewed-on: http://gerrit.cloudera.org:8080/8251
Tested-by: Alexey Serbin <as...@cloudera.com>
Reviewed-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: 24308aa1a5fd05e616dad2e8ee7482a74b9350de
Parents: 83b6931
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Oct 10 14:08:26 2017 -0700
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Sun Oct 15 04:09:19 2017 +0000

----------------------------------------------------------------------
 build-support/run-test.sh | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/24308aa1/build-support/run-test.sh
----------------------------------------------------------------------
diff --git a/build-support/run-test.sh b/build-support/run-test.sh
index 941912b..6073f19 100755
--- a/build-support/run-test.sh
+++ b/build-support/run-test.sh
@@ -104,6 +104,9 @@ fi
 # Configure TSAN (ignored if this isn't a TSAN build).
 TSAN_OPTIONS="$TSAN_OPTIONS suppressions=$SOURCE_ROOT/build-support/tsan-suppressions.txt"
 TSAN_OPTIONS="$TSAN_OPTIONS history_size=7"
+#   Flush TSAN memory every 10 seconds - this prevents RSS blowup in unit tests
+#   which can cause tests to get killed by the OOM killer.
+TSAN_OPTIONS="$TSAN_OPTIONS flush_memory_ms=10000"
 TSAN_OPTIONS="$TSAN_OPTIONS external_symbolizer_path=$ASAN_SYMBOLIZER_PATH"
 export TSAN_OPTIONS
 


[2/3] kudu git commit: Reduce log spew from rpcz_store.cc

Posted by al...@apache.org.
Reduce log spew from rpcz_store.cc

The sampled RPC call statement has been filing the log. Let's dial down its log level.
This particularly impacts Impala as its default log level is 1.

If it turns out that this log statement is useful for Kudu deployment, we can
consider doing this change in Impala only.

Testing done:
  ctest -R rpc-test;
  Built debug and release builds;

Change-Id: I8efe68be0ae7f9ab05937d5c81fb53a709a881f1
Reviewed-on: http://gerrit.cloudera.org:8080/8273
Reviewed-by: Todd Lipcon <to...@apache.org>
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/83b69315
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/83b69315
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/83b69315

Branch: refs/heads/master
Commit: 83b693150d0c5772ae52cf9bc2bad58a46c25a92
Parents: eb5c7e2
Author: Michael Ho <kw...@cloudera.com>
Authored: Thu Oct 12 16:36:27 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Sat Oct 14 00:46:29 2017 +0000

----------------------------------------------------------------------
 src/kudu/rpc/rpcz_store.cc | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/83b69315/src/kudu/rpc/rpcz_store.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpcz_store.cc b/src/kudu/rpc/rpcz_store.cc
index ffbbf2f..8c8c04a 100644
--- a/src/kudu/rpc/rpcz_store.cc
+++ b/src/kudu/rpc/rpcz_store.cc
@@ -159,7 +159,7 @@ void MethodSampler::SampleCall(InboundCall* call) {
       std::swap(bucket->sample, new_sample);
       bucket->last_sample_time.Store(now);
     }
-    VLOG(1) << "Sampled call " << call->ToString();
+    VLOG(2) << "Sampled call " << call->ToString();
   }
 }