You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2016/10/20 18:10:24 UTC

kudu git commit: KUDU-1692: tie various reader memtrackers to the tablet tracker

Repository: kudu
Updated Branches:
  refs/heads/master 676c0d0cf -> 6bc075385


KUDU-1692: tie various reader memtrackers to the tablet tracker

Without the explicit connection, these memtrackers are all parented to the
root tracker, whose Release() behavior is rather expensive.

There's also some refactoring here:
- It was frustrating to pass the tablet's main and DMS memtrackers into
  various places, so there's a TabletMemTrackers struct to encapsulate them.
- There's some pass-by-value and std::move() for objects that are either
  large or may be large one day (e.g. ReaderOptions).
- CFileSet, MemRowSet and DeltaMemStore now follow the object construction
  idiom that we use elsewhere: a static method that creates the object and
  may return failure.

Note: this was originally https://gerrit.sjc.cloudera.com/#/c/7029. It's
since been rebased and all of the additional tracking was dropped.

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

Branch: refs/heads/master
Commit: 6bc075385f71784cca09120ed9ca3d2c34d7b4d7
Parents: 676c0d0
Author: Adar Dembo <ad...@cloudera.com>
Authored: Wed Oct 12 17:59:13 2016 -0700
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Thu Oct 20 18:09:48 2016 +0000

----------------------------------------------------------------------
 src/kudu/cfile/bloomfile.cc                 | 16 ++---
 src/kudu/cfile/bloomfile.h                  |  6 +-
 src/kudu/cfile/cfile_reader.cc              | 17 +++---
 src/kudu/cfile/cfile_reader.h               | 19 ++++--
 src/kudu/tablet/cfile_set-test.cc           | 17 +++---
 src/kudu/tablet/cfile_set.cc                | 75 +++++++++++++-----------
 src/kudu/tablet/cfile_set.h                 | 15 ++---
 src/kudu/tablet/compaction-test.cc          | 64 +++++++++++++++-----
 src/kudu/tablet/delta_compaction-test.cc    | 10 ++--
 src/kudu/tablet/delta_compaction.h          |  4 --
 src/kudu/tablet/delta_tracker.cc            | 62 +++++++++++++++-----
 src/kudu/tablet/delta_tracker.h             | 31 +++++-----
 src/kudu/tablet/deltafile-test.cc           | 17 +++---
 src/kudu/tablet/deltafile.cc                | 35 +++++------
 src/kudu/tablet/deltafile.h                 | 22 ++++---
 src/kudu/tablet/deltamemstore-test.cc       |  6 +-
 src/kudu/tablet/deltamemstore.cc            | 49 +++++++++-------
 src/kudu/tablet/deltamemstore.h             | 27 +++++----
 src/kudu/tablet/diskrowset-test-base.h      | 14 +++--
 src/kudu/tablet/diskrowset.cc               | 34 ++++++-----
 src/kudu/tablet/diskrowset.h                | 11 ++--
 src/kudu/tablet/memrowset-test.cc           | 36 +++++++++---
 src/kudu/tablet/memrowset.cc                | 25 ++++++--
 src/kudu/tablet/memrowset.h                 | 28 +++++----
 src/kudu/tablet/tablet-test.cc              |  4 +-
 src/kudu/tablet/tablet.cc                   | 40 ++++++++-----
 src/kudu/tablet/tablet.h                    | 17 +++---
 src/kudu/tablet/tablet_mem_trackers.h       | 52 ++++++++++++++++
 src/kudu/tools/tool_action_local_replica.cc | 11 ++--
 29 files changed, 488 insertions(+), 276 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/cfile/bloomfile.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/bloomfile.cc b/src/kudu/cfile/bloomfile.cc
index f7e59e7..c4822f2 100644
--- a/src/kudu/cfile/bloomfile.cc
+++ b/src/kudu/cfile/bloomfile.cc
@@ -139,10 +139,11 @@ Status BloomFileWriter::FinishCurrentBloomBlock() {
 ////////////////////////////////////////////////////////////
 
 Status BloomFileReader::Open(gscoped_ptr<ReadableBlock> block,
-                             const ReaderOptions& options,
+                             ReaderOptions options,
                              gscoped_ptr<BloomFileReader> *reader) {
   gscoped_ptr<BloomFileReader> bf_reader;
-  RETURN_NOT_OK(OpenNoInit(std::move(block), options, &bf_reader));
+  RETURN_NOT_OK(OpenNoInit(std::move(block),
+                           std::move(options), &bf_reader));
   RETURN_NOT_OK(bf_reader->Init());
 
   *reader = std::move(bf_reader);
@@ -150,12 +151,13 @@ Status BloomFileReader::Open(gscoped_ptr<ReadableBlock> block,
 }
 
 Status BloomFileReader::OpenNoInit(gscoped_ptr<ReadableBlock> block,
-                                   const ReaderOptions& options,
+                                   ReaderOptions options,
                                    gscoped_ptr<BloomFileReader> *reader) {
   gscoped_ptr<CFileReader> cf_reader;
-  RETURN_NOT_OK(CFileReader::OpenNoInit(std::move(block), options, &cf_reader));
+  RETURN_NOT_OK(CFileReader::OpenNoInit(std::move(block),
+                                        options, &cf_reader));
   gscoped_ptr<BloomFileReader> bf_reader(new BloomFileReader(
-      std::move(cf_reader), options));
+      std::move(cf_reader), std::move(options)));
   if (!FLAGS_cfile_lazy_open) {
     RETURN_NOT_OK(bf_reader->Init());
   }
@@ -165,9 +167,9 @@ Status BloomFileReader::OpenNoInit(gscoped_ptr<ReadableBlock> block,
 }
 
 BloomFileReader::BloomFileReader(gscoped_ptr<CFileReader> reader,
-                                 const ReaderOptions& options)
+                                 ReaderOptions options)
   : reader_(std::move(reader)),
