This is an automated email from the ASF dual-hosted git repository.

yiguolei pushed a commit to branch branch-4.0
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/branch-4.0 by this push:
     new ed96af8d197 branch-4.0: Revert "[enhance](memtable) support adaptive 
memtable write buffer size (#56948)" (#59926)
ed96af8d197 is described below

commit ed96af8d197854a5708f638df43df9c23a7cbb76
Author: hui lai <[email protected]>
AuthorDate: Fri Jan 16 15:04:17 2026 +0800

    branch-4.0: Revert "[enhance](memtable) support adaptive memtable write 
buffer size (#56948)" (#59926)
    
    This reverts commit 0a2b8d8a9af5dcf39900002b5dc2b9fc0c8483a2.
    
    ### What problem does this PR solve?
    
    Issue Number: close #xxx
    
    Related PR: #xxx
    
    Problem Summary:
    
    ### Release note
    
    None
    
    ### Check List (For Author)
    
    - Test <!-- At least one of them must be included. -->
        - [ ] Regression test
        - [ ] Unit Test
        - [ ] Manual test (add detailed scripts or steps below)
        - [ ] No need to test or manual test. Explain why:
    - [ ] This is a refactor/code format and no logic has been changed.
            - [ ] Previous test can cover this change.
            - [ ] No code files have been changed.
            - [ ] Other reason <!-- Add your reason?  -->
    
    - Behavior changed:
        - [ ] No.
        - [ ] Yes. <!-- Explain the behavior change -->
    
    - Does this need documentation?
        - [ ] No.
    - [ ] Yes. <!-- Add document PR link here. eg:
    https://github.com/apache/doris-website/pull/1214 -->
    
    ### Check List (For Reviewer who merge this PR)
    
    - [ ] Confirm the release note
    - [ ] Confirm test cases
    - [ ] Confirm document
    - [ ] Add branch pick label <!-- Add branch pick label that this PR
    should merge into -->
---
 be/src/common/config.cpp                           |  1 -
 be/src/common/config.h                             |  1 -
 be/src/olap/memtable.cpp                           | 21 +---------
 be/src/olap/memtable.h                             |  6 ---
 be/src/olap/memtable_writer.cpp                    | 24 +++--------
 be/src/olap/memtable_writer.h                      |  2 -
 .../memtable/test_memtable_too_many_rows.groovy    | 49 ----------------------
 7 files changed, 6 insertions(+), 98 deletions(-)

diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp
index 00f103bd4fb..1cf2bddad3e 100644
--- a/be/src/common/config.cpp
+++ b/be/src/common/config.cpp
@@ -683,7 +683,6 @@ DEFINE_mInt32(memory_gc_sleep_time_ms, "500");
 
 // max write buffer size before flush, default 200MB
 DEFINE_mInt64(write_buffer_size, "209715200");
-DEFINE_mBool(enable_adaptive_write_buffer_size, "true");
 // max buffer size used in memtable for the aggregated table, default 400MB
 DEFINE_mInt64(write_buffer_size_for_agg, "104857600");
 DEFINE_mInt64(min_write_buffer_size_for_partial_update, "1048576");
diff --git a/be/src/common/config.h b/be/src/common/config.h
index 3da56c30523..f7d1ccab1c6 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -725,7 +725,6 @@ DECLARE_mInt32(memory_gc_sleep_time_ms);
 
 // max write buffer size before flush, default 200MB
 DECLARE_mInt64(write_buffer_size);
-DECLARE_mBool(enable_adaptive_write_buffer_size);
 // max buffer size used in memtable for the aggregated table, default 400MB
 DECLARE_mInt64(write_buffer_size_for_agg);
 
diff --git a/be/src/olap/memtable.cpp b/be/src/olap/memtable.cpp
index c8d2bc86c84..a5481d39a4f 100644
--- a/be/src/olap/memtable.cpp
+++ b/be/src/olap/memtable.cpp
@@ -81,7 +81,6 @@ MemTable::MemTable(int64_t tablet_id, 
std::shared_ptr<TabletSchema> tablet_schem
     _init_columns_offset_by_slot_descs(slot_descs, tuple_desc);
     // TODO: Support ZOrderComparator in the future
     _row_in_blocks = 
std::make_unique<DorisVector<std::shared_ptr<RowInBlock>>>();
-    _load_mem_limit = MemInfo::mem_limit() * 
config::load_process_max_memory_limit_percent / 100;
 }
 
 void MemTable::_init_columns_offset_by_slot_descs(const 
std::vector<SlotDescriptor*>* slot_descs,
@@ -657,7 +656,7 @@ void MemTable::shrink_memtable_by_agg() {
 
 bool MemTable::need_flush() const {
     DBUG_EXECUTE_IF("MemTable.need_flush", { return true; });
-    auto max_size = _adaptive_write_buffer_size();
+    auto max_size = config::write_buffer_size;
     if (_partial_update_mode == UniqueKeyUpdateModePB::UPDATE_FIXED_COLUMNS) {
         auto update_columns_size = _num_columns;
         auto min_buffer_size = 
config::min_write_buffer_size_for_partial_update;
@@ -672,24 +671,6 @@ bool MemTable::need_flush() const {
     return false;
 }
 
-int64_t MemTable::_adaptive_write_buffer_size() const {
-    if (!config::enable_adaptive_write_buffer_size) [[unlikely]] {
-        return config::write_buffer_size;
-    }
-    const int64_t current_load_mem_value = MemoryProfile::load_current_usage();
-    int64_t factor = 4;
-    // Memory usage intervals:
-    // (80 %, 100 %] → 1× buffer
-    // (50 %, 80 %]  → 2× buffer
-    // [0 %, 50 %]   → 4× buffer
-    if (current_load_mem_value > (_load_mem_limit * 4) / 5) { // > 80 %
-        factor = 1;
-    } else if (current_load_mem_value > _load_mem_limit / 2) { // > 50 %
-        factor = 2;
-    }
-    return config::write_buffer_size * factor;
-}
-
 bool MemTable::need_agg() const {
     if (_keys_type == KeysType::AGG_KEYS) {
         auto max_size = _last_agg_pos + config::write_buffer_size_for_agg;
diff --git a/be/src/olap/memtable.h b/be/src/olap/memtable.h
index 47426e354d0..5cd70e812a9 100644
--- a/be/src/olap/memtable.h
+++ b/be/src/olap/memtable.h
@@ -20,7 +20,6 @@
 #include <stddef.h>
 #include <stdint.h>
 
-#include <cstdint>
 #include <cstring>
 #include <functional>
 #include <memory>
@@ -205,8 +204,6 @@ public:
 
     void update_mem_type(MemType memtype) { _mem_type = memtype; }
 
-    int64_t raw_rows() { return _stat.raw_rows.load(); }
-
 private:
     // for vectorized
     template <bool has_skip_bitmap_col>
@@ -216,8 +213,6 @@ private:
     // Used to wrapped by to_block to do exception handle logic
     Status _to_block(std::unique_ptr<vectorized::Block>* res);
 
-    int64_t _adaptive_write_buffer_size() const;
-
 private:
     std::atomic<MemType> _mem_type;
     int64_t _tablet_id;
@@ -236,7 +231,6 @@ private:
     // In this way, we can make MemTable::memory_usage() to be more accurate, 
and eventually
     // reduce the number of segment files that are generated by current load
     vectorized::Arena _arena;
-    int64_t _load_mem_limit = -1;
 
     void _init_columns_offset_by_slot_descs(const 
std::vector<SlotDescriptor*>* slot_descs,
                                             const TupleDescriptor* tuple_desc);
diff --git a/be/src/olap/memtable_writer.cpp b/be/src/olap/memtable_writer.cpp
index 12f0c070ade..808eaf9eec8 100644
--- a/be/src/olap/memtable_writer.cpp
+++ b/be/src/olap/memtable_writer.cpp
@@ -105,15 +105,6 @@ Status MemTableWriter::write(const vectorized::Block* 
block,
                                              _req.tablet_id, 
_req.load_id.hi(), _req.load_id.lo());
     }
 
-    // Flush and reset memtable if it is raw rows great than int32_t.
-    int64_t raw_rows = _mem_table->raw_rows();
-    DBUG_EXECUTE_IF("MemTableWriter.too_many_raws",
-                    { raw_rows = std::numeric_limits<int32_t>::max(); });
-    if (raw_rows + row_idxs.size() > std::numeric_limits<int32_t>::max()) {
-        g_flush_cuz_rowscnt_oveflow << 1;
-        RETURN_IF_ERROR(_flush_memtable());
-    }
-
     _total_received_rows += row_idxs.size();
     auto st = _mem_table->insert(block, row_idxs);
 
@@ -138,21 +129,16 @@ Status MemTableWriter::write(const vectorized::Block* 
block,
         _mem_table->shrink_memtable_by_agg();
     }
     if (UNLIKELY(_mem_table->need_flush())) {
-        RETURN_IF_ERROR(_flush_memtable());
+        auto s = _flush_memtable_async();
+        _reset_mem_table();
+        if (UNLIKELY(!s.ok())) {
+            return s;
+        }
     }
 
     return Status::OK();
 }
 
-Status MemTableWriter::_flush_memtable() {
-    auto s = _flush_memtable_async();
-    _reset_mem_table();
-    if (UNLIKELY(!s.ok())) {
-        return s;
-    }
-    return Status::OK();
-}
-
 Status MemTableWriter::_flush_memtable_async() {
     DCHECK(_flush_token != nullptr);
     std::shared_ptr<MemTable> memtable;
diff --git a/be/src/olap/memtable_writer.h b/be/src/olap/memtable_writer.h
index 7465e79a452..aa1fd4025ed 100644
--- a/be/src/olap/memtable_writer.h
+++ b/be/src/olap/memtable_writer.h
@@ -25,7 +25,6 @@
 #include <cstdint>
 #include <memory>
 #include <mutex>
-#include <random>
 #include <vector>
 
 #include "common/status.h"
@@ -110,7 +109,6 @@ public:
     }
 
 private:
-    Status _flush_memtable();
     // push a full memtable to flush executor
     Status _flush_memtable_async();
 
diff --git 
a/regression-test/suites/fault_injection_p0/memtable/test_memtable_too_many_rows.groovy
 
b/regression-test/suites/fault_injection_p0/memtable/test_memtable_too_many_rows.groovy
deleted file mode 100644
index 38516864727..00000000000
--- 
a/regression-test/suites/fault_injection_p0/memtable/test_memtable_too_many_rows.groovy
+++ /dev/null
@@ -1,49 +0,0 @@
-// 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.
-
-suite("test_memtable_too_many_rows", "nonConcurrent") {
-    GetDebugPoint().clearDebugPointsForAllBEs()
-    def testTable = "test_memtable_too_many_rows"
-    sql """ DROP TABLE IF EXISTS ${testTable}"""
-
-    sql """
-        CREATE TABLE IF NOT EXISTS `${testTable}` (
-          `id` BIGINT NOT NULL,
-          `value` int(11) NOT NULL
-        ) ENGINE=OLAP
-        DUPLICATE KEY(`id`)
-        COMMENT "OLAP"
-        DISTRIBUTED BY HASH(`id`) BUCKETS 1
-        PROPERTIES (
-        "replication_allocation" = "tag.location.default: 1"
-        )
-    """
-
-    def debugPoint = "MemTableWriter.too_many_raws"
-    try {
-        GetDebugPoint().enableDebugPointForAllBEs(debugPoint)
-        sql "insert into ${testTable} values(1,1)"
-        def res = sql "select * from ${testTable}"
-        logger.info("res: " + res.size())
-        assertTrue(res.size() == 1)
-    } catch (Exception e){
-        logger.info(e.getMessage())
-        assertTrue(e.getMessage().contains("write memtable too many rows 
fail"))
-    } finally {
-        GetDebugPoint().disableDebugPointForAllBEs(debugPoint)
-    }
-}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to