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 2022/10/13 11:13:21 UTC

[GitHub] [doris] xinyiZzz commented on a diff in pull request #13285: [improvement](memory) disable page cache and chunk allocator, optimize memory allocate size

xinyiZzz commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r994460430


##########
be/src/runtime/exec_env_init.cpp:
##########
@@ -185,6 +185,8 @@ Status ExecEnv::_init(const std::vector<StorePath>& store_paths) {
 }
 
 Status ExecEnv::_init_mem_tracker() {
+    LOG(INFO) << "Physical memory is: "

Review Comment:
   physical memory is printed on the line `mem_info.cpp:105`



##########
be/src/runtime/exec_env_init.cpp:
##########
@@ -323,8 +317,6 @@ Status ExecEnv::_init_mem_tracker() {
            << config::chunk_reserved_bytes_limit;
         return Status::InternalError(ss.str());
     }
-    chunk_reserved_bytes_limit =
-            BitUtil::RoundDown(chunk_reserved_bytes_limit, config::min_chunk_reserved_bytes);

Review Comment:
   The `BitUtil::RoundDown(chunk_reserved_bytes_limit, 4096)` here ensures that chunk_reserved_bytes_limit is a multiple of 4096
   
   4096 is the minimum chunk size currently allocated by the chunk allocator
   
   A separate conf `min_chunk_reserved_bytes` is not necessary, but `RoundDown` is meaningful



##########
be/src/vec/common/arena.h:
##########
@@ -127,11 +127,16 @@ class Arena : private boost::noncopyable {
 
 public:
     Arena(size_t initial_size_ = 4096, size_t growth_factor_ = 2,
-          size_t linear_growth_threshold_ = 128 * 1024 * 1024)
+          size_t linear_growth_threshold_ = -1)
             : growth_factor(growth_factor_),
-              linear_growth_threshold(linear_growth_threshold_),
               head(new Chunk(initial_size_, nullptr)),
-              size_in_bytes(head->size()) {}
+              size_in_bytes(head->size()) {
+        if (linear_growth_threshold_ < 0) {
+            linear_growth_threshold = config::memory_linear_growth_threshold;

Review Comment:
   Have tested the impact of different values of `memory_linear_growth_threshold` on performance?
   Is it necessary to dynamically set



##########
be/src/runtime/exec_env_init.cpp:
##########
@@ -199,9 +201,7 @@ Status ExecEnv::_init_mem_tracker() {
     if (global_memory_limit_bytes > MemInfo::physical_mem()) {
         LOG(WARNING) << "Memory limit "
                      << PrettyPrinter::print(global_memory_limit_bytes, TUnit::BYTES)
-                     << " exceeds physical memory of "
-                     << PrettyPrinter::print(MemInfo::physical_mem(), TUnit::BYTES)
-                     << ". Using physical memory instead";

Review Comment:
   recommend to restore, make error messages more obvious



##########
be/src/vec/common/pod_array.h:
##########
@@ -120,8 +122,9 @@ class PODArrayBase : private boost::noncopyable,
         }
     }
 
+    /// Not round up, keep the size just as the application pass in like std::vector
     void alloc_for_num_elements(size_t num_elements) {
-        alloc(round_up_to_power_of_two_or_zero(minimum_memory_for_elements(num_elements)));
+        alloc(minimum_memory_for_elements(num_elements));

Review Comment:
   Allocating in powers of 2 has a positive impact on performance, if you wish to reduce memory usage,
   
   join `#ifndef STRICT_MEMORY_USE`, similar to `hash_table.h` expansion



##########
be/src/runtime/memory/chunk_allocator.cpp:
##########
@@ -204,7 +205,7 @@ Status ChunkAllocator::allocate(size_t size, Chunk* chunk) {
 void ChunkAllocator::free(const Chunk& chunk) {
     DCHECK(chunk.core_id != -1);
     CHECK((chunk.size & (chunk.size - 1)) == 0);
-    if (config::disable_mem_pools) {
+    if (config::disable_chunk_allocator) {

Review Comment:
   Is it better to move the condition into `MemPool` and rename `disable_chunk_allocator_in_mem_pool`



##########
be/src/vec/common/pod_array.h:
##########
@@ -229,7 +235,7 @@ class PODArrayBase : private boost::noncopyable,
     template <typename... TAllocatorParams>
     void reserve(size_t n, TAllocatorParams&&... allocator_params) {
         if (n > capacity())
-            realloc(round_up_to_power_of_two_or_zero(minimum_memory_for_elements(n)),
+            realloc(minimum_memory_for_elements(n),

Review Comment:
   Ditto



##########
be/src/runtime/mem_pool.cpp:
##########
@@ -131,8 +131,9 @@ Status MemPool::find_chunk(size_t min_size, bool check_limits) {
         DCHECK_GE(next_chunk_size_, INITIAL_CHUNK_SIZE);
         chunk_size = std::max<size_t>(min_size, next_chunk_size_);
     }
-
-    chunk_size = BitUtil::RoundUpToPowerOfTwo(chunk_size);

Review Comment:
   chunk must be a power of 2 for the reasons:
   https://github.com/apache/doris/pull/11040



##########
be/src/common/config.h:
##########
@@ -439,14 +439,20 @@ CONF_Bool(disable_mem_pools, "false");
 // increase this variable can improve performance,
 // but will acquire more free memory which can not be used by other modules.
 CONF_mString(chunk_reserved_bytes_limit, "10%");
-// 1024, The minimum chunk allocator size (in bytes)
-CONF_Int32(min_chunk_reserved_bytes, "1024");
+
+// Whether using chunk allocator to cache memory chunk
+CONF_Bool(disable_chunk_allocator, "true");

Review Comment:
   `disable_chunk_allocator_in_mem_pool`



##########
be/src/runtime/mem_pool.h:
##########
@@ -231,9 +231,9 @@ class MemPool {
         // I refers to https://github.com/mcgov/asan_alignment_example.
 
         ChunkInfo& info = chunks_[current_chunk_idx_];
-        int64_t aligned_allocated_bytes =
-                BitUtil::RoundUpToPowerOf2(info.allocated_bytes + DEFAULT_PADDING_SIZE, alignment);
-        if (aligned_allocated_bytes + size <= info.chunk.size) {
+        int64_t aligned_allocated_bytes = BitUtil::RoundUpToMultiplyOfFactor(
+                info.allocated_bytes + DEFAULT_PADDING_SIZE, alignment);
+        if (aligned_allocated_bytes + size + DEFAULT_PADDING_SIZE <= info.chunk.size) {

Review Comment:
   Reason for removing `DEFAULT_PADDING_SIZE`?
   This helps with performance



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

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