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 08:34:38 UTC

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

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