-    mem_consumption_(options.parent_mem_tracker,
+    mem_consumption_(std::move(options.parent_mem_tracker),
                      memory_footprint_excluding_reader()) {
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/cfile/bloomfile.h
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/bloomfile.h b/src/kudu/cfile/bloomfile.h
index bd94450..1394ced 100644
--- a/src/kudu/cfile/bloomfile.h
+++ b/src/kudu/cfile/bloomfile.h
@@ -78,7 +78,7 @@ class BloomFileReader {
   //
   // After this call, the bloom reader is safe for use.
   static Status Open(gscoped_ptr<fs::ReadableBlock> block,
-                     const ReaderOptions& options,
+                     ReaderOptions options,
                      gscoped_ptr<BloomFileReader> *reader);
 
   // Lazily opens a bloom file using a previously opened block. A lazy open
@@ -87,7 +87,7 @@ class BloomFileReader {
   //
   // Init() must be called before using CheckKeyPresent().
   static Status OpenNoInit(gscoped_ptr<fs::ReadableBlock> block,
-                           const ReaderOptions& options,
+                           ReaderOptions options,
                            gscoped_ptr<BloomFileReader> *reader);
 
   // Fully opens a previously lazily opened bloom file, parsing and
@@ -106,7 +106,7 @@ class BloomFileReader {
  private:
   DISALLOW_COPY_AND_ASSIGN(BloomFileReader);
 
-  BloomFileReader(gscoped_ptr<CFileReader> reader, const ReaderOptions& options);
+  BloomFileReader(gscoped_ptr<CFileReader> reader, ReaderOptions options);
 
   // Parse the header present in the given block.
   //

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/cfile/cfile_reader.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile_reader.cc b/src/kudu/cfile/cfile_reader.cc
index 8b2d42c..ff57931 100644
--- a/src/kudu/cfile/cfile_reader.cc
+++ b/src/kudu/cfile/cfile_reader.cc
@@ -81,19 +81,22 @@ static Status ParseMagicAndLength(const Slice &data,
   return Status::OK();
 }
 
-CFileReader::CFileReader(const ReaderOptions &options,
-                         const uint64_t file_size,
+CFileReader::CFileReader(ReaderOptions options,
+                         uint64_t file_size,
                          gscoped_ptr<ReadableBlock> block) :
   block_(std::move(block)),
   file_size_(file_size),
-  mem_consumption_(options.parent_mem_tracker, memory_footprint()) {
+  mem_consumption_(std::move(options.parent_mem_tracker),
+                   memory_footprint()) {
 }
 
 Status CFileReader::Open(gscoped_ptr<ReadableBlock> block,
-                         const ReaderOptions& options,
+                         ReaderOptions options,
                          gscoped_ptr<CFileReader> *reader) {
   gscoped_ptr<CFileReader> reader_local;
-  RETURN_NOT_OK(OpenNoInit(std::move(block), options, &reader_local));
+  RETURN_NOT_OK(OpenNoInit(std::move(block),
+                           std::move(options),
+                           &reader_local));
   RETURN_NOT_OK(reader_local->Init());
 
   reader->reset(reader_local.release());
@@ -101,12 +104,12 @@ Status CFileReader::Open(gscoped_ptr<ReadableBlock> block,
 }
 
 Status CFileReader::OpenNoInit(gscoped_ptr<ReadableBlock> block,
-                               const ReaderOptions& options,
+                               ReaderOptions options,
                                gscoped_ptr<CFileReader> *reader) {
   uint64_t block_size;
   RETURN_NOT_OK(block->Size(&block_size));
   gscoped_ptr<CFileReader> reader_local(
-      new CFileReader(options, block_size, std::move(block)));
+      new CFileReader(std::move(options), block_size, std::move(block)));
   if (!FLAGS_cfile_lazy_open) {
     RETURN_NOT_OK(reader_local->Init());
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/cfile/cfile_reader.h
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile_reader.h b/src/kudu/cfile/cfile_reader.h
index 40106d3..f11b4e2 100644
--- a/src/kudu/cfile/cfile_reader.h
+++ b/src/kudu/cfile/cfile_reader.h
@@ -60,17 +60,17 @@ class CFileReader {
   //
   // After this call, the reader is safe for use.
   static Status Open(gscoped_ptr<fs::ReadableBlock> block,
-                     const ReaderOptions& options,
+                     ReaderOptions options,
                      gscoped_ptr<CFileReader>* reader);
 
   // Lazily open a cfile using a previously opened block. A lazy open does
   // not incur additional I/O, nor does it validate the contents of the
   // cfile.
   //
-  // Init() must be called before using most methods. Exceptions include
-  // NewIterator() and file_size().
+  // Init() must be called before most methods; the exceptions are documented
+  // below.
   static Status OpenNoInit(gscoped_ptr<fs::ReadableBlock> block,
-                           const ReaderOptions& options,
+                           ReaderOptions options,
                            gscoped_ptr<CFileReader>* reader);
 
   // Fully opens a previously lazily opened cfile, parsing and validating
@@ -84,6 +84,7 @@ class CFileReader {
     DONT_CACHE_BLOCK
   };
 
+  // Can be called before Init().
   Status NewIterator(CFileIterator **iter, CacheControl cache_control);
   Status NewIterator(gscoped_ptr<CFileIterator> *iter,
                      CacheControl cache_control) {
@@ -115,6 +116,11 @@ class CFileReader {
     return file_size_;
   }
 
+  // Can be called before Init().
+  const BlockId& block_id() const {
+    return block_->id();
+  }
+
   const TypeInfo *type_info() const {
     DCHECK(init_once_.initted());
     return type_info_;
@@ -161,13 +167,14 @@ class CFileReader {
     return BlockPointer(footer().validx_info().root_block());
   }
 
+  // Can be called before Init().
   std::string ToString() const { return block_->id().ToString(); }
 
  private:
   DISALLOW_COPY_AND_ASSIGN(CFileReader);
 
-  CFileReader(const ReaderOptions &options,
-              const uint64_t file_size,
+  CFileReader(ReaderOptions options,
+              uint64_t file_size,
               gscoped_ptr<fs::ReadableBlock> block);
 
   // Callback used in 'init_once_' to initialize this cfile.

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/cfile_set-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/cfile_set-test.cc b/src/kudu/tablet/cfile_set-test.cc
index c66593b..2e7b77a 100644
--- a/src/kudu/tablet/cfile_set-test.cc
+++ b/src/kudu/tablet/cfile_set-test.cc
@@ -23,6 +23,7 @@
 #include "kudu/tablet/cfile_set.h"
 #include "kudu/tablet/diskrowset-test-base.h"
 #include "kudu/tablet/tablet-test-base.h"
+#include "kudu/util/mem_tracker.h"
 #include "kudu/util/test_util.h"
 
 DECLARE_int32(cfile_default_block_size);
@@ -131,8 +132,8 @@ TEST_F(TestCFileSet, TestPartiallyMaterialize) {
   const int kNumRows = 100000;
   WriteTestRowSet(kNumRows);
 
-  shared_ptr<CFileSet> fileset(new CFileSet(rowset_meta_));
-  ASSERT_OK(fileset->Open());
+  shared_ptr<CFileSet> fileset;
+  ASSERT_OK(CFileSet::Open(rowset_meta_, MemTracker::GetRootTracker(), &fileset));
 
   gscoped_ptr<CFileSet::Iterator> iter(fileset->NewIterator(&schema_));
   ASSERT_OK(iter->Init(nullptr));
@@ -211,8 +212,8 @@ TEST_F(TestCFileSet, TestIteratePartialSchema) {
   const int kNumRows = 100;
   WriteTestRowSet(kNumRows);
 
-  shared_ptr<CFileSet> fileset(new CFileSet(rowset_meta_));
-  ASSERT_OK(fileset->Open());
+  shared_ptr<CFileSet> fileset;
+  ASSERT_OK(CFileSet::Open(rowset_meta_, MemTracker::GetRootTracker(), &fileset));
 
   Schema new_schema;
   ASSERT_OK(schema_.CreateProjectionByNames({ "c0", "c2" }, &new_schema));
@@ -244,8 +245,8 @@ TEST_F(TestCFileSet, TestRangeScan) {
   const int kNumRows = 10000;
   WriteTestRowSet(kNumRows);
 
-  shared_ptr<CFileSet> fileset(new CFileSet(rowset_meta_));
-  ASSERT_OK(fileset->Open());
+  shared_ptr<CFileSet> fileset;
+  ASSERT_OK(CFileSet::Open(rowset_meta_, MemTracker::GetRootTracker(), &fileset));
 
   // Create iterator.
   shared_ptr<CFileSet::Iterator> cfile_iter(fileset->NewIterator(&schema_));
@@ -296,8 +297,8 @@ TEST_F(TestCFileSet, TestRangePredicates2) {
   const int kNumRows = 10000;
   WriteTestRowSet(kNumRows);
 
-  shared_ptr<CFileSet> fileset(new CFileSet(rowset_meta_));
-  ASSERT_OK(fileset->Open());
+  shared_ptr<CFileSet> fileset;
+  ASSERT_OK(CFileSet::Open(rowset_meta_, MemTracker::GetRootTracker(), &fileset));
 
   // Range scan where rows match on both ends
   DoTestRangeScan(fileset, 2000, 2010);

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/cfile_set.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/cfile_set.cc b/src/kudu/tablet/cfile_set.cc
index 0719da6..f61556b 100644
--- a/src/kudu/tablet/cfile_set.cc
+++ b/src/kudu/tablet/cfile_set.cc
@@ -16,6 +16,8 @@
 // under the License.
 
 #include <algorithm>
+#include <memory>
+
 #include <gflags/gflags.h>
 #include <glog/logging.h>
 
@@ -49,31 +51,45 @@ using strings::Substitute;
 // Utilities
 ////////////////////////////////////////////////////////////
 
-static Status OpenReader(const shared_ptr<RowSetMetadata>& rowset_metadata,
-                         ColumnId col_id,
+static Status OpenReader(FsManager* fs,
+                         shared_ptr<MemTracker> parent_mem_tracker,
+                         const BlockId& block_id,
                          gscoped_ptr<CFileReader> *new_reader) {
-  FsManager* fs = rowset_metadata->fs_manager();
   gscoped_ptr<ReadableBlock> block;
-  BlockId block_id = rowset_metadata->column_data_block_for_col_id(col_id);
   RETURN_NOT_OK(fs->OpenBlock(block_id, &block));
 
-  // TODO: somehow pass reader options in schema
   ReaderOptions opts;
-  return CFileReader::OpenNoInit(std::move(block), opts, new_reader);
+  opts.parent_mem_tracker = std::move(parent_mem_tracker);
+  return CFileReader::OpenNoInit(std::move(block),
+                                 std::move(opts),
+                                 new_reader);
 }
 
 ////////////////////////////////////////////////////////////
 // CFile Base
 ////////////////////////////////////////////////////////////
 
-CFileSet::CFileSet(shared_ptr<RowSetMetadata> rowset_metadata)
-    : rowset_metadata_(std::move(rowset_metadata)) {}
+CFileSet::CFileSet(shared_ptr<RowSetMetadata> rowset_metadata,
+                   shared_ptr<MemTracker> parent_mem_tracker)
+    : rowset_metadata_(std::move(rowset_metadata)),
+      parent_mem_tracker_(std::move(parent_mem_tracker)) {
+}
 
 CFileSet::~CFileSet() {
 }
 
+Status CFileSet::Open(shared_ptr<RowSetMetadata> rowset_metadata,
+                      shared_ptr<MemTracker> parent_mem_tracker,
+                      shared_ptr<CFileSet>* cfile_set) {
+  shared_ptr<CFileSet> cfs(new CFileSet(std::move(rowset_metadata),
+                                        std::move(parent_mem_tracker)));
+  RETURN_NOT_OK(cfs->DoOpen());
+
+  cfile_set->swap(cfs);
+  return Status::OK();
+}
 
-Status CFileSet::Open() {
+Status CFileSet::DoOpen() {
   RETURN_NOT_OK(OpenBloomReader());
 
   // Lazily open the column data cfiles. Each one will be fully opened
@@ -84,51 +100,42 @@ Status CFileSet::Open() {
     DCHECK(!ContainsKey(readers_by_col_id_, col_id)) << "already open";
 
     gscoped_ptr<CFileReader> reader;
-    RETURN_NOT_OK(OpenReader(rowset_metadata_, col_id, &reader));
+    RETURN_NOT_OK(OpenReader(rowset_metadata_->fs_manager(),
+                             parent_mem_tracker_,
+                             rowset_metadata_->column_data_block_for_col_id(col_id),
+                             &reader));
     readers_by_col_id_[col_id] = shared_ptr<CFileReader>(reader.release());
     VLOG(1) << "Successfully opened cfile for column id " << col_id
             << " in " << rowset_metadata_->ToString();
   }
 
-  // However, the key reader should always be fully opened, so that we
-  // can figure out where in the rowset tree we belong.
   if (rowset_metadata_->has_adhoc_index_block()) {
-    RETURN_NOT_OK(OpenAdHocIndexReader());
-  } else {
-    RETURN_NOT_OK(key_index_reader()->Init());
+    RETURN_NOT_OK(OpenReader(rowset_metadata_->fs_manager(),
+                             parent_mem_tracker_,
+                             rowset_metadata_->adhoc_index_block(),
+                             &ad_hoc_idx_reader_));
   }
 
+  // However, the key reader should always be fully opened, so that we
+  // can figure out where in the rowset tree we belong.
+  RETURN_NOT_OK(key_index_reader()->Init());
+
   // Determine the upper and lower key bounds for this CFileSet.
   RETURN_NOT_OK(LoadMinMaxKeys());
 
   return Status::OK();
 }
 
-Status CFileSet::OpenAdHocIndexReader() {
-  if (ad_hoc_idx_reader_ != nullptr) {
-    return Status::OK();
-  }
-
-  FsManager* fs = rowset_metadata_->fs_manager();
-  gscoped_ptr<ReadableBlock> block;
-  RETURN_NOT_OK(fs->OpenBlock(rowset_metadata_->adhoc_index_block(), &block));
-
-  ReaderOptions opts;
-  return CFileReader::Open(std::move(block), opts, &ad_hoc_idx_reader_);
-}
-
-
 Status CFileSet::OpenBloomReader() {
-  if (bloom_reader_ != nullptr) {
-    return Status::OK();
-  }
-
   FsManager* fs = rowset_metadata_->fs_manager();
   gscoped_ptr<ReadableBlock> block;
   RETURN_NOT_OK(fs->OpenBlock(rowset_metadata_->bloom_block(), &block));
 
   ReaderOptions opts;
-  Status s = BloomFileReader::OpenNoInit(std::move(block), opts, &bloom_reader_);
+  opts.parent_mem_tracker = parent_mem_tracker_;
+  Status s = BloomFileReader::OpenNoInit(std::move(block),
+                                         std::move(opts),
+                                         &bloom_reader_);
   if (!s.ok()) {
     LOG(WARNING) << "Unable to open bloom file in " << rowset_metadata_->ToString() << ": "
                  << s.ToString();

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/cfile_set.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/cfile_set.h b/src/kudu/tablet/cfile_set.h
index 7d46cf5..288d105 100644
--- a/src/kudu/tablet/cfile_set.h
+++ b/src/kudu/tablet/cfile_set.h
@@ -38,10 +38,6 @@
 
 namespace kudu {
 
-namespace metadata {
-class RowSetMetadata;
-}
-
 namespace tablet {
 
 using kudu::cfile::BloomFileReader;
@@ -57,9 +53,9 @@ class CFileSet : public std::enable_shared_from_this<CFileSet> {
  public:
   class Iterator;
 
-  explicit CFileSet(std::shared_ptr<RowSetMetadata> rowset_metadata);
-
-  Status Open();
+  static Status Open(std::shared_ptr<RowSetMetadata> rowset_metadata,
+                     std::shared_ptr<MemTracker> parent_mem_tracker,
+                     std::shared_ptr<CFileSet>* cfile_set);
 
   // Create an iterator with the given projection. 'projection' must remain valid
   // for the lifetime of the returned iterator.
@@ -98,6 +94,10 @@ class CFileSet : public std::enable_shared_from_this<CFileSet> {
 
   DISALLOW_COPY_AND_ASSIGN(CFileSet);
 
+  CFileSet(std::shared_ptr<RowSetMetadata> rowset_metadata,
+           std::shared_ptr<MemTracker> parent_mem_tracker);
+
+  Status DoOpen();
   Status OpenBloomReader();
   Status OpenAdHocIndexReader();
   Status LoadMinMaxKeys();
@@ -113,6 +113,7 @@ class CFileSet : public std::enable_shared_from_this<CFileSet> {
   const Schema &tablet_schema() const { return rowset_metadata_->tablet_schema(); }
 
   std::shared_ptr<RowSetMetadata> rowset_metadata_;
+  std::shared_ptr<MemTracker> parent_mem_tracker_;
 
   std::string min_encoded_key_;
   std::string max_encoded_key_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/compaction-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/compaction-test.cc b/src/kudu/tablet/compaction-test.cc
index 923f3ea..a558da2 100644
--- a/src/kudu/tablet/compaction-test.cc
+++ b/src/kudu/tablet/compaction-test.cc
@@ -16,10 +16,11 @@
 // under the License.
 
 #include <algorithm>
+#include <memory>
+
 #include <gflags/gflags.h>
 #include <glog/logging.h>
 #include <gtest/gtest.h>
-#include <memory>
 
 #include "kudu/common/partial_row.h"
 #include "kudu/consensus/log_anchor_registry.h"
@@ -32,6 +33,7 @@
 #include "kudu/tablet/compaction.h"
 #include "kudu/tablet/local_tablet_writer.h"
 #include "kudu/tablet/tablet-test-util.h"
+#include "kudu/tablet/tablet_mem_trackers.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/test_util.h"
 
@@ -221,7 +223,8 @@ class TestCompaction : public KuduRowSetTest {
       // Re-open the outputs
       for (const shared_ptr<RowSetMetadata>& meta : metas) {
         shared_ptr<DiskRowSet> rs;
-        ASSERT_OK(DiskRowSet::Open(meta, log_anchor_registry_.get(), &rs));
+        ASSERT_OK(DiskRowSet::Open(meta, log_anchor_registry_.get(),
+                                   mem_trackers_, &rs));
         result_rowsets->push_back(rs);
       }
     }
@@ -296,7 +299,9 @@ class TestCompaction : public KuduRowSetTest {
     int delta = 0;
     for (const Schema& schema : schemas) {
       // Create a memrowset with a bunch of rows and updates.
-      shared_ptr<MemRowSet> mrs(new MemRowSet(delta, schema, log_anchor_registry_.get()));
+      shared_ptr<MemRowSet> mrs;
+      CHECK_OK(MemRowSet::Create(delta, schema, log_anchor_registry_.get(),
+                                 mem_trackers_.tablet_tracker, &mrs));
       InsertRows(mrs.get(), 1000, delta);
       UpdateRows(mrs.get(), 1000, delta, 1);
 
@@ -330,7 +335,9 @@ class TestCompaction : public KuduRowSetTest {
       // Create inputs.
       for (int i = 0; i < FLAGS_merge_benchmark_num_rowsets; i++) {
         // Create a memrowset with a bunch of rows and updates.
-        shared_ptr<MemRowSet> mrs(new MemRowSet(i, schema_, log_anchor_registry_.get()));
+        shared_ptr<MemRowSet> mrs;
+        CHECK_OK(MemRowSet::Create(i, schema_, log_anchor_registry_.get(),
+                                   mem_trackers_.tablet_tracker, &mrs));
 
         for (int n = 0; n < FLAGS_merge_benchmark_num_rows_per_rowset; n++) {
 
@@ -361,7 +368,8 @@ class TestCompaction : public KuduRowSetTest {
 
       for (const shared_ptr<RowSetMetadata>& meta : input_meta->rowsets()) {
         shared_ptr<DiskRowSet> rs;
-        CHECK_OK(DiskRowSet::Open(meta, log_anchor_registry_.get(), &rs));
+        CHECK_OK(DiskRowSet::Open(meta, log_anchor_registry_.get(),
+                                  mem_trackers_, &rs));
         rowsets.push_back(rs);
       }
 
@@ -392,11 +400,15 @@ class TestCompaction : public KuduRowSetTest {
   MvccManager mvcc_;
 
   scoped_refptr<LogAnchorRegistry> log_anchor_registry_;
+
+  TabletMemTrackers mem_trackers_;
 };
 
 TEST_F(TestCompaction, TestMemRowSetInput) {
   // Create a memrowset with 10 rows and several updates.
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              mem_trackers_.tablet_tracker, &mrs));
   InsertRows(mrs.get(), 10, 0);
   UpdateRows(mrs.get(), 10, 0, 1);
   UpdateRows(mrs.get(), 10, 0, 2);
@@ -421,7 +433,9 @@ TEST_F(TestCompaction, TestMemRowSetInput) {
 TEST_F(TestCompaction, TestFlushMRSWithRolling) {
   // Create a memrowset with enough rows so that, when we flush with a small
   // roll threshold, we'll end up creating multiple DiskRowSets.
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              mem_trackers_.tablet_tracker, &mrs));
   InsertRows(mrs.get(), 30000, 0);
 
   vector<shared_ptr<DiskRowSet> > rowsets;
@@ -449,7 +463,9 @@ TEST_F(TestCompaction, TestRowSetInput) {
   // Create a memrowset with a bunch of rows, flush and reopen.
   shared_ptr<DiskRowSet> rs;
   {
-    shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+    shared_ptr<MemRowSet> mrs;
+    ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                                mem_trackers_.tablet_tracker, &mrs));
     InsertRows(mrs.get(), 10, 0);
     FlushMRSAndReopenNoRoll(*mrs, schema_, &rs);
     ASSERT_NO_FATAL_FAILURE();
@@ -492,7 +508,9 @@ TEST_F(TestCompaction, TestRowSetInput) {
 TEST_F(TestCompaction, TestDuplicatedGhostRowsDontSurviveCompaction) {
   shared_ptr<DiskRowSet> rs1;
   {
-    shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+    shared_ptr<MemRowSet> mrs;
+    ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                                mem_trackers_.tablet_tracker, &mrs));
     InsertRows(mrs.get(), 10, 0);
     FlushMRSAndReopenNoRoll(*mrs, schema_, &rs1);
     ASSERT_NO_FATAL_FAILURE();
@@ -503,7 +521,9 @@ TEST_F(TestCompaction, TestDuplicatedGhostRowsDontSurviveCompaction) {
 
   shared_ptr<DiskRowSet> rs2;
   {
-    shared_ptr<MemRowSet> mrs(new MemRowSet(1, schema_, log_anchor_registry_.get()));
+    shared_ptr<MemRowSet> mrs;
+    ASSERT_OK(MemRowSet::Create(1, schema_, log_anchor_registry_.get(),
+                                mem_trackers_.tablet_tracker, &mrs));
     InsertRows(mrs.get(), 10, 0);
     UpdateRows(mrs.get(), 10, 0, 1);
     FlushMRSAndReopenNoRoll(*mrs, schema_, &rs2);
@@ -513,7 +533,9 @@ TEST_F(TestCompaction, TestDuplicatedGhostRowsDontSurviveCompaction) {
 
   shared_ptr<DiskRowSet> rs3;
   {
-    shared_ptr<MemRowSet> mrs(new MemRowSet(1, schema_, log_anchor_registry_.get()));
+    shared_ptr<MemRowSet> mrs;
+    ASSERT_OK(MemRowSet::Create(1, schema_, log_anchor_registry_.get(),
+                                mem_trackers_.tablet_tracker, &mrs));
     InsertRows(mrs.get(), 10, 0);
     UpdateRows(mrs.get(), 10, 0, 2);
     FlushMRSAndReopenNoRoll(*mrs, schema_, &rs3);
@@ -554,7 +576,9 @@ TEST_F(TestCompaction, TestDuplicatedGhostRowsDontSurviveCompaction) {
 // output rowset (on disk).
 TEST_F(TestCompaction, TestOneToOne) {
   // Create a memrowset with a bunch of rows and updates.
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              mem_trackers_.tablet_tracker, &mrs));
   InsertRows(mrs.get(), 1000, 0);
   UpdateRows(mrs.get(), 1000, 0, 1);
   MvccSnapshot snap(mvcc_);
@@ -601,13 +625,17 @@ TEST_F(TestCompaction, TestOneToOne) {
 // output of a compaction, and trying to merge two MRS.
 TEST_F(TestCompaction, TestKUDU102) {
   // Create 2 row sets, flush them
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              mem_trackers_.tablet_tracker, &mrs));
   InsertRows(mrs.get(), 10, 0);
   shared_ptr<DiskRowSet> rs;
   FlushMRSAndReopenNoRoll(*mrs, schema_, &rs);
   ASSERT_NO_FATAL_FAILURE();
 
-  shared_ptr<MemRowSet> mrs_b(new MemRowSet(1, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs_b;
+  ASSERT_OK(MemRowSet::Create(1, schema_, log_anchor_registry_.get(),
+                              mem_trackers_.tablet_tracker, &mrs_b));
   InsertRows(mrs_b.get(), 10, 100);
   MvccSnapshot snap(mvcc_);
   shared_ptr<DiskRowSet> rs_b;
@@ -666,10 +694,14 @@ TEST_F(TestCompaction, TestMergeMultipleSchemas) {
 // used (we never compact in-memory), but this is a regression test for a bug
 // encountered during development where the first row of each MRS got dropped.
 TEST_F(TestCompaction, TestMergeMRS) {
-  shared_ptr<MemRowSet> mrs_a(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs_a;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              mem_trackers_.tablet_tracker, &mrs_a));
   InsertRows(mrs_a.get(), 10, 0);
 
-  shared_ptr<MemRowSet> mrs_b(new MemRowSet(1, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs_b;
+  ASSERT_OK(MemRowSet::Create(1, schema_, log_anchor_registry_.get(),
+                              mem_trackers_.tablet_tracker, &mrs_b));
   InsertRows(mrs_b.get(), 10, 1);
 
   MvccSnapshot snap(mvcc_);

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/delta_compaction-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/delta_compaction-test.cc b/src/kudu/tablet/delta_compaction-test.cc
index 0f1dca1..78721c0 100644
--- a/src/kudu/tablet/delta_compaction-test.cc
+++ b/src/kudu/tablet/delta_compaction-test.cc
@@ -15,13 +15,14 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <string>
+#include <vector>
+
 #include <gflags/gflags.h>
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
-#include <string>
-#include <vector>
-
+#include "kudu/cfile/cfile_util.h"
 #include "kudu/common/schema.h"
 #include "kudu/tablet/deltafile.h"
 #include "kudu/tablet/delta_compaction.h"
@@ -48,6 +49,7 @@ using std::vector;
 namespace kudu {
 namespace tablet {
 
+using cfile::ReaderOptions;
 using fs::ReadableBlock;
 using fs::WritableBlock;
 
@@ -79,7 +81,7 @@ class TestDeltaCompaction : public KuduTest {
     gscoped_ptr<ReadableBlock> block;
     RETURN_NOT_OK(fs_manager_->OpenBlock(block_id, &block));
     shared_ptr<DeltaFileReader> delta_reader;
-    return DeltaFileReader::Open(std::move(block), block_id, dfr, REDO);
+    return DeltaFileReader::Open(std::move(block), REDO, ReaderOptions(), dfr);
   }
 
   virtual void SetUp() OVERRIDE {

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/delta_compaction.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/delta_compaction.h b/src/kudu/tablet/delta_compaction.h
index 85760bb..5f99f0c 100644
--- a/src/kudu/tablet/delta_compaction.h
+++ b/src/kudu/tablet/delta_compaction.h
@@ -29,10 +29,6 @@
 
 namespace kudu {
 
-namespace metadata {
-class RowSetMetadata;
-} // namespace metadata
-
 namespace tablet {
 
 class CFileSet;

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/delta_tracker.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/delta_tracker.cc b/src/kudu/tablet/delta_tracker.cc
index 04d6292..ea09750 100644
--- a/src/kudu/tablet/delta_tracker.cc
+++ b/src/kudu/tablet/delta_tracker.cc
@@ -35,24 +35,39 @@
 namespace kudu {
 namespace tablet {
 
+using cfile::ReaderOptions;
 using fs::ReadableBlock;
 using fs::WritableBlock;
+using log::LogAnchorRegistry;
 using std::shared_ptr;
 using std::string;
 using std::unique_ptr;
 using strings::Substitute;
 
+Status DeltaTracker::Open(const shared_ptr<RowSetMetadata>& rowset_metadata,
+                          rowid_t num_rows,
+                          LogAnchorRegistry* log_anchor_registry,
+                          const TabletMemTrackers& mem_trackers,
+                          gscoped_ptr<DeltaTracker>* delta_tracker) {
+  gscoped_ptr<DeltaTracker> local_dt(
+      new DeltaTracker(rowset_metadata, num_rows, log_anchor_registry,
+                       mem_trackers));
+  RETURN_NOT_OK(local_dt->DoOpen());
+
+  delta_tracker->swap(local_dt);
+  return Status::OK();
+}
+
 DeltaTracker::DeltaTracker(shared_ptr<RowSetMetadata> rowset_metadata,
                            rowid_t num_rows,
-                           log::LogAnchorRegistry* log_anchor_registry,
-                           shared_ptr<MemTracker> parent_tracker)
+                           LogAnchorRegistry* log_anchor_registry,
+                           const TabletMemTrackers& mem_trackers)
     : rowset_metadata_(std::move(rowset_metadata)),
       num_rows_(num_rows),
       open_(false),
       log_anchor_registry_(log_anchor_registry),
-      parent_tracker_(std::move(parent_tracker)),
-      dms_empty_(true) {
-}
+      mem_trackers_(mem_trackers),
+      dms_empty_(true) {}
 
 Status DeltaTracker::OpenDeltaReaders(const vector<BlockId>& blocks,
                                       vector<shared_ptr<DeltaStore> >* stores,
@@ -69,7 +84,12 @@ Status DeltaTracker::OpenDeltaReaders(const vector<BlockId>& blocks,
     }
 
     shared_ptr<DeltaFileReader> dfr;
-    s = DeltaFileReader::OpenNoInit(std::move(block), block_id, &dfr, type);
+    ReaderOptions options;
+    options.parent_mem_tracker = mem_trackers_.tablet_tracker;
+    s = DeltaFileReader::OpenNoInit(std::move(block),
+                                    type,
+                                    std::move(options),
+                                    &dfr);
     if (!s.ok()) {
       LOG(ERROR) << "Failed to open " << DeltaType_Name(type)
                  << " delta file reader " << block_id.ToString() << ": "
@@ -86,7 +106,7 @@ Status DeltaTracker::OpenDeltaReaders(const vector<BlockId>& blocks,
 
 
 // Open any previously flushed DeltaFiles in this rowset
-Status DeltaTracker::Open() {
+Status DeltaTracker::DoOpen() {
   CHECK(redo_delta_stores_.empty()) << "should call before opening any readers";
   CHECK(undo_delta_stores_.empty()) << "should call before opening any readers";
   CHECK(!open_);
@@ -99,10 +119,13 @@ Status DeltaTracker::Open() {
                                  UNDO));
 
   // the id of the first DeltaMemStore is the max id of the current ones +1
-  dms_.reset(new DeltaMemStore(rowset_metadata_->last_durable_redo_dms_id() + 1,
-                               rowset_metadata_->id(),
-                               log_anchor_registry_,
-                               parent_tracker_));
+  RETURN_NOT_OK(DeltaMemStore::Create(rowset_metadata_->last_durable_redo_dms_id() + 1,
+                                      rowset_metadata_->id(),
+                                      log_anchor_registry_,
+                                      mem_trackers_.dms_tracker,
+                                      &dms_));
+  RETURN_NOT_OK(dms_->Init());
+
   open_ = true;
   return Status::OK();
 }
@@ -405,7 +428,12 @@ Status DeltaTracker::FlushDMS(DeltaMemStore* dms,
   // Now re-open for read
   gscoped_ptr<ReadableBlock> readable_block;
   RETURN_NOT_OK(fs->OpenBlock(block_id, &readable_block));
-  RETURN_NOT_OK(DeltaFileReader::OpenNoInit(std::move(readable_block), block_id, dfr, REDO));
+  ReaderOptions options;
+  options.parent_mem_tracker = mem_trackers_.tablet_tracker;
+  RETURN_NOT_OK(DeltaFileReader::OpenNoInit(std::move(readable_block),
+                                            REDO,
+                                            std::move(options),
+                                            dfr));
   LOG(INFO) << "Reopened delta block for read: " << block_id.ToString();
 
   RETURN_NOT_OK(rowset_metadata_->CommitRedoDeltaDataBlock(dms->id(), block_id));
@@ -434,8 +462,12 @@ Status DeltaTracker::Flush(MetadataFlushType flush_type) {
 
     // Swap the DeltaMemStore to use the new schema
     old_dms = dms_;
-    dms_.reset(new DeltaMemStore(old_dms->id() + 1, rowset_metadata_->id(),
-                                 log_anchor_registry_, parent_tracker_));
+    RETURN_NOT_OK(DeltaMemStore::Create(old_dms->id() + 1,
+                                        rowset_metadata_->id(),
+                                        log_anchor_registry_,
+                                        mem_trackers_.dms_tracker,
+                                        &dms_));
+    RETURN_NOT_OK(dms_->Init());
     dms_empty_.Store(true);
 
     if (count == 0) {
@@ -480,7 +512,7 @@ Status DeltaTracker::Flush(MetadataFlushType flush_type) {
 
 size_t DeltaTracker::DeltaMemStoreSize() const {
   shared_lock<rw_spinlock> lock(component_lock_);
-  return dms_->memory_footprint();
+  return dms_->EstimateSize();
 }
 
 int64_t DeltaTracker::MinUnflushedLogIndex() const {

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/delta_tracker.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/delta_tracker.h b/src/kudu/tablet/delta_tracker.h
index f4be010..ba56d31 100644
--- a/src/kudu/tablet/delta_tracker.h
+++ b/src/kudu/tablet/delta_tracker.h
@@ -26,8 +26,9 @@
 #include "kudu/common/rowid.h"
 #include "kudu/consensus/metadata.pb.h"
 #include "kudu/gutil/macros.h"
-#include "kudu/tablet/delta_store.h"
 #include "kudu/tablet/cfile_set.h"
+#include "kudu/tablet/delta_store.h"
+#include "kudu/tablet/tablet_mem_trackers.h"
 #include "kudu/util/atomic.h"
 #include "kudu/util/status.h"
 
@@ -43,10 +44,6 @@ namespace log {
 class LogAnchorRegistry;
 }
 
-namespace metadata {
-class RowSetMetadata;
-}
-
 namespace tablet {
 
 class DeltaMemStore;
@@ -68,9 +65,11 @@ class DeltaTracker {
     NO_FLUSH_METADATA
   };
 
-  DeltaTracker(std::shared_ptr<RowSetMetadata> rowset_metadata,
-               rowid_t num_rows, log::LogAnchorRegistry* log_anchor_registry,
-               std::shared_ptr<MemTracker> parent_tracker);
+  static Status Open(const std::shared_ptr<RowSetMetadata>& rowset_metadata,
+                     rowid_t num_rows,
+                     log::LogAnchorRegistry* log_anchor_registry,
+                     const TabletMemTrackers& mem_trackers,
+                     gscoped_ptr<DeltaTracker>* delta_tracker);
 
   Status WrapIterator(const std::shared_ptr<CFileSet::Iterator> &base,
                       const MvccSnapshot &mvcc_snap,
@@ -113,8 +112,6 @@ class DeltaTracker {
     std::vector<std::shared_ptr<DeltaStore> >* included_stores,
     std::unique_ptr<DeltaIterator>* out) const;
 
-  Status Open();
-
   // Flushes the current DeltaMemStore and replaces it with a new one.
   // Caller selects whether to also have the RowSetMetadata (and consequently
   // the TabletMetadata) flushed.
@@ -189,16 +186,18 @@ class DeltaTracker {
   }
 
  private:
-  friend class DiskRowSet;
-
-  DISALLOW_COPY_AND_ASSIGN(DeltaTracker);
-
   FRIEND_TEST(TestRowSet, TestRowSetUpdate);
   FRIEND_TEST(TestRowSet, TestDMSFlush);
   FRIEND_TEST(TestRowSet, TestMakeDeltaIteratorMergerUnlocked);
   FRIEND_TEST(TestRowSet, TestCompactStores);
   FRIEND_TEST(TestMajorDeltaCompaction, TestCompact);
 
+  DeltaTracker(std::shared_ptr<RowSetMetadata> rowset_metadata,
+               rowid_t num_rows, log::LogAnchorRegistry* log_anchor_registry,
+               const TabletMemTrackers& mem_trackers);
+
+  Status DoOpen();
+
   Status OpenDeltaReaders(const std::vector<BlockId>& blocks,
                           std::vector<std::shared_ptr<DeltaStore> >* stores,
                           DeltaType type);
@@ -249,7 +248,7 @@ class DeltaTracker {
 
   log::LogAnchorRegistry* log_anchor_registry_;
 
-  std::shared_ptr<MemTracker> parent_tracker_;
+  TabletMemTrackers mem_trackers_;
 
   // The current DeltaMemStore into which updates should be written.
   std::shared_ptr<DeltaMemStore> dms_;
@@ -279,6 +278,8 @@ class DeltaTracker {
   //
   // TODO(perf): this needs to be more fine grained
   mutable Mutex compact_flush_lock_;
+
+  DISALLOW_COPY_AND_ASSIGN(DeltaTracker);
 };
 
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/deltafile-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/deltafile-test.cc b/src/kudu/tablet/deltafile-test.cc
index 87d6dd3..96938b0 100644
--- a/src/kudu/tablet/deltafile-test.cc
+++ b/src/kudu/tablet/deltafile-test.cc
@@ -15,16 +15,18 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <memory>
+
 #include <gflags/gflags.h>
 #include <gtest/gtest.h>
-#include <memory>
 
+#include "kudu/cfile/cfile_util.h"
 #include "kudu/common/schema.h"
 #include "kudu/fs/fs-test-util.h"
+#include "kudu/gutil/strings/strcat.h"
 #include "kudu/tablet/delta_store.h"
-#include "kudu/tablet/deltafile.h"
 #include "kudu/tablet/delta_tracker.h"
-#include "kudu/gutil/strings/strcat.h"
+#include "kudu/tablet/deltafile.h"
 #include "kudu/util/test_util.h"
 
 DECLARE_int32(deltafile_default_block_size);
@@ -40,6 +42,7 @@ using std::shared_ptr;
 namespace kudu {
 namespace tablet {
 
+using cfile::ReaderOptions;
 using fs::CountingReadableBlock;
 using fs::ReadableBlock;
 using fs::WritableBlock;
@@ -107,10 +110,9 @@ class TestDeltaFile : public KuduTest {
   Status OpenDeltaFileReader(const BlockId& block_id, shared_ptr<DeltaFileReader>* out) {
     gscoped_ptr<ReadableBlock> block;
     RETURN_NOT_OK(fs_manager_->OpenBlock(block_id, &block));
-    return DeltaFileReader::Open(std::move(block), block_id, out, REDO);
+    return DeltaFileReader::Open(std::move(block), REDO, ReaderOptions(), out);
   }
 
-  // TODO handle UNDO deltas
   Status OpenDeltaFileIterator(const BlockId& block_id, gscoped_ptr<DeltaIterator>* out) {
     shared_ptr<DeltaFileReader> reader;
     RETURN_NOT_OK(OpenDeltaFileReader(block_id, &reader));
@@ -342,7 +344,7 @@ TEST_F(TestDeltaFile, TestLazyInit) {
   // Lazily opening the delta file should not trigger any reads.
   shared_ptr<DeltaFileReader> reader;
   ASSERT_OK(DeltaFileReader::OpenNoInit(
-      std::move(count_block), test_block_, &reader, REDO));
+      std::move(count_block), REDO, ReaderOptions(), &reader));
   ASSERT_EQ(0, bytes_read);
 
   // But initializing it should (only the first time).
@@ -357,7 +359,8 @@ TEST_F(TestDeltaFile, TestLazyInit) {
   ASSERT_OK(fs_manager_->OpenBlock(test_block_, &block));
   bytes_read = 0;
   count_block.reset(new CountingReadableBlock(std::move(block), &bytes_read));
-  ASSERT_OK(DeltaFileReader::Open(std::move(count_block), test_block_, &reader, REDO));
+  ASSERT_OK(DeltaFileReader::Open(
+      std::move(count_block), REDO, ReaderOptions(), &reader));
   ASSERT_EQ(bytes_read_after_init, bytes_read);
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/deltafile.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/deltafile.cc b/src/kudu/tablet/deltafile.cc
index 08a8ffa..8f36392 100644
--- a/src/kudu/tablet/deltafile.cc
+++ b/src/kudu/tablet/deltafile.cc
@@ -48,11 +48,11 @@ using std::unique_ptr;
 
 namespace kudu {
 
-using cfile::BlockHandle;
-using cfile::BlockPointer;
-using cfile::IndexTreeIterator;
 using cfile::BinaryPlainBlockDecoder;
+using cfile::BlockPointer;
 using cfile::CFileReader;
+using cfile::IndexTreeIterator;
+using cfile::ReaderOptions;
 using fs::ReadableBlock;
 using fs::ScopedWritableBlockCloser;
 using fs::WritableBlock;
@@ -174,12 +174,14 @@ Status DeltaFileWriter::WriteDeltaStats(const DeltaStats& stats) {
 ////////////////////////////////////////////////////////////
 
 Status DeltaFileReader::Open(gscoped_ptr<ReadableBlock> block,
-                             const BlockId& block_id,
-                             shared_ptr<DeltaFileReader>* reader_out,
-                             DeltaType delta_type) {
+                             DeltaType delta_type,
+                             ReaderOptions options,
+                             shared_ptr<DeltaFileReader>* reader_out) {
   shared_ptr<DeltaFileReader> df_reader;
   RETURN_NOT_OK(DeltaFileReader::OpenNoInit(std::move(block),
-                                            block_id, &df_reader, delta_type));
+                                            delta_type,
+                                            std::move(options),
+                                            &df_reader));
   RETURN_NOT_OK(df_reader->Init());
 
   *reader_out = df_reader;
@@ -187,15 +189,15 @@ Status DeltaFileReader::Open(gscoped_ptr<ReadableBlock> block,
 }
 
 Status DeltaFileReader::OpenNoInit(gscoped_ptr<ReadableBlock> block,
-                                   const BlockId& block_id,
-                                   shared_ptr<DeltaFileReader>* reader_out,
-                                   DeltaType delta_type) {
+                                   DeltaType delta_type,
+                                   ReaderOptions options,
+                                   shared_ptr<DeltaFileReader>* reader_out) {
   gscoped_ptr<CFileReader> cf_reader;
   RETURN_NOT_OK(CFileReader::OpenNoInit(std::move(block),
-                                        cfile::ReaderOptions(), &cf_reader));
-  gscoped_ptr<DeltaFileReader> df_reader(new DeltaFileReader(block_id,
-                                                             cf_reader.release(),
-                                                             delta_type));
+                                        std::move(options),
+                                        &cf_reader));
+  gscoped_ptr<DeltaFileReader> df_reader(
+      new DeltaFileReader(std::move(cf_reader), delta_type));
   if (!FLAGS_cfile_lazy_open) {
     RETURN_NOT_OK(df_reader->Init());
   }
@@ -205,10 +207,9 @@ Status DeltaFileReader::OpenNoInit(gscoped_ptr<ReadableBlock> block,
   return Status::OK();
 }
 
-DeltaFileReader::DeltaFileReader(BlockId block_id, CFileReader *cf_reader,
+DeltaFileReader::DeltaFileReader(gscoped_ptr<CFileReader> cf_reader,
                                  DeltaType delta_type)
-    : reader_(cf_reader),
-      block_id_(std::move(block_id)),
+    : reader_(cf_reader.release()),
       delta_type_(delta_type) {}
 
 Status DeltaFileReader::Init() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/deltafile.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/deltafile.h b/src/kudu/tablet/deltafile.h
index a7091e0..b7951ff 100644
--- a/src/kudu/tablet/deltafile.h
+++ b/src/kudu/tablet/deltafile.h
@@ -110,20 +110,20 @@ class DeltaFileReader : public DeltaStore,
   // Fully open a delta file using a previously opened block.
   //
   // After this call, the delta reader is safe for use.
-  static Status Open(gscoped_ptr<fs::ReadableBlock> file,
-                     const BlockId& block_id,
-                     std::shared_ptr<DeltaFileReader>* reader_out,
-                     DeltaType delta_type);
+  static Status Open(gscoped_ptr<fs::ReadableBlock> block,
+                     DeltaType delta_type,
+                     cfile::ReaderOptions options,
+                     std::shared_ptr<DeltaFileReader>* reader_out);
 
   // Lazily opens a delta file using a previously opened block. A lazy open
   // does not incur additional I/O, nor does it validate the contents of
   // the delta file.
   //
   // Init() must be called before using the file's stats.
-  static Status OpenNoInit(gscoped_ptr<fs::ReadableBlock> file,
-                           const BlockId& block_id,
-                           std::shared_ptr<DeltaFileReader>* reader_out,
-                           DeltaType delta_type);
+  static Status OpenNoInit(gscoped_ptr<fs::ReadableBlock> block,
+                           DeltaType delta_type,
+                           cfile::ReaderOptions options,
+                           std::shared_ptr<DeltaFileReader>* reader_out);
 
   virtual Status Init() OVERRIDE;
 
@@ -141,7 +141,7 @@ class DeltaFileReader : public DeltaStore,
 
   virtual uint64_t EstimateSize() const OVERRIDE;
 
-  const BlockId& block_id() const { return block_id_; }
+  const BlockId& block_id() const { return reader_->block_id(); }
 
   virtual const DeltaStats& delta_stats() const OVERRIDE {
     DCHECK(init_once_.initted());
@@ -166,7 +166,7 @@ class DeltaFileReader : public DeltaStore,
     return reader_;
   }
 
-  DeltaFileReader(BlockId block_id, cfile::CFileReader *cf_reader,
+  DeltaFileReader(gscoped_ptr<cfile::CFileReader> cf_reader,
                   DeltaType delta_type);
 
   // Callback used in 'init_once_' to initialize this delta file.
@@ -177,8 +177,6 @@ class DeltaFileReader : public DeltaStore,
   std::shared_ptr<cfile::CFileReader> reader_;
   gscoped_ptr<DeltaStats> delta_stats_;
 
-  const BlockId block_id_;
-
   // The type of this delta, i.e. UNDO or REDO.
   const DeltaType delta_type_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/deltamemstore-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/deltamemstore-test.cc b/src/kudu/tablet/deltamemstore-test.cc
index 4bcc131..65e78f6 100644
--- a/src/kudu/tablet/deltamemstore-test.cc
+++ b/src/kudu/tablet/deltamemstore-test.cc
@@ -29,6 +29,7 @@
 #include "kudu/tablet/deltamemstore.h"
 #include "kudu/tablet/deltafile.h"
 #include "kudu/tablet/mutation.h"
+#include "kudu/util/mem_tracker.h"
 #include "kudu/util/stopwatch.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
@@ -48,9 +49,12 @@ class TestDeltaMemStore : public KuduTest {
   TestDeltaMemStore()
     : op_id_(consensus::MaximumOpId()),
       schema_(CreateSchema()),
-      dms_(new DeltaMemStore(0, 0, new log::LogAnchorRegistry())),
       mvcc_(scoped_refptr<server::Clock>(
           server::LogicalClock::CreateStartingAt(Timestamp::kInitialTimestamp))) {
+    CHECK_OK(DeltaMemStore::Create(0, 0,
+                                   new log::LogAnchorRegistry(),
+                                   MemTracker::GetRootTracker(), &dms_));
+    CHECK_OK(dms_->Init());
   }
 
   void SetUp() OVERRIDE {

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/deltamemstore.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/deltamemstore.cc b/src/kudu/tablet/deltamemstore.cc
index 7653867..103ba29 100644
--- a/src/kudu/tablet/deltamemstore.cc
+++ b/src/kudu/tablet/deltamemstore.cc
@@ -42,26 +42,33 @@ using strings::Substitute;
 static const int kInitialArenaSize = 16;
 static const int kMaxArenaBufferSize = 5*1024*1024;
 
+Status DeltaMemStore::Create(int64_t id,
+                             int64_t rs_id,
+                             LogAnchorRegistry* log_anchor_registry,
+                             shared_ptr<MemTracker> parent_tracker,
+                             shared_ptr<DeltaMemStore>* dms) {
+  shared_ptr<DeltaMemStore> local_dms(new DeltaMemStore(id, rs_id,
+                                                        log_anchor_registry,
+                                                        std::move(parent_tracker)));
+
+  dms->swap(local_dms);
+  return Status::OK();
+}
+
 DeltaMemStore::DeltaMemStore(int64_t id,
                              int64_t rs_id,
                              LogAnchorRegistry* log_anchor_registry,
-                             const shared_ptr<MemTracker>& parent_tracker)
+                             shared_ptr<MemTracker> parent_tracker)
   : id_(id),
     rs_id_(rs_id),
-    anchorer_(log_anchor_registry, Substitute("Rowset-$0/DeltaMemStore-$1", rs_id_, id_)),
+    allocator_(new MemoryTrackingBufferAllocator(
+        HeapBufferAllocator::Get(), std::move(parent_tracker))),
+    arena_(new ThreadSafeMemoryTrackingArena(
+        kInitialArenaSize, kMaxArenaBufferSize, allocator_)),
+    tree_(arena_),
+    anchorer_(log_anchor_registry,
+              Substitute("Rowset-$0/DeltaMemStore-$1", rs_id_, id_)),
     disambiguator_sequence_number_(0) {
-  if (parent_tracker) {
-    CHECK(MemTracker::FindTracker(Tablet::kDMSMemTrackerId,
-                                  &mem_tracker_,
-                                  parent_tracker));
-  } else {
-    mem_tracker_ = MemTracker::GetRootTracker();
-  }
-  allocator_.reset(new MemoryTrackingBufferAllocator(
-      HeapBufferAllocator::Get(), mem_tracker_));
-  arena_.reset(new ThreadSafeMemoryTrackingArena(
-      kInitialArenaSize, kMaxArenaBufferSize, allocator_));
-  tree_.reset(new DMSTree(arena_));
 }
 
 Status DeltaMemStore::Init() {
@@ -80,7 +87,7 @@ Status DeltaMemStore::Update(Timestamp timestamp,
 
   Slice key_slice(buf);
   btree::PreparedMutation<DMSTreeTraits> mutation(key_slice);
-  mutation.Prepare(tree_.get());
+  mutation.Prepare(&tree_);
   if (PREDICT_FALSE(mutation.exists())) {
     // We already have a delta for this row at the same timestamp.
     // Try again with a disambiguating sequence number appended to the key.
@@ -88,7 +95,7 @@ Status DeltaMemStore::Update(Timestamp timestamp,
     PutMemcmpableVarint64(&buf, seq);
     key_slice = Slice(buf);
     mutation.Reset(key_slice);
-    mutation.Prepare(tree_.get());
+    mutation.Prepare(&tree_);
     CHECK(!mutation.exists())
       << "Appended a sequence number but still hit a duplicate "
       << "for rowid " << row_idx << " at timestamp " << timestamp;
@@ -106,7 +113,7 @@ Status DeltaMemStore::FlushToFile(DeltaFileWriter *dfw,
                                   gscoped_ptr<DeltaStats>* stats_ret) {
   gscoped_ptr<DeltaStats> stats(new DeltaStats());
 
-  gscoped_ptr<DMSTreeIter> iter(tree_->NewIterator());
+  gscoped_ptr<DMSTreeIter> iter(tree_.NewIterator());
   iter->SeekToStart();
   while (iter->IsValid()) {
     Slice key_slice, val;
@@ -141,8 +148,8 @@ Status DeltaMemStore::CheckRowDeleted(rowid_t row_idx, bool *deleted) const {
 
   bool exact;
 
-  // TODO: can we avoid the allocation here?
-  gscoped_ptr<DMSTreeIter> iter(tree_->NewIterator());
+  // TODO(unknown): can we avoid the allocation here?
+  gscoped_ptr<DMSTreeIter> iter(tree_.NewIterator());
   if (!iter->SeekAtOrAfter(key_slice, &exact)) {
     return Status::OK();
   }
@@ -168,7 +175,7 @@ Status DeltaMemStore::CheckRowDeleted(rowid_t row_idx, bool *deleted) const {
 }
 
 void DeltaMemStore::DebugPrint() const {
-  tree_->DebugPrint();
+  tree_.DebugPrint();
 }
 
 ////////////////////////////////////////////////////////////
@@ -179,7 +186,7 @@ DMSIterator::DMSIterator(const shared_ptr<const DeltaMemStore>& dms,
                          const Schema* projection, MvccSnapshot snapshot)
     : dms_(dms),
       mvcc_snapshot_(std::move(snapshot)),
-      iter_(dms->tree_->NewIterator()),
+      iter_(dms->tree_.NewIterator()),
       initted_(false),
       prepared_idx_(0),
       prepared_count_(0),

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/deltamemstore.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/deltamemstore.h b/src/kudu/tablet/deltamemstore.h
index a9bdf51..3ded59a 100644
--- a/src/kudu/tablet/deltamemstore.h
+++ b/src/kudu/tablet/deltamemstore.h
@@ -61,9 +61,10 @@ struct DMSTreeTraits : public btree::BTreeTraits {
 class DeltaMemStore : public DeltaStore,
                       public std::enable_shared_from_this<DeltaMemStore> {
  public:
-  DeltaMemStore(int64_t id, int64_t rs_id,
-                log::LogAnchorRegistry* log_anchor_registry,
-                const std::shared_ptr<MemTracker>& parent_tracker = std::shared_ptr<MemTracker>());
+  static Status Create(int64_t id, int64_t rs_id,
+                       log::LogAnchorRegistry* log_anchor_registry,
+                       std::shared_ptr<MemTracker> parent_tracker,
+                       std::shared_ptr<DeltaMemStore>* dms);
 
   virtual Status Init() OVERRIDE;
 
@@ -79,11 +80,11 @@ class DeltaMemStore : public DeltaStore,
                 const consensus::OpId& op_id);
 
   size_t Count() const {
-    return tree_->count();
+    return tree_.count();
   }
 
   bool Empty() const {
-    return tree_->empty();
+    return tree_.empty();
   }
 
   // Dump a debug version of the tree to the logs. This is not thread-safe, so
@@ -113,7 +114,7 @@ class DeltaMemStore : public DeltaStore,
   virtual Status CheckRowDeleted(rowid_t row_idx, bool *deleted) const OVERRIDE;
 
   virtual uint64_t EstimateSize() const OVERRIDE {
-    return memory_footprint();
+    return arena_->memory_footprint();
   }
 
   const int64_t id() const { return id_; }
@@ -121,10 +122,6 @@ class DeltaMemStore : public DeltaStore,
   typedef btree::CBTree<DMSTreeTraits> DMSTree;
   typedef btree::CBTreeIterator<DMSTreeTraits> DMSTreeIter;
 
-  size_t memory_footprint() const {
-    return arena_->memory_footprint();
-  }
-
   virtual std::string ToString() const OVERRIDE {
     return "DMS";
   }
@@ -142,20 +139,24 @@ class DeltaMemStore : public DeltaStore,
  private:
   friend class DMSIterator;
 
+  DeltaMemStore(int64_t id,
+                int64_t rs_id,
+                log::LogAnchorRegistry* log_anchor_registry,
+                std::shared_ptr<MemTracker> parent_tracker);
+
   const DMSTree& tree() const {
-    return *tree_;
+    return tree_;
   }
 
   const int64_t id_;    // DeltaMemStore ID.
   const int64_t rs_id_; // Rowset ID.
 
-  std::shared_ptr<MemTracker> mem_tracker_;
   std::shared_ptr<MemoryTrackingBufferAllocator> allocator_;
 
   std::shared_ptr<ThreadSafeMemoryTrackingArena> arena_;
 
   // Concurrent B-Tree storing <key index> -> RowChangeList
-  gscoped_ptr<DMSTree> tree_;
+  DMSTree tree_;
 
   log::MinLogIndexAnchorer anchorer_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/diskrowset-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/diskrowset-test-base.h b/src/kudu/tablet/diskrowset-test-base.h
index 262a21c..41fd79e 100644
--- a/src/kudu/tablet/diskrowset-test-base.h
+++ b/src/kudu/tablet/diskrowset-test-base.h
@@ -17,14 +17,16 @@
 #ifndef KUDU_TABLET_LAYER_TEST_BASE_H
 #define KUDU_TABLET_LAYER_TEST_BASE_H
 
-#include <glog/logging.h>
-#include <gtest/gtest.h>
+#include <unistd.h>
+
 #include <memory>
 #include <string>
-#include <unistd.h>
 #include <unordered_set>
 #include <vector>
 
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
 #include "kudu/common/iterator.h"
 #include "kudu/common/rowblock.h"
 #include "kudu/common/scan_spec.h"
@@ -35,6 +37,7 @@
 #include "kudu/server/logical_clock.h"
 #include "kudu/tablet/diskrowset.h"
 #include "kudu/tablet/tablet-test-util.h"
+#include "kudu/tablet/tablet_mem_trackers.h"
 #include "kudu/util/env.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/stopwatch.h"
@@ -320,7 +323,10 @@ class TestRowSet : public KuduRowSetTest {
   }
 
   Status OpenTestRowSet(std::shared_ptr<DiskRowSet> *rowset) {
-    return DiskRowSet::Open(rowset_meta_, new log::LogAnchorRegistry(), rowset);
+    return DiskRowSet::Open(rowset_meta_,
+                            new log::LogAnchorRegistry(),
+                            TabletMemTrackers(),
+                            rowset);
   }
 
   void FormatKey(int i, char *buf, size_t buf_len) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/diskrowset.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/diskrowset.cc b/src/kudu/tablet/diskrowset.cc
index e7fabf4..16c7031 100644
--- a/src/kudu/tablet/diskrowset.cc
+++ b/src/kudu/tablet/diskrowset.cc
@@ -461,9 +461,11 @@ RollingDiskRowSetWriter::~RollingDiskRowSetWriter() {
 
 Status DiskRowSet::Open(const shared_ptr<RowSetMetadata>& rowset_metadata,
                         log::LogAnchorRegistry* log_anchor_registry,
-                        shared_ptr<DiskRowSet> *rowset,
-                        const shared_ptr<MemTracker>& parent_tracker) {
-  shared_ptr<DiskRowSet> rs(new DiskRowSet(rowset_metadata, log_anchor_registry, parent_tracker));
+                        const TabletMemTrackers& mem_trackers,
+                        shared_ptr<DiskRowSet> *rowset) {
+  shared_ptr<DiskRowSet> rs(new DiskRowSet(rowset_metadata,
+                                           log_anchor_registry,
+                                           mem_trackers));
 
   RETURN_NOT_OK(rs->Open());
 
@@ -473,24 +475,24 @@ Status DiskRowSet::Open(const shared_ptr<RowSetMetadata>& rowset_metadata,
 
 DiskRowSet::DiskRowSet(shared_ptr<RowSetMetadata> rowset_metadata,
                        LogAnchorRegistry* log_anchor_registry,
-                       shared_ptr<MemTracker> parent_tracker)
+                       const TabletMemTrackers& mem_trackers)
     : rowset_metadata_(std::move(rowset_metadata)),
       open_(false),
       log_anchor_registry_(log_anchor_registry),
-      parent_tracker_(std::move(parent_tracker)) {}
+      mem_trackers_(mem_trackers) {}
 
 Status DiskRowSet::Open() {
   TRACE_EVENT0("tablet", "DiskRowSet::Open");
-  gscoped_ptr<CFileSet> new_base(new CFileSet(rowset_metadata_));
-  RETURN_NOT_OK(new_base->Open());
-  base_data_.reset(new_base.release());
+  RETURN_NOT_OK(CFileSet::Open(rowset_metadata_,
+                               mem_trackers_.tablet_tracker,
+                               &base_data_));
 
   rowid_t num_rows;
   RETURN_NOT_OK(base_data_->CountRows(&num_rows));
-  delta_tracker_.reset(new DeltaTracker(rowset_metadata_, num_rows,
-                                        log_anchor_registry_,
-                                        parent_tracker_));
-  RETURN_NOT_OK(delta_tracker_->Open());
+    RETURN_NOT_OK(DeltaTracker::Open(rowset_metadata_, num_rows,
+                                     log_anchor_registry_,
+                                     mem_trackers_,
+                                     &delta_tracker_));
 
   open_ = true;
 
@@ -537,12 +539,14 @@ Status DiskRowSet::MajorCompactDeltaStoresWithColumnIds(const vector<ColumnId>&
   RETURN_NOT_OK(rowset_metadata_->Flush());
 
   // Make the new base data and delta files visible.
-  gscoped_ptr<CFileSet> new_base(new CFileSet(rowset_metadata_));
-  RETURN_NOT_OK(new_base->Open());
+  shared_ptr<CFileSet> new_base;
+  RETURN_NOT_OK(CFileSet::Open(rowset_metadata_,
+                               mem_trackers_.tablet_tracker,
+                               &new_base));
   {
     std::lock_guard<percpu_rwlock> lock(component_lock_);
     RETURN_NOT_OK(compaction->UpdateDeltaTracker(delta_tracker_.get()));
-    base_data_.reset(new_base.release());
+    base_data_.swap(new_base);
   }
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/diskrowset.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/diskrowset.h b/src/kudu/tablet/diskrowset.h
index d096793..feb5afd 100644
--- a/src/kudu/tablet/diskrowset.h
+++ b/src/kudu/tablet/diskrowset.h
@@ -35,9 +35,11 @@
 #include "kudu/tablet/delta_key.h"
 #include "kudu/tablet/rowset_metadata.h"
 #include "kudu/tablet/rowset.h"
+#include "kudu/tablet/tablet_mem_trackers.h"
 #include "kudu/util/atomic.h"
 #include "kudu/util/bloom_filter.h"
 #include "kudu/util/locks.h"
+#include "kudu/util/mem_tracker.h"
 
 namespace kudu {
 
@@ -270,9 +272,8 @@ class DiskRowSet : public RowSet {
   // If successful, sets *rowset to the newly open rowset
   static Status Open(const std::shared_ptr<RowSetMetadata>& rowset_metadata,
                      log::LogAnchorRegistry* log_anchor_registry,
-                     std::shared_ptr<DiskRowSet> *rowset,
-                     const std::shared_ptr<MemTracker>& parent_tracker =
-                     std::shared_ptr<MemTracker>());
+                     const TabletMemTrackers& mem_trackers,
+                     std::shared_ptr<DiskRowSet> *rowset);
 
   ////////////////////////////////////////////////////////////
   // "Management" functions
@@ -378,7 +379,7 @@ class DiskRowSet : public RowSet {
 
   DiskRowSet(std::shared_ptr<RowSetMetadata> rowset_metadata,
              log::LogAnchorRegistry* log_anchor_registry,
-             std::shared_ptr<MemTracker> parent_tracker);
+             const TabletMemTrackers& mem_trackers);
 
   Status Open();
 
@@ -397,7 +398,7 @@ class DiskRowSet : public RowSet {
 
   log::LogAnchorRegistry* log_anchor_registry_;
 
-  std::shared_ptr<MemTracker> parent_tracker_;
+  TabletMemTrackers mem_trackers_;
 
   // Base data for this rowset.
   mutable percpu_rwlock component_lock_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/memrowset-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/memrowset-test.cc b/src/kudu/tablet/memrowset-test.cc
index 4247dc0..021de32 100644
--- a/src/kudu/tablet/memrowset-test.cc
+++ b/src/kudu/tablet/memrowset-test.cc
@@ -189,7 +189,9 @@ class TestMemRowSet : public ::testing::Test {
 
 
 TEST_F(TestMemRowSet, TestInsertAndIterate) {
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              MemTracker::GetRootTracker(), &mrs));
 
   ASSERT_OK(InsertRow(mrs.get(), "hello world", 12345));
   ASSERT_OK(InsertRow(mrs.get(), "goodbye world", 54321));
@@ -223,7 +225,9 @@ TEST_F(TestMemRowSet, TestInsertAndIterateCompoundKey) {
   ASSERT_OK(builder.AddColumn("val", UINT32));
   Schema compound_key_schema = builder.Build();
 
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, compound_key_schema, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, compound_key_schema, log_anchor_registry_.get(),
+                              MemTracker::GetRootTracker(), &mrs));
 
   RowBuilder rb(compound_key_schema);
   {
@@ -293,7 +297,9 @@ TEST_F(TestMemRowSet, TestInsertAndIterateCompoundKey) {
 
 // Test that inserting duplicate key data fails with Status::AlreadyPresent
 TEST_F(TestMemRowSet, TestInsertDuplicate) {
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              MemTracker::GetRootTracker(), &mrs));
 
   ASSERT_OK(InsertRow(mrs.get(), "hello world", 12345));
   Status s = InsertRow(mrs.get(), "hello world", 12345);
@@ -302,7 +308,9 @@ TEST_F(TestMemRowSet, TestInsertDuplicate) {
 
 // Test for updating rows in memrowset
 TEST_F(TestMemRowSet, TestUpdate) {
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              MemTracker::GetRootTracker(), &mrs));
 
   ASSERT_OK(InsertRow(mrs.get(), "hello world", 1));
 
@@ -328,7 +336,9 @@ TEST_F(TestMemRowSet, TestUpdate) {
 // Test which inserts many rows into memrowset and checks for their
 // existence
 TEST_F(TestMemRowSet, TestInsertCopiesToArena) {
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              MemTracker::GetRootTracker(), &mrs));
 
   ASSERT_OK(InsertRows(mrs.get(), 100));
   // Validate insertion
@@ -344,7 +354,9 @@ TEST_F(TestMemRowSet, TestDelete) {
   const char kRowKey[] = "hello world";
   bool present;
 
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              MemTracker::GetRootTracker(), &mrs));
 
   // Insert row.
   ASSERT_OK(InsertRow(mrs.get(), kRowKey, 1));
@@ -416,7 +428,9 @@ TEST_F(TestMemRowSet, TestDelete) {
 // Test for basic operations.
 // Can operate as a benchmark by setting --roundtrip_num_rows to a high value like 10M
 TEST_F(TestMemRowSet, TestMemRowSetInsertCountAndScan) {
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              MemTracker::GetRootTracker(), &mrs));
 
   LOG_TIMING(INFO, "Inserting rows") {
     ASSERT_OK(InsertRows(mrs.get(), FLAGS_roundtrip_num_rows));
@@ -442,7 +456,9 @@ TEST_F(TestMemRowSet, TestMemRowSetInsertCountAndScan) {
 // Test that scanning at past MVCC snapshots will hide rows which are
 // not committed in that snapshot.
 TEST_F(TestMemRowSet, TestInsertionMVCC) {
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              MemTracker::GetRootTracker(), &mrs));
   vector<MvccSnapshot> snapshots;
 
   // Insert 5 rows in tx 0 through 4
@@ -481,7 +497,9 @@ TEST_F(TestMemRowSet, TestInsertionMVCC) {
 // Test that updates respect MVCC -- i.e. that scanning with a past MVCC snapshot
 // will yield old versions of a row which has been updated.
 TEST_F(TestMemRowSet, TestUpdateMVCC) {
-  shared_ptr<MemRowSet> mrs(new MemRowSet(0, schema_, log_anchor_registry_.get()));
+  shared_ptr<MemRowSet> mrs;
+  ASSERT_OK(MemRowSet::Create(0, schema_, log_anchor_registry_.get(),
+                              MemTracker::GetRootTracker(), &mrs));
 
   // Insert a row ("myrow", 0)
   ASSERT_OK(InsertRow(mrs.get(), "my row", 0));

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/memrowset.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/memrowset.cc b/src/kudu/tablet/memrowset.cc
index d30f4a1..2ea5e60 100644
--- a/src/kudu/tablet/memrowset.cc
+++ b/src/kudu/tablet/memrowset.cc
@@ -17,11 +17,13 @@
 
 #include "kudu/tablet/memrowset.h"
 
-#include <gflags/gflags.h>
-#include <glog/logging.h>
+#include <memory>
 #include <string>
 #include <vector>
 
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
 #include "kudu/codegen/compilation_manager.h"
 #include "kudu/codegen/row_projector.h"
 #include "kudu/common/common.pb.h"
@@ -83,15 +85,26 @@ shared_ptr<MemTracker> CreateMemTrackerForMemRowSet(
 
 } // anonymous namespace
 
+Status MemRowSet::Create(int64_t id,
+                         const Schema &schema,
+                         LogAnchorRegistry* log_anchor_registry,
+                         shared_ptr<MemTracker> parent_tracker,
+                         shared_ptr<MemRowSet>* mrs) {
+  shared_ptr<MemRowSet> local_mrs(new MemRowSet(
+      id, schema, log_anchor_registry, std::move(parent_tracker)));
+
+  mrs->swap(local_mrs);
+  return Status::OK();
+}
+
 MemRowSet::MemRowSet(int64_t id,
                      const Schema &schema,
                      LogAnchorRegistry* log_anchor_registry,
-                     const shared_ptr<MemTracker>& parent_tracker)
+                     shared_ptr<MemTracker> parent_tracker)
   : id_(id),
     schema_(schema),
-    parent_tracker_(parent_tracker),
-    mem_tracker_(CreateMemTrackerForMemRowSet(id, parent_tracker)),
-    allocator_(new MemoryTrackingBufferAllocator(HeapBufferAllocator::Get(), mem_tracker_)),
+    allocator_(new MemoryTrackingBufferAllocator(HeapBufferAllocator::Get(),
+                                                 CreateMemTrackerForMemRowSet(id, parent_tracker))),
     arena_(new ThreadSafeMemoryTrackingArena(kInitialArenaSize, kMaxArenaBufferSize,
                                              allocator_)),
     tree_(arena_),

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/memrowset.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/memrowset.h b/src/kudu/tablet/memrowset.h
index d964211..8f81110 100644
--- a/src/kudu/tablet/memrowset.h
+++ b/src/kudu/tablet/memrowset.h
@@ -17,20 +17,23 @@
 #ifndef KUDU_TABLET_MEMROWSET_H
 #define KUDU_TABLET_MEMROWSET_H
 
-#include <boost/optional/optional_fwd.hpp>
 #include <memory>
 #include <mutex>
 #include <string>
 #include <vector>
 
-#include "kudu/common/scan_spec.h"
+#include <boost/optional/optional_fwd.hpp>
+
 #include "kudu/common/rowblock.h"
+#include "kudu/common/scan_spec.h"
 #include "kudu/common/schema.h"
 #include "kudu/consensus/log_anchor_registry.h"
 #include "kudu/tablet/concurrent_btree.h"
 #include "kudu/tablet/mutation.h"
 #include "kudu/tablet/rowset.h"
+#include "kudu/tablet/rowset_metadata.h"
 #include "kudu/tablet/tablet.pb.h"
+#include "kudu/util/mem_tracker.h"
 #include "kudu/util/memory/arena.h"
 #include "kudu/util/memory/memory.h"
 #include "kudu/util/status.h"
@@ -181,11 +184,11 @@ class MemRowSet : public RowSet,
  public:
   class Iterator;
 
-  MemRowSet(int64_t id,
-            const Schema &schema,
-            log::LogAnchorRegistry* log_anchor_registry,
-            const std::shared_ptr<MemTracker>& parent_tracker =
-            std::shared_ptr<MemTracker>());
+  static Status Create(int64_t id,
+                       const Schema &schema,
+                       log::LogAnchorRegistry* log_anchor_registry,
+                       std::shared_ptr<MemTracker> parent_tracker,
+                       std::shared_ptr<MemRowSet>* mrs);
 
   ~MemRowSet();
 
@@ -341,19 +344,22 @@ class MemRowSet : public RowSet,
  private:
   friend class Iterator;
 
+  MemRowSet(int64_t id,
+            const Schema &schema,
+            log::LogAnchorRegistry* log_anchor_registry,
+            std::shared_ptr<MemTracker> parent_tracker);
+
   // Perform a "Reinsert" -- handle an insertion into a row which was previously
   // inserted and deleted, but still has an entry in the MemRowSet.
   Status Reinsert(Timestamp timestamp,
-                  const ConstContiguousRow& row_data,
-                  MRSRow *row);
+                  const ConstContiguousRow& row,
+                  MRSRow *ms_row);
 
   typedef btree::CBTree<MSBTreeTraits> MSBTree;
 
   int64_t id_;
 
   const Schema schema_;
-  std::shared_ptr<MemTracker> parent_tracker_;
-  std::shared_ptr<MemTracker> mem_tracker_;
   std::shared_ptr<MemoryTrackingBufferAllocator> allocator_;
   std::shared_ptr<ThreadSafeMemoryTrackingArena> arena_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/tablet-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet-test.cc b/src/kudu/tablet/tablet-test.cc
index e2844d9..73ae522 100644
--- a/src/kudu/tablet/tablet-test.cc
+++ b/src/kudu/tablet/tablet-test.cc
@@ -19,6 +19,7 @@
 
 #include <glog/logging.h>
 
+#include "kudu/cfile/cfile_util.h"
 #include "kudu/common/iterator.h"
 #include "kudu/common/row.h"
 #include "kudu/common/scan_spec.h"
@@ -43,6 +44,7 @@ using std::shared_ptr;
 namespace kudu {
 namespace tablet {
 
+using cfile::ReaderOptions;
 using fs::ReadableBlock;
 
 template<class SETUP>
@@ -85,7 +87,7 @@ TYPED_TEST(TestTablet, TestFlush) {
   ASSERT_OK(this->fs_manager()->OpenBlock(undo_blocks[0], &block));
 
   shared_ptr<DeltaFileReader> dfr;
-  ASSERT_OK(DeltaFileReader::Open(std::move(block), undo_blocks[0], &dfr, UNDO));
+  ASSERT_OK(DeltaFileReader::Open(std::move(block), UNDO, ReaderOptions(), &dfr));
   // Assert there were 'max_rows' deletions in the undo delta (one for each inserted row)
   ASSERT_EQ(dfr->delta_stats().delete_count(), max_rows);
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/tablet.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index 0185619..27d6be7 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -154,8 +154,6 @@ TabletComponents::TabletComponents(shared_ptr<MemRowSet> mrs,
 // Tablet
 ////////////////////////////////////////////////////////////
 
-const char* Tablet::kDMSMemTrackerId = "DeltaMemStores";
-
 Tablet::Tablet(const scoped_refptr<TabletMetadata>& metadata,
                const scoped_refptr<server::Clock>& clock,
                const shared_ptr<MemTracker>& parent_mem_tracker,
@@ -164,11 +162,7 @@ Tablet::Tablet(const scoped_refptr<TabletMetadata>& metadata,
   : key_schema_(metadata->schema().CreateKeyProjection()),
     metadata_(metadata),
     log_anchor_registry_(log_anchor_registry),
-    mem_tracker_(MemTracker::CreateTracker(
-        -1, Substitute("tablet-$0", tablet_id()),
-                       parent_mem_tracker)),
-    dms_mem_tracker_(MemTracker::CreateTracker(
-        -1, kDMSMemTrackerId, mem_tracker_)),
+    mem_trackers_(tablet_id(), parent_mem_tracker),
     next_mrs_id_(0),
     clock_(clock),
     mvcc_(clock),
@@ -219,7 +213,10 @@ Status Tablet::Open() {
   // open the tablet row-sets
   for (const shared_ptr<RowSetMetadata>& rowset_meta : metadata_->rowsets()) {
     shared_ptr<DiskRowSet> rowset;
-    Status s = DiskRowSet::Open(rowset_meta, log_anchor_registry_.get(), &rowset, mem_tracker_);
+    Status s = DiskRowSet::Open(rowset_meta,
+                                log_anchor_registry_.get(),
+                                mem_trackers_,
+                                &rowset);
     if (!s.ok()) {
       LOG_WITH_PREFIX(ERROR) << "Failed to open rowset " << rowset_meta->ToString() << ": "
                              << s.ToString();
@@ -232,9 +229,11 @@ Status Tablet::Open() {
   shared_ptr<RowSetTree> new_rowset_tree(new RowSetTree());
   CHECK_OK(new_rowset_tree->Reset(rowsets_opened));
   // now that the current state is loaded, create the new MemRowSet with the next id
-  shared_ptr<MemRowSet> new_mrs(new MemRowSet(next_mrs_id_++, *schema(),
-                                              log_anchor_registry_.get(),
-                                              mem_tracker_));
+  shared_ptr<MemRowSet> new_mrs;
+  RETURN_NOT_OK(MemRowSet::Create(next_mrs_id_++, *schema(),
+                                  log_anchor_registry_.get(),
+                                  mem_trackers_.tablet_tracker,
+                                  &new_mrs));
   components_ = new TabletComponents(new_mrs, new_rowset_tree);
 
   state_ = kBootstrapping;
@@ -766,8 +765,11 @@ Status Tablet::ReplaceMemRowSetUnlocked(RowSetsInCompaction *compaction,
   // Add to compaction.
   compaction->AddRowSet(*old_ms, std::move(ms_lock));
 
-  shared_ptr<MemRowSet> new_mrs(new MemRowSet(next_mrs_id_++, *schema(), log_anchor_registry_.get(),
-                                mem_tracker_));
+  shared_ptr<MemRowSet> new_mrs;
+  RETURN_NOT_OK(MemRowSet::Create(next_mrs_id_++, *schema(),
+                                  log_anchor_registry_.get(),
+                                  mem_trackers_.tablet_tracker,
+                                  &new_mrs));
   shared_ptr<RowSetTree> new_rst(new RowSetTree());
   ModifyRowSetTree(*components_->rowsets,
                    RowSetVector(), // remove nothing
@@ -903,8 +905,11 @@ 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());
-    shared_ptr<MemRowSet> new_mrs(new MemRowSet(old_mrs->mrs_id(), new_schema,
-                                                log_anchor_registry_.get(), mem_tracker_));
+    shared_ptr<MemRowSet> new_mrs;
+    RETURN_NOT_OK(MemRowSet::Create(old_mrs->mrs_id(), new_schema,
+                                    log_anchor_registry_.get(),
+                                    mem_trackers_.tablet_tracker,
+                                    &new_mrs));
     components_ = new TabletComponents(new_mrs, old_rowsets);
   }
   return Status::OK();
@@ -1113,7 +1118,10 @@ Status Tablet::DoMergeCompactionOrFlush(const RowSetsInCompaction &input,
     TRACE_EVENT0("tablet", "Opening compaction results");
     for (const shared_ptr<RowSetMetadata>& meta : new_drs_metas) {
       shared_ptr<DiskRowSet> new_rowset;
-      Status s = DiskRowSet::Open(meta, log_anchor_registry_.get(), &new_rowset, mem_tracker_);
+      Status s = DiskRowSet::Open(meta,
+                                  log_anchor_registry_.get(),
+                                  mem_trackers_,
+                                  &new_rowset);
       if (!s.ok()) {
         LOG_WITH_PREFIX(WARNING) << "Unable to open snapshot " << op_name << " results "
                                  << meta->ToString() << ": " << s.ToString();

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/tablet.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet.h b/src/kudu/tablet/tablet.h
index fcc1c01..77bc9fe 100644
--- a/src/kudu/tablet/tablet.h
+++ b/src/kudu/tablet/tablet.h
@@ -29,11 +29,12 @@
 #include "kudu/gutil/atomicops.h"
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/macros.h"
-#include "kudu/tablet/rowset_metadata.h"
-#include "kudu/tablet/tablet_metadata.h"
 #include "kudu/tablet/lock_manager.h"
 #include "kudu/tablet/mvcc.h"
 #include "kudu/tablet/rowset.h"
+#include "kudu/tablet/rowset_metadata.h"
+#include "kudu/tablet/tablet_mem_trackers.h"
+#include "kudu/tablet/tablet_metadata.h"
 #include "kudu/util/locks.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/semaphore.h"
@@ -375,10 +376,14 @@ class Tablet {
   TabletMetrics* metrics() { return metrics_.get(); }
 
   // Return handle to the metric entity of this tablet.
-  const scoped_refptr<MetricEntity>& GetMetricEntity() const { return metric_entity_; }
+  const scoped_refptr<MetricEntity>& GetMetricEntity() const {
+    return metric_entity_;
+  }
 
   // Returns a reference to this tablet's memory tracker.
-  const std::shared_ptr<MemTracker>& mem_tracker() const { return mem_tracker_; }
+  const std::shared_ptr<MemTracker>& mem_tracker() const {
+    return mem_trackers_.tablet_tracker;
+  }
 
   // Throttle a RPC with 'bytes' request size.
   // Return true if this RPC is allowed.
@@ -386,7 +391,6 @@ class Tablet {
 
   scoped_refptr<server::Clock> clock() const { return clock_; }
 
-  static const char* kDMSMemTrackerId;
  private:
   friend class Iterator;
   friend class TabletPeerTest;
@@ -544,8 +548,7 @@ class Tablet {
   scoped_refptr<TabletComponents> components_;
 
   scoped_refptr<log::LogAnchorRegistry> log_anchor_registry_;
-  std::shared_ptr<MemTracker> mem_tracker_;
-  std::shared_ptr<MemTracker> dms_mem_tracker_;
+  TabletMemTrackers mem_trackers_;
 
   scoped_refptr<MetricEntity> metric_entity_;
   gscoped_ptr<TabletMetrics> metrics_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tablet/tablet_mem_trackers.h
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet_mem_trackers.h b/src/kudu/tablet/tablet_mem_trackers.h
new file mode 100644
index 0000000..4882318
--- /dev/null
+++ b/src/kudu/tablet/tablet_mem_trackers.h
@@ -0,0 +1,52 @@
+// 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.
+#pragma once
+
+#include <memory>
+#include <string>
+
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/util/mem_tracker.h"
+
+namespace kudu {
+namespace tablet {
+
+struct TabletMemTrackers {
+
+  // Intended for unit tests where the tracker hierarchy doesn't matter.
+  TabletMemTrackers()
+    : tablet_tracker(MemTracker::GetRootTracker()),
+      dms_tracker(MemTracker::GetRootTracker()) {
+  }
+
+  TabletMemTrackers(std::string tablet_id,
+                    std::shared_ptr<MemTracker> parent_mem_tracker)
+    : tablet_tracker(MemTracker::CreateTracker(
+        -1,
+        strings::Substitute("tablet-$0", tablet_id),
+        parent_mem_tracker)),
+      dms_tracker(MemTracker::CreateTracker(-1, "DeltaMemStores", tablet_tracker)) {
+  }
+
+  std::shared_ptr<MemTracker> tablet_tracker;
+
+  // All of the below are children of tablet_tracker;
+  std::shared_ptr<MemTracker> dms_tracker;
+};
+
+} // namespace tablet
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/6bc07538/src/kudu/tools/tool_action_local_replica.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_local_replica.cc b/src/kudu/tools/tool_action_local_replica.cc
index 461235e..431f7ad 100644
--- a/src/kudu/tools/tool_action_local_replica.cc
+++ b/src/kudu/tools/tool_action_local_replica.cc
@@ -24,6 +24,7 @@
 #include <utility>
 
 #include "kudu/cfile/cfile_reader.h"
+#include "kudu/cfile/cfile_util.h"
 #include "kudu/common/common.pb.h"
 #include "kudu/common/row_changelist.h"
 #include "kudu/common/row_operations.h"
@@ -408,9 +409,9 @@ Status DumpDeltaCFileBlockInternal(FsManager* fs_manager,
   RETURN_NOT_OK(fs_manager->OpenBlock(block_id, &readable_block));
   shared_ptr<DeltaFileReader> delta_reader;
   RETURN_NOT_OK(DeltaFileReader::Open(std::move(readable_block),
-                                      block_id,
-                                      &delta_reader,
-                                      delta_type));
+                                      delta_type,
+                                      ReaderOptions(),
+                                      &delta_reader));
 
   cout << Indent(indent) << "Delta stats: "
        << delta_reader->delta_stats().ToString() << endl;
@@ -450,8 +451,8 @@ Status DumpDeltaCFileBlockInternal(FsManager* fs_manager,
   // information stored in the footer/store additional information in the
   // footer as to make it feasible iterate over all deltas using a
   // DeltaFileIterator alone.
-  shared_ptr<CFileSet> cfileset(new CFileSet(rs_meta));
-  RETURN_NOT_OK(cfileset->Open());
+  shared_ptr<CFileSet> cfileset;
+  RETURN_NOT_OK(CFileSet::Open(rs_meta, MemTracker::GetRootTracker(), &cfileset));
   gscoped_ptr<CFileSet::Iterator> cfileset_iter(cfileset->NewIterator(&schema));
 
   RETURN_NOT_OK(cfileset_iter->Init(NULL));