You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2021/03/13 04:10:00 UTC

[GitHub] [incubator-doris] HappenLee opened a new pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

HappenLee opened a new pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516


    This PR main about:
    1. Improve the readability of MemTrackers' name
    2. Add the MemTracker of:
       * Load
       * Compaction
       * SchemaChange
       * StoragePageCache
       * TabletManager
    3. Change SchemaChange to a Singleon
   
   ## Proposed changes
   
   Describe the big picture of your changes here to communicate to the maintainers why we should accept this pull request. If it fixes a bug or resolves a feature request, be sure to link to that issue.
   
   ## Types of changes
   
   What types of changes does your code introduce to Doris?
   _Put an `x` in the boxes that apply_
   
   - [ ] Bugfix (non-breaking change which fixes an issue)
   - [x] New feature (non-breaking change which adds functionality)
   - [ ] Breaking change (fix or feature that would cause existing functionality to not work as expected)
   - [ ] Documentation Update (if none of the other choices apply)
   - [ ] Code refactor (Modify the code structure, format the code, etc...)
   
   ## Checklist
   
   _Put an `x` in the boxes that apply. You can also fill these out after creating the PR. If you're unsure about any of them, don't hesitate to ask. We're here to help! This is simply a reminder of what we are going to look for before merging your code._
   
   - [x] I have created an issue on (Fix #5515) and described the bug/feature there in detail
   - [x] Compiling and unit tests pass locally with my changes
   - [x] I have added tests that prove my fix is effective or that my feature works
   - [x] If these changes need document changes, I have updated the document
   - [x] Any dependent changes have been merged
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at dev@doris.apache.org by explaining why you chose the solution you did and what alternatives you considered, etc...
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] acelyc111 commented on a change in pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on a change in pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516#discussion_r594932439



##########
File path: be/src/olap/schema_change.cpp
##########
@@ -1660,16 +1664,18 @@ OLAPStatus SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tabl
     for (int i = 0; i < num_cols; ++i) {
         return_columns[i] = i;
     }
-    _reader_context.reader_type = READER_ALTER_TABLE;
-    _reader_context.tablet_schema = &base_tablet->tablet_schema();
-    _reader_context.need_ordered_result = true;
-    _reader_context.delete_handler = &delete_handler;
-    _reader_context.return_columns = &return_columns;
-    _reader_context.seek_columns = &return_columns;
+
+    RowsetReaderContext reader_context;
+    reader_context.reader_type = READER_ALTER_TABLE;
+    reader_context.tablet_schema = &base_tablet->tablet_schema();
+    reader_context.need_ordered_result = true;
+    reader_context.delete_handler = &delete_handler;
+    reader_context.return_columns = &return_columns;
+    reader_context.seek_columns = &return_columns;
 
     RowsetReaderSharedPtr rowset_reader;
     RETURN_NOT_OK((*base_rowset)->create_reader(&rowset_reader));
-    rowset_reader->init(&_reader_context);
+    rowset_reader->init(&reader_context);

Review comment:
       Sorry, it's different with above, it's no problem here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] HappenLee commented on a change in pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

Posted by GitBox <gi...@apache.org>.
HappenLee commented on a change in pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516#discussion_r594289096



##########
File path: be/src/olap/schema_change.cpp
##########
@@ -1518,16 +1521,17 @@ OLAPStatus SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletRe
             break;
         }
 
-        _reader_context.reader_type = READER_ALTER_TABLE;
-        _reader_context.tablet_schema = &base_tablet->tablet_schema();
-        _reader_context.need_ordered_result = true;
-        _reader_context.delete_handler = &delete_handler;
-        _reader_context.return_columns = &return_columns;
+        RowsetReaderContext reader_context;
+        reader_context.reader_type = READER_ALTER_TABLE;
+        reader_context.tablet_schema = &base_tablet->tablet_schema();
+        reader_context.need_ordered_result = true;
+        reader_context.delete_handler = &delete_handler;
+        reader_context.return_columns = &return_columns;
         // for schema change, seek_columns is the same to return_columns
