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/11 08:07:32 UTC

[GitHub] [doris] yiguolei opened a new pull request, #13285: [improvement](memory) disable page cache and chunk allocator, optimize memory allocate size

yiguolei opened a new pull request, #13285:
URL: https://github.com/apache/doris/pull/13285

   
   
   # Proposed changes
   1. disable page cache by default
   2. disable chunk allocator by default
   3. not use chunk allocator for vectorized allocator by default
   4. add a new config memory_linear_growth_threshold = 128Mb, not allocate memory by RoundUpToPowerOf2 if the allocated size is larger than this threshold. This config is added to MemPool, ChunkAllocator, PodArray, Arena.
   
   ## Problem summary
   
   Describe your changes.
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: 
       - [ ] Yes
       - [ ] No
       - [ ] I don't know
   5. Has unit tests been added:
       - [ ] Yes
       - [ ] No
       - [ ] No Need
   6. Has document been added or modified:
       - [ ] Yes
       - [ ] No
       - [ ] No Need
   7. Does it need to update dependencies:
       - [ ] Yes
       - [ ] No
   8. Are there any changes that cannot be rolled back:
       - [ ] Yes (If Yes, please explain WHY)
       - [ ] No
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto: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.

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


[GitHub] [doris] github-actions[bot] commented on pull request #13285: [improvement](memory) disable page cache and chunk allocator, optimize memory allocate size

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13285:
URL: https://github.com/apache/doris/pull/13285#issuecomment-1279704272

   PR approved by at least one committer and no changes requested.


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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz merged PR #13285:
URL: https://github.com/apache/doris/pull/13285


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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on PR #13285:
URL: https://github.com/apache/doris/pull/13285#issuecomment-1275693556

   > Gives the performance impact of `disable page cache` and `disable chunk allocator`, and the benefits of doing so, such as increasing OOM risk because they have no gc.
   
   1. Currently, we use a hard limit for page cache(20%) or chunk allocator(10%), if the application need memory, there is no gc mechanism to collect memory from page cache or chunk allocator.
   2. page cache is very useful in some cases such as POC test, run benchmarks, some reporting secenarios. But in some cases like adhoc query, etl query, it is not very useful. And we have disable page cache in 1.1 and many company have disable page cache online such as xiaomi.
   3. chunk allocator impacts about 10% performance in clickbench test. 
   
   Doris's memory usage is not very stable, I want to disable them first, and try to fix other memory problems and then open them again. This may need about 2 months and there are many releases during this stage, so that I disable them  like we have done in branch 1.1-lts.
   
   
   


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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r994608055


##########
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:
   mem info initialization and `_init_mem_tracker` are both called when be start,
   It seems better to print when mem info is initialized



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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r995470080


##########
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:
   Try removing `disable_chunk_allocator_in_vec` and replace with `disable_chunk_allocator`. (more detailed config comments)



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


[GitHub] [doris] github-actions[bot] commented on pull request #13285: [improvement](memory) disable page cache and chunk allocator, optimize memory allocate size

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13285:
URL: https://github.com/apache/doris/pull/13285#issuecomment-1279704279

   PR approved by anyone and no changes requested.


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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r994512182


##########
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:
   There are many if elses to print physical memory, sometimes it is not print. I think we should always print physical memory size not depend on other conditions.



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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on PR #13285:
URL: https://github.com/apache/doris/pull/13285#issuecomment-1276965810

   > > Gives the performance impact of `disable page cache` and `disable chunk allocator`, and the benefits of doing so, such as increasing OOM risk because they have no gc.
   > 
   > 1. Currently, we use a hard limit for page cache(20%) or chunk allocator(10%), if the application need memory, there is no gc mechanism to collect memory from page cache or chunk allocator.
   > 2. page cache is very useful in some cases such as POC test, run benchmarks, some reporting secenarios. But in some cases like adhoc query, etl query, it is not very useful. And we have disable page cache in 1.1 and many company have disable page cache online such as xiaomi.
   > 3. chunk allocator impacts about 10% performance in clickbench test.
   > 
   > Doris's memory usage is not very stable, I want to disable them first, and try to fix other memory problems and then open them again. This may need about 2 months and there are many releases during this stage, so that I disable them like we have done in branch 1.1-lts.
   
   I agree, `page cache` and `chunk allocator` may hide memory issues on the code.
   
   Later we can find a time to talk about the use of the cache


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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r995284788