-        _reader_context.seek_columns = &return_columns;
+        reader_context.seek_columns = &return_columns;
 
         for (auto& rs_reader : rs_readers) {
-            rs_reader->init(&_reader_context);
+            rs_reader->init(&reader_context);

Review comment:
       I check again. the `rowset_reader` and `reader_context` have the same lifetime in schema change handler, So do not need to afraid to wild pointer here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516#discussion_r593697205



##########
File path: be/src/olap/compaction.cpp
##########
@@ -30,6 +30,7 @@ Compaction::Compaction(TabletSharedPtr tablet, const std::string& label,
                        const std::shared_ptr<MemTracker>& parent_tracker)
         : _mem_tracker(MemTracker::CreateTracker(-1, label, parent_tracker)),
           _readers_tracker(MemTracker::CreateTracker(-1, "Compaction:RowsetReaders:" + std::to_string(tablet->tablet_id()), _mem_tracker)),
+          _writer_tracker(MemTracker::CreateTracker(-1, "CompationWriterTracker" + std::to_string(tablet->tablet_id()), _mem_tracker)),

Review comment:
       Change the `Compaction:RowsetReaders:` to `CompactionReadersTracker`?

##########
File path: be/src/olap/base_tablet.cpp
##########
@@ -31,6 +31,8 @@ extern MetricPrototype METRIC_query_scan_count;
 BaseTablet::BaseTablet(TabletMetaSharedPtr tablet_meta, DataDir* data_dir)
         : _state(tablet_meta->tablet_state()),
           _tablet_meta(tablet_meta),
+          // TODO: Think we really copy tablet schema here, which cost double mem
+          // cost in TabletManager

Review comment:
       I think it is unnecessary. schema can be got directly from `tablet_meta`. And `tablet_meta` is also saved in tablet.

##########
File path: be/src/olap/delta_writer.cpp
##########
@@ -307,8 +308,8 @@ OLAPStatus DeltaWriter::close_wait(google::protobuf::RepeatedPtrField<PTabletInf
 
     if (_new_tablet != nullptr) {
         LOG(INFO) << "convert version for schema change";
-        SchemaChangeHandler schema_change;
-        res = schema_change.schema_version_convert(_tablet, _new_tablet, &_cur_rowset,
+        auto schema_change_handler = SchemaChangeHandler::instance();

Review comment:
       Why changing it to singleton?

##########
File path: be/src/olap/rowset/segment_v2/segment_iterator.cpp
##########
@@ -194,6 +198,7 @@ Status SegmentIterator::_prepare_seek(const StorageReadOptions::KeyRange& key_ra
             ColumnIteratorOptions iter_opts;
             iter_opts.stats = _opts.stats;
             iter_opts.rblock = _rblock.get();
+            iter_opts.mem_tracker = MemTracker::CreateTracker(-1, "ColumnIterator", _mem_tracker, false);

Review comment:
       Memtracker for each column?

##########
File path: be/src/olap/row_block2.cpp
##########
@@ -27,10 +27,13 @@ using strings::Substitute;
 namespace doris {
 
 RowBlockV2::RowBlockV2(const Schema& schema, uint16_t capacity)
+        : RowBlockV2(schema, capacity, nullptr) {}
+
+RowBlockV2::RowBlockV2(const Schema& schema, uint16_t capacity, std::shared_ptr<MemTracker> parent)
         : _schema(schema),
           _capacity(capacity),
           _column_vector_batches(_schema.num_columns()),
-          _tracker(new MemTracker(-1, "RowBlockV2")),
+          _tracker(MemTracker::CreateTracker(-1, "RowBlockV2", parent)),

Review comment:
       create mem tracker for each row block v2? Is it too heavy?
   And when to use RowBlockV2 with or without MemTracker?

##########
File path: be/src/olap/tablet_manager.cpp
##########
@@ -185,9 +186,13 @@ OLAPStatus TabletManager::_add_tablet_to_map_unlocked(TTabletId tablet_id, Schem
     tablet_map[tablet_id].table_arr.push_back(tablet);
     tablet_map[tablet_id].table_arr.sort(_cmp_tablet_by_create_time);
     _add_tablet_to_partition(*tablet);
+    // table schema will copy in tablet, there will be double mem cost
+    // so here multiply 2

Review comment:
       Add a TODO here




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] HappenLee commented on a change in pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

Posted by GitBox <gi...@apache.org>.
HappenLee commented on a change in pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516#discussion_r594287747



##########
File path: be/src/olap/tablet_manager.cpp
##########
@@ -185,9 +186,13 @@ OLAPStatus TabletManager::_add_tablet_to_map_unlocked(TTabletId tablet_id, Schem
     tablet_map[tablet_id].table_arr.push_back(tablet);
     tablet_map[tablet_id].table_arr.sort(_cmp_tablet_by_create_time);
     _add_tablet_to_partition(*tablet);
+    // table schema will copy in tablet, there will be double mem cost
+    // so here multiply 2

Review comment:
       Already add




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516#discussion_r595944026



##########
File path: be/src/olap/schema_change.cpp
##########
@@ -710,26 +710,26 @@ bool RowBlockSorter::sort(RowBlock** row_block) {
     return true;
 }
 
-RowBlockAllocator::RowBlockAllocator(const TabletSchema& tablet_schema, size_t memory_limitation)
+RowBlockAllocator::RowBlockAllocator(const TabletSchema& tablet_schema, std::shared_ptr<MemTracker> parent, size_t memory_limitation)
         : _tablet_schema(tablet_schema),
-          _memory_allocated(0),
+          _mem_tracker(MemTracker::CreateTracker(-1, "RowBlockAllocator", parent, false)),
           _row_len(tablet_schema.row_size()),
           _memory_limitation(memory_limitation) {
     VLOG_NOTICE << "RowBlockAllocator(). row_len=" << _row_len;
 }
 
 RowBlockAllocator::~RowBlockAllocator() {
-    if (_memory_allocated != 0) {
-        LOG(WARNING) << "memory lost in RowBlockAllocator. memory_size=" << _memory_allocated;
+    if (_mem_tracker->consumption() != 0) {
+        LOG(WARNING) << "memory lost in RowBlockAllocator. memory_size=" << _mem_tracker->consumption();
     }
 }
 
 OLAPStatus RowBlockAllocator::allocate(RowBlock** row_block, size_t num_rows, bool null_supported) {
     size_t row_block_size = _row_len * num_rows;
 
-    if (_memory_limitation > 0 && _memory_allocated + row_block_size > _memory_limitation) {
+    if (_memory_limitation > 0 && _mem_tracker->consumption() + row_block_size > _memory_limitation) {
         VLOG_NOTICE << "RowBlockAllocator::alocate() memory exceeded. "

Review comment:
       Change it to LOG(WARNING)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] acelyc111 commented on a change in pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on a change in pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516#discussion_r594029833



##########
File path: be/src/olap/compaction.cpp
##########
@@ -29,7 +29,8 @@ namespace doris {
 Compaction::Compaction(TabletSharedPtr tablet, const std::string& label,
                        const std::shared_ptr<MemTracker>& parent_tracker)
         : _mem_tracker(MemTracker::CreateTracker(-1, label, parent_tracker)),
-          _readers_tracker(MemTracker::CreateTracker(-1, "Compaction:RowsetReaders:" + std::to_string(tablet->tablet_id()), _mem_tracker)),
+          _readers_tracker(MemTracker::CreateTracker(-1, "CompactionReaderTracker:" + std::to_string(tablet->tablet_id()), _mem_tracker)),
+          _writer_tracker(MemTracker::CreateTracker(-1, "CompationWriterTracker" + std::to_string(tablet->tablet_id()), _mem_tracker)),

Review comment:
       ```suggestion
             _writer_tracker(MemTracker::CreateTracker(-1, "CompationWriterTracker:" + std::to_string(tablet->tablet_id()), _mem_tracker)),
   ```

##########
File path: be/src/olap/schema_change.cpp
##########
@@ -1518,16 +1521,17 @@ OLAPStatus SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletRe
             break;
         }
 
-        _reader_context.reader_type = READER_ALTER_TABLE;
-        _reader_context.tablet_schema = &base_tablet->tablet_schema();
-        _reader_context.need_ordered_result = true;
-        _reader_context.delete_handler = &delete_handler;
-        _reader_context.return_columns = &return_columns;
+        RowsetReaderContext reader_context;
+        reader_context.reader_type = READER_ALTER_TABLE;
+        reader_context.tablet_schema = &base_tablet->tablet_schema();
+        reader_context.need_ordered_result = true;
+        reader_context.delete_handler = &delete_handler;
+        reader_context.return_columns = &return_columns;
         // for schema change, seek_columns is the same to return_columns
-        _reader_context.seek_columns = &return_columns;
+        reader_context.seek_columns = &return_columns;
 
         for (auto& rs_reader : rs_readers) {
-            rs_reader->init(&_reader_context);
+            rs_reader->init(&reader_context);

Review comment:
       Now `reader_context` is a local variable on stack, rs_reader init by a pointer to it may cause some problem.

##########
File path: be/src/olap/schema_change.cpp
##########
@@ -1660,16 +1664,18 @@ OLAPStatus SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tabl
     for (int i = 0; i < num_cols; ++i) {
         return_columns[i] = i;
     }
-    _reader_context.reader_type = READER_ALTER_TABLE;
-    _reader_context.tablet_schema = &base_tablet->tablet_schema();
-    _reader_context.need_ordered_result = true;
-    _reader_context.delete_handler = &delete_handler;
-    _reader_context.return_columns = &return_columns;
-    _reader_context.seek_columns = &return_columns;
+
+    RowsetReaderContext reader_context;
+    reader_context.reader_type = READER_ALTER_TABLE;
+    reader_context.tablet_schema = &base_tablet->tablet_schema();
+    reader_context.need_ordered_result = true;
+    reader_context.delete_handler = &delete_handler;
+    reader_context.return_columns = &return_columns;
+    reader_context.seek_columns = &return_columns;
 
     RowsetReaderSharedPtr rowset_reader;
     RETURN_NOT_OK((*base_rowset)->create_reader(&rowset_reader));
-    rowset_reader->init(&_reader_context);
+    rowset_reader->init(&reader_context);

Review comment:
       Same

##########
File path: be/src/olap/rowset/segment_v2/segment_writer.cpp
##########
@@ -36,12 +37,15 @@ const char* k_segment_magic = "D0R1";
 const uint32_t k_segment_magic_length = 4;
 
 SegmentWriter::SegmentWriter(fs::WritableBlock* wblock, uint32_t segment_id,
-                             const TabletSchema* tablet_schema, const SegmentWriterOptions& opts)
-        : _segment_id(segment_id), _tablet_schema(tablet_schema), _opts(opts), _wblock(wblock) {
+                             const TabletSchema* tablet_schema, const SegmentWriterOptions& opts, std::shared_ptr<MemTracker> parent)
+        : _segment_id(segment_id), _tablet_schema(tablet_schema), _opts(opts), _wblock(wblock), _mem_tracker(MemTracker::CreateTracker(
+                -1, "Segment", parent, false)) {

Review comment:
       Class Segment has a MemTracker with the same name "Segment", how to distinguish them?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] yangzhg merged pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

Posted by GitBox <gi...@apache.org>.
yangzhg merged pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] HappenLee commented on a change in pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

Posted by GitBox <gi...@apache.org>.
HappenLee commented on a change in pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516#discussion_r594020979



##########
File path: be/src/olap/delta_writer.cpp
##########
@@ -307,8 +308,8 @@ OLAPStatus DeltaWriter::close_wait(google::protobuf::RepeatedPtrField<PTabletInf
 
     if (_new_tablet != nullptr) {
         LOG(INFO) << "convert version for schema change";
-        SchemaChangeHandler schema_change;
-        res = schema_change.schema_version_convert(_tablet, _new_tablet, &_cur_rowset,
+        auto schema_change_handler = SchemaChangeHandler::instance();

Review comment:
       Just to make it easy to mem trace all mem use in `SchemaChange`.
   At the same time , there is no need to new a instance each time of schemchange




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] HappenLee commented on a change in pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

Posted by GitBox <gi...@apache.org>.
HappenLee commented on a change in pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516#discussion_r594021297



##########
File path: be/src/olap/rowset/segment_v2/segment_iterator.cpp
##########
@@ -194,6 +198,7 @@ Status SegmentIterator::_prepare_seek(const StorageReadOptions::KeyRange& key_ra
             ColumnIteratorOptions iter_opts;
             iter_opts.stats = _opts.stats;
             iter_opts.rblock = _rblock.get();
+            iter_opts.mem_tracker = MemTracker::CreateTracker(-1, "ColumnIterator", _mem_tracker, false);

Review comment:
       Yes, each Column manage mem of itselfs.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] HappenLee commented on a change in pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

Posted by GitBox <gi...@apache.org>.
HappenLee commented on a change in pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516#discussion_r594072448



##########
File path: be/src/olap/rowset/segment_v2/segment_writer.cpp
##########
@@ -36,12 +37,15 @@ const char* k_segment_magic = "D0R1";
 const uint32_t k_segment_magic_length = 4;
 
 SegmentWriter::SegmentWriter(fs::WritableBlock* wblock, uint32_t segment_id,
-                             const TabletSchema* tablet_schema, const SegmentWriterOptions& opts)
-        : _segment_id(segment_id), _tablet_schema(tablet_schema), _opts(opts), _wblock(wblock) {
+                             const TabletSchema* tablet_schema, const SegmentWriterOptions& opts, std::shared_ptr<MemTracker> parent)
+        : _segment_id(segment_id), _tablet_schema(tablet_schema), _opts(opts), _wblock(wblock), _mem_tracker(MemTracker::CreateTracker(
+                -1, "Segment", parent, false)) {

Review comment:
       Yes, At first I add the segment id in the memtracker indeedly. But I find the memtracker still hard to distinguish, so i remove it. I will add segment id in the name agein~~~




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] HappenLee commented on a change in pull request #5516: [Enhance] Make MemTracker more accurate (#5515)

Posted by GitBox <gi...@apache.org>.
HappenLee commented on a change in pull request #5516:
URL: https://github.com/apache/incubator-doris/pull/5516#discussion_r594072589



##########
File path: be/src/olap/compaction.cpp
##########
@@ -29,7 +29,8 @@ namespace doris {
 Compaction::Compaction(TabletSharedPtr tablet, const std::string& label,
                        const std::shared_ptr<MemTracker>& parent_tracker)
         : _mem_tracker(MemTracker::CreateTracker(-1, label, parent_tracker)),
-          _readers_tracker(MemTracker::CreateTracker(-1, "Compaction:RowsetReaders:" + std::to_string(tablet->tablet_id()), _mem_tracker)),
+          _readers_tracker(MemTracker::CreateTracker(-1, "CompactionReaderTracker:" + std::to_string(tablet->tablet_id()), _mem_tracker)),
+          _writer_tracker(MemTracker::CreateTracker(-1, "CompationWriterTracker" + std::to_string(tablet->tablet_id()), _mem_tracker)),

Review comment:
       sorry, I forget this. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org