##########
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:
   yes...



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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r994508227


##########
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`, Similar to `disable_chunk_allocator_in_vec`



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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r994608055


##########
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:
   Both mem info initialization and `_init_mem_tracker` are running,
   It seems better to print when mem info is initialized



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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r995278670


##########
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:
   No, so I just keep the default value to 128Mb, the same with the previous value.
   It needs to be a config, because we have also have mempool, they should use the same config. 



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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r996240764


##########
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:
   OK, I remove it.



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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on PR #13285:
URL: https://github.com/apache/doris/pull/13285#issuecomment-1274867621

   If want to discard the doris application layer cache. what is the replacement method?
   such as, using optimized code + general memory allocator + system page cache
   
   There are more application layer caches in doris, such as segment cache.
   
   I think a self-managed application layer cache is necessary.
   I think in the end, an efficient unified application layer cache, rather than relying entirely on the system.


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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r995281892


##########
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:
   I do not think this should be a config, because if it is a config, we do not know when to open the config since it is a macro. Actually, there are two types of memory allocation in PODArray:
   1. reserve, sometimes the developer know the expected size of the array, then he should call reserve method to allocate the EXPECTED memory.
   2. push_back, the developer does not know the expected size of the array, then he just call push back to allocate memory. In this scenario, we should allocate memory using power of 2.
   
   For most cases, we should reserve or resize memory size before push back, then we could reduce memory reallocation or memory copy. 
   This PR try to fix some problems. https://github.com/apache/doris/pull/13088. 



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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r995284110


##########
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:
    I did not remove DEFAULT_PADDING_SIZE...., JUST ensure there will be enough memory left in chunk.



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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r995444935


##########
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:
   I see wrong = =



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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on PR #13285:
URL: https://github.com/apache/doris/pull/13285#issuecomment-1274895854

   Maybe we can discuss the idea of subsequent cache optimization~, I mentioned a proposal a long time ago
   https://github.com/apache/doris/issues/9580


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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r995282920


##########
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:
   No, I will remove mempool after we removed non-vectorized engine. MemPool is used as MemPool.cpp, it is like a arena. The config disable_mem_pools is also very confused. I will remove 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.

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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r995443142


##########
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:
   can remove min_chunk_reserved_bytes, const 4096 is fine, the user will not modify it
   This knowledge suggests~



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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on PR #13285:
URL: https://github.com/apache/doris/pull/13285#issuecomment-1276965623

   I agree, `page cache` and `chunk allocator` may hide memory issues on the code.
   
   Later we can find a time to talk about the use of the cache


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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r994512598


##########
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:
   Not need, if we print memory info at the beginning, this message is useless.



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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r994469725


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



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


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

Posted by GitBox <gi...@apache.org>.
xinyiZzz commented on PR #13285:
URL: https://github.com/apache/doris/pull/13285#issuecomment-1274832649

   Give the performance impact of `disable page cache` and `disable chunk allocator`, and the benefits of doing so.


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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r995283645


##########
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:
   Not use this. I will remove disable_mem_pools in the future after non-vectorized engine is removed.  It is very confused with MemPool. 



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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r995285283


##########
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:
   yes ... I will move back min_chunk_reserved_bytes



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


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

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #13285:
URL: https://github.com/apache/doris/pull/13285#discussion_r995276788


##########
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:
   Yes.  you are right.



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