You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by zh...@apache.org on 2019/10/18 06:33:17 UTC

[incubator-doris] branch replace-arena created (now 7cc8d32)

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

zhaoc pushed a change to branch replace-arena
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git.


      at 7cc8d32  Replace Arena with MemPool

This branch includes the following new commits:

     new 7cc8d32  Replace Arena with MemPool

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



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


[incubator-doris] 01/01: Replace Arena with MemPool

Posted by zh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

zhaoc pushed a commit to branch replace-arena
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git

commit 7cc8d3243a42679a89e64e62f4e658d19a5c1540
Author: zhaochun <zh...@baidu.com>
AuthorDate: Fri Oct 18 14:32:41 2019 +0800

    Replace Arena with MemPool
---
 LICENSE.txt                                        | 16 -----
 be/src/olap/aggregate_func.h                       |  4 +-
 be/src/olap/column_block.h                         | 13 ++--
 be/src/olap/field.h                                | 40 ++++-------
 be/src/olap/generic_iterators.cpp                  |  3 +-
 be/src/olap/key_coder.h                            | 26 +++----
 be/src/olap/row.h                                  | 10 ---
 be/src/olap/row_block2.cpp                         |  3 +-
 be/src/olap/row_block2.h                           | 17 +++--
 be/src/olap/rowset/segment_v2/binary_dict_page.cpp | 11 +--
 be/src/olap/rowset/segment_v2/binary_dict_page.h   |  7 +-
 be/src/olap/rowset/segment_v2/binary_plain_page.h  |  4 +-
 be/src/olap/rowset/segment_v2/column_reader.h      |  7 +-
 be/src/olap/rowset/segment_v2/column_zone_map.cpp  | 10 +--
 be/src/olap/rowset/segment_v2/column_zone_map.h    |  9 ++-
 be/src/olap/rowset/segment_v2/segment_iterator.cpp |  4 +-
 be/src/olap/skiplist.h                             |  2 +-
 be/src/olap/types.cpp                              |  1 -
 be/src/olap/types.h                                | 20 ------
 be/src/util/CMakeLists.txt                         |  1 -
 be/src/util/arena.cpp                              | 68 ------------------
 be/src/util/arena.h                                | 74 --------------------
 be/test/olap/delta_writer_test.cpp                 | 15 ++--
 be/test/olap/key_coder_test.cpp                    | 29 ++++----
 .../rowset/segment_v2/binary_dict_page_test.cpp    | 16 +++--
 .../rowset/segment_v2/binary_plain_page_test.cpp   | 18 ++---
 .../rowset/segment_v2/bitshuffle_page_test.cpp     | 17 +++--
 .../segment_v2/column_reader_writer_test.cpp       | 24 ++++---
 .../segment_v2/frame_of_reference_page_test.cpp    | 17 +++--
 be/test/olap/rowset/segment_v2/plain_page_test.cpp | 17 +++--
 be/test/olap/rowset/segment_v2/rle_page_test.cpp   | 18 ++---
 be/test/olap/rowset/segment_v2/segment_test.cpp    | 17 +++--
 be/test/olap/tablet_schema_helper.h                |  8 ++-
 be/test/util/CMakeLists.txt                        |  1 -
 be/test/util/arena_test.cpp                        | 81 ----------------------
 run-ut.sh                                          |  1 -
 36 files changed, 191 insertions(+), 438 deletions(-)

diff --git a/LICENSE.txt b/LICENSE.txt
index 636a419..e224581 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -414,22 +414,6 @@ be/src/olap/skiplist.h : BSD-style license
 
 --------------------------------------------------------------------------------
 
-be/src/util/arena.h : BSD-style license
-
-  Copyright (c) 2011 The LevelDB Authors. All rights reserved.
-  Use of this source code is governed by a BSD-style license that can be
-  found in the LICENSE file. See the AUTHORS file for names of contributors.
-
---------------------------------------------------------------------------------
-
-be/src/util/arena.cpp : BSD-style license
-
-  Copyright (c) 2011 The LevelDB Authors. All rights reserved.
-  Use of this source code is governed by a BSD-style license that can be
-  found in the LICENSE file. See the AUTHORS file for names of contributors.
-
---------------------------------------------------------------------------------
-
 be/src/olap/new_status.h : BSD-style license
 
   Copyright (c) 2011 The LevelDB Authors. All rights reserved.
diff --git a/be/src/olap/aggregate_func.h b/be/src/olap/aggregate_func.h
index 17485ba..3540581 100644
--- a/be/src/olap/aggregate_func.h
+++ b/be/src/olap/aggregate_func.h
@@ -42,7 +42,7 @@ public:
     // Init function will initialize aggregation execute environment in dst with source
     // and convert the source raw data to storage aggregate format
     //
-    // Memory Note: For plain memory can be allocated from arena, whose lifetime
+    // Memory Note: For plain memory can be allocated from *mem_pool, whose lifetime
     // will last util finalize function is called. Memory allocated from heap should
     // be freed in finalize functioin to avoid memory leak.
     inline void init(RowCursorCell* dst, const char* src, bool src_null, MemPool* mem_pool, ObjectPool* agg_pool) const {
@@ -66,7 +66,7 @@ public:
     // saved in src.
     //
     // Memory Note: All heap memory allocated in init and update function should be freed
-    // before this function return. Memory allocated from arena will be still available
+    // before this function return. Memory allocated from *mem_pool will be still available
     // and will be freed by client.
     inline void finalize(RowCursorCell* src, MemPool* mem_pool) const {
         _finalize_fn(src, mem_pool);
diff --git a/be/src/olap/column_block.h b/be/src/olap/column_block.h
index 4fc5403..37f61ad 100644
--- a/be/src/olap/column_block.h
+++ b/be/src/olap/column_block.h
@@ -24,23 +24,22 @@
 
 namespace doris {
 
-class Arena;
+class MemPool;
 class TypeInfo;
-
 class ColumnBlockCell;
 
 // Block of data belong to a single column.
 // It doesn't own any data, user should keep the life of input data.
 class ColumnBlock {
 public:
-    ColumnBlock(const TypeInfo* type_info, uint8_t* data, uint8_t* null_bitmap, Arena* arena)
-        : _type_info(type_info), _data(data), _null_bitmap(null_bitmap), _arena(arena) { }
+    ColumnBlock(const TypeInfo* type_info, uint8_t* data, uint8_t* null_bitmap, MemPool* pool)
+        : _type_info(type_info), _data(data), _null_bitmap(null_bitmap), _pool(pool) { }
 
     const TypeInfo* type_info() const { return _type_info; }
     uint8_t* data() const { return _data; }
     uint8_t* null_bitmap() const { return _null_bitmap; }
     bool is_nullable() const { return _null_bitmap != nullptr; }
-    Arena* arena() const { return _arena; }
+    MemPool* pool() const { return _pool; }
     const uint8_t* cell_ptr(size_t idx) const { return _data + idx * _type_info->size(); }
     uint8_t* mutable_cell_ptr(size_t idx) const { return _data + idx * _type_info->size(); }
     bool is_null(size_t idx) const {
@@ -57,7 +56,7 @@ private:
     const TypeInfo* _type_info;
     uint8_t* _data;
     uint8_t* _null_bitmap;
-    Arena* _arena;
+    MemPool* _pool;
 };
 
 struct ColumnBlockCell {
@@ -86,7 +85,7 @@ public:
     void advance(size_t skip) { _row_offset += skip; }
     size_t first_row_index() const { return _row_offset; }
     ColumnBlock* column_block() { return _block; }
-    Arena* arena() const { return _block->arena(); }
+    MemPool* pool() const { return _block->pool(); }
     void set_null_bits(size_t num_rows, bool val) {
         BitmapChangeBits(_block->null_bitmap(), _row_offset, num_rows, val);
     }
diff --git a/be/src/olap/field.h b/be/src/olap/field.h
index ad51377..d128711 100644
--- a/be/src/olap/field.h
+++ b/be/src/olap/field.h
@@ -57,7 +57,10 @@ public:
 
     virtual inline void set_to_max(char* buf) const { return _type_info->set_to_max(buf); }
     inline void set_to_min(char* buf) const { return _type_info->set_to_min(buf); }
-    virtual inline char* allocate_value_from_arena(Arena* arena) const { return arena->Allocate(_type_info->size()); }
+
+    // This function allocate memory from pool, other than allocate_memory
+    // reserve memory from continuous memory.
+    virtual inline char* allocate_value(MemPool* pool) const { return (char*)pool->allocate(_type_info->size()); }
 
     inline void agg_update(RowCursorCell* dest, const RowCursorCell& src, MemPool* mem_pool = nullptr) const {
         _agg_info->update(dest, src, mem_pool);
@@ -172,18 +175,6 @@ public:
         _type_info->deep_copy(dst->mutable_cell_ptr(), src.cell_ptr(), pool);
     }
 
-    template<typename DstCellType, typename SrcCellType>
-    void deep_copy(DstCellType* dst,
-                   const SrcCellType& src,
-                   Arena* arena) const {
-        bool is_null = src.is_null();
-        dst->set_is_null(is_null);
-        if (is_null) {
-            return;
-        }
-        _type_info->deep_copy_with_arena(dst->mutable_cell_ptr(), src.cell_ptr(), arena);
-    }
-
     // deep copy filed content from `src` to `dst` without null-byte
     inline void deep_copy_content(char* dst, const char* src, MemPool* mem_pool) const {
         _type_info->deep_copy(dst, src, mem_pool);
@@ -195,11 +186,6 @@ public:
         _type_info->shallow_copy(dst, src);
     }
 
-    // copy filed content from src to dest without nullbyte
-    inline void deep_copy_content(char* dest, const char* src, Arena* arena) const {
-        _type_info->deep_copy_with_arena(dest, src, arena);
-    }
-
     // Copy srouce content to destination in index format.
     template<typename DstCellType, typename SrcCellType>
     void to_index(DstCellType* dst, const SrcCellType& src) const;
@@ -240,8 +226,8 @@ public:
         _key_coder->encode_ascending(value, _index_size, buf);
     }
     
-    Status decode_ascending(Slice* encoded_key, uint8_t* cell_ptr, Arena* arena) const {
-        return _key_coder->decode_ascending(encoded_key, _index_size, cell_ptr, arena);
+    Status decode_ascending(Slice* encoded_key, uint8_t* cell_ptr, MemPool* pool) const {
+        return _key_coder->decode_ascending(encoded_key, _index_size, cell_ptr, pool);
     }
 private:
     // Field的最大长度,单位为字节,通常等于length, 变长字符串不同
@@ -256,11 +242,11 @@ protected:
     // 除字符串外,其它类型都是确定的
     uint32_t _length;
 
-    char* allocate_string_value_from_arena(Arena* arena) const {
-        char* type_value = arena->Allocate(sizeof(Slice));
+    char* allocate_string_value(MemPool* pool) const {
+        char* type_value = (char*)pool->allocate(sizeof(Slice));
         auto slice = reinterpret_cast<Slice*>(type_value);
         slice->size = _length;
-        slice->data = arena->Allocate(slice->size);
+        slice->data = (char*)pool->allocate(slice->size);
         return type_value;
     };
 };
@@ -383,8 +369,8 @@ public:
         return new CharField(*this);
     }
 
-    char* allocate_value_from_arena(Arena* arena) const override {
-        return Field::allocate_string_value_from_arena(arena);
+    char* allocate_value(MemPool* pool) const override {
+        return Field::allocate_string_value(pool);
     }
 
     void set_to_max(char* ch) const override {
@@ -416,8 +402,8 @@ public:
         return new VarcharField(*this);
     }
 
-    char* allocate_value_from_arena(Arena* arena) const override {
-        return Field::allocate_string_value_from_arena(arena);
+    char* allocate_value(MemPool* pool) const override {
+        return Field::allocate_string_value(pool);
     }
 
     void set_to_max(char* ch) const override {
diff --git a/be/src/olap/generic_iterators.cpp b/be/src/olap/generic_iterators.cpp
index f9630a2..6970973 100644
--- a/be/src/olap/generic_iterators.cpp
+++ b/be/src/olap/generic_iterators.cpp
@@ -22,7 +22,6 @@
 #include "olap/row_block2.h"
 #include "olap/row_cursor_cell.h"
 #include "olap/row.h"
-#include "util/arena.h"
 
 namespace doris {
 
@@ -258,7 +257,7 @@ Status MergeIterator::next_batch(RowBlockV2* block) {
 
         RowBlockRow dst_row = block->row(row_idx);
         // copy current row to block
-        copy_row(&dst_row, ctx->current_row(), block->arena());
+        copy_row(&dst_row, ctx->current_row(), block->pool());
 
         RETURN_IF_ERROR(ctx->advance());
         if (ctx->valid()) {
diff --git a/be/src/olap/key_coder.h b/be/src/olap/key_coder.h
index 773c4e9..5c79fec 100644
--- a/be/src/olap/key_coder.h
+++ b/be/src/olap/key_coder.h
@@ -24,14 +24,14 @@
 #include "gutil/endian.h"
 #include "gutil/strings/substitute.h"
 #include "olap/types.h"
-#include "util/arena.h"
+#include "runtime/mem_pool.h"
 
 namespace doris {
 
 using strings::Substitute;
 
 using EncodeAscendingFunc = void (*)(const void* value, size_t index_size, std::string* buf);
-using DecodeAscendingFunc = Status (*)(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr, Arena* arena);
+using DecodeAscendingFunc = Status (*)(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr, MemPool* pool);
 
 // Helper class that is used to encode types of value in memory format
 // into a sorted binary. For example, this class will encode unsigned
@@ -44,8 +44,8 @@ public:
     void encode_ascending(const void* value, size_t index_size, std::string* buf) const {
         _encode_ascending(value, index_size, buf);
     }
-    Status decode_ascending(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr, Arena* arena) const {
-        return _decode_ascending(encoded_key, index_size, cell_ptr, arena);
+    Status decode_ascending(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr, MemPool* pool) const {
+        return _decode_ascending(encoded_key, index_size, cell_ptr, pool);
     }
 
 private:
@@ -97,7 +97,7 @@ public:
     }
 
     static Status decode_ascending(Slice* encoded_key, size_t index_size,
-                                   uint8_t* cell_ptr, Arena* arena) {
+                                   uint8_t* cell_ptr, MemPool* pool) {
         if (encoded_key->size < sizeof(UnsignedCppType)) {
             return Status::InvalidArgument(
                 Substitute("Key too short, need=$0 vs real=$1",
@@ -131,7 +131,7 @@ public:
     }
 
     static Status decode_ascending(Slice* encoded_key, size_t index_size,
-                                   uint8_t* cell_ptr, Arena* arena) {
+                                   uint8_t* cell_ptr, MemPool* pool) {
         if (encoded_key->size < sizeof(UnsignedCppType)) {
             return Status::InvalidArgument(
                 Substitute("Key too short, need=$0 vs real=$1",
@@ -161,12 +161,12 @@ public:
     }
 
     static Status decode_ascending(Slice* encoded_key, size_t index_size,
-                                   uint8_t* cell_ptr, Arena* arena) {
+                                   uint8_t* cell_ptr, MemPool* pool) {
         decimal12_t decimal_val;
         RETURN_IF_ERROR(KeyCoderTraits<OLAP_FIELD_TYPE_BIGINT>::decode_ascending(
-                encoded_key, sizeof(decimal_val.integer), (uint8_t*)&decimal_val.integer, arena));
+                encoded_key, sizeof(decimal_val.integer), (uint8_t*)&decimal_val.integer, pool));
         RETURN_IF_ERROR(KeyCoderTraits<OLAP_FIELD_TYPE_INT>::decode_ascending(
-                encoded_key, sizeof(decimal_val.fraction), (uint8_t*)&decimal_val.fraction, arena));
+                encoded_key, sizeof(decimal_val.fraction), (uint8_t*)&decimal_val.fraction, pool));
         memcpy(cell_ptr, &decimal_val, sizeof(decimal12_t));
         return Status::OK();
     }
@@ -182,14 +182,14 @@ public:
     }
 
     static Status decode_ascending(Slice* encoded_key, size_t index_size,
-                                   uint8_t* cell_ptr, Arena* arena) {
+                                   uint8_t* cell_ptr, MemPool* pool) {
         if (encoded_key->size < index_size) {
             return Status::InvalidArgument(
                 Substitute("Key too short, need=$0 vs real=$1",
                            index_size, encoded_key->size));
         }
         Slice* slice = (Slice*)cell_ptr;
-        slice->data = arena->Allocate(index_size);
+        slice->data = (char*)pool->allocate(index_size);
         slice->size = index_size;
         memcpy(slice->data, encoded_key->data, index_size);
         encoded_key->remove_prefix(index_size);
@@ -207,13 +207,13 @@ public:
     }
 
     static Status decode_ascending(Slice* encoded_key, size_t index_size,
-                                   uint8_t* cell_ptr, Arena* arena) {
+                                   uint8_t* cell_ptr, MemPool* pool) {
         CHECK(encoded_key->size <= index_size)
             << "encoded_key size is larger than index_size, key_size=" << encoded_key->size
             << ", index_size=" << index_size;
         auto copy_size = encoded_key->size;
         Slice* slice = (Slice*)cell_ptr;
-        slice->data = arena->Allocate(copy_size);
+        slice->data = (char*)pool->allocate(copy_size);
         slice->size = copy_size;
         memcpy(slice->data, encoded_key->data, copy_size);
         encoded_key->remove_prefix(copy_size);
diff --git a/be/src/olap/row.h b/be/src/olap/row.h
index b6b2926..760941c 100644
--- a/be/src/olap/row.h
+++ b/be/src/olap/row.h
@@ -134,16 +134,6 @@ void copy_row(DstRowType* dst, const SrcRowType& src, MemPool* pool) {
     }
 }
 
-// Deep copy src row to dst row. Schema of src and dst row must be same.
-template<typename DstRowType, typename SrcRowType>
-void copy_row(DstRowType* dst, const SrcRowType& src, Arena* arena) {
-    for (auto cid : dst->schema()->column_ids()) {
-        auto dst_cell = dst->cell(cid);
-        auto src_cell = src.cell(cid);
-        dst->schema()->column(cid)->deep_copy(&dst_cell, src_cell, arena);
-    }
-}
-
 template<typename DstRowType, typename SrcRowType>
 void agg_update_row(DstRowType* dst, const SrcRowType& src, MemPool* mem_pool) {
     for (uint32_t cid = dst->schema()->num_key_columns(); cid < dst->schema()->num_columns(); ++cid) {
diff --git a/be/src/olap/row_block2.cpp b/be/src/olap/row_block2.cpp
index 36aebdd..5c0c878 100644
--- a/be/src/olap/row_block2.cpp
+++ b/be/src/olap/row_block2.cpp
@@ -30,7 +30,8 @@ RowBlockV2::RowBlockV2(const Schema& schema, uint16_t capacity)
     : _schema(schema),
       _capacity(capacity),
       _column_datas(_schema.num_columns(), nullptr),
-      _column_null_bitmaps(_schema.num_columns(), nullptr) {
+      _column_null_bitmaps(_schema.num_columns(), nullptr),
+      _pool(new MemPool(&_tracker)) {
     auto bitmap_size = BitmapSize(capacity);
     for (auto cid : _schema.column_ids()) {
         size_t data_size = _schema.column(cid)->type_info()->size() * _capacity;
diff --git a/be/src/olap/row_block2.h b/be/src/olap/row_block2.h
index 8791262..c5756f6 100644
--- a/be/src/olap/row_block2.h
+++ b/be/src/olap/row_block2.h
@@ -25,10 +25,12 @@
 #include "olap/column_block.h"
 #include "olap/schema.h"
 #include "olap/types.h"
+#include "runtime/mem_pool.h"
+#include "runtime/mem_tracker.h"
 
 namespace doris {
 
-class Arena;
+class MemPool;
 class RowCursor;
 
 // This struct contains a block of rows, in which each column's data is stored
@@ -46,14 +48,14 @@ public:
     // return the maximum number of rows that can be contained in this block.
     // invariant: 0 <= num_rows() <= capacity()
     size_t capacity() const { return _capacity; }
-    Arena* arena() const { return _arena.get(); }
+    MemPool* pool() const { return _pool.get(); }
 
     // reset the state of the block so that it can be reused for write.
     // all previously returned ColumnBlocks are invalidated after clear(), accessing them
     // will result in undefined behavior.
     void clear() {
         _num_rows = 0;
-        _arena.reset(new Arena);
+        _pool->clear();
     }
 
     // Copy the row_idx row's data into given row_cursor.
@@ -67,7 +69,7 @@ public:
         const TypeInfo* type_info = _schema.column(cid)->type_info();
         uint8_t* data = _column_datas[cid];
         uint8_t* null_bitmap = _column_null_bitmaps[cid];
-        return ColumnBlock(type_info, data, null_bitmap, _arena.get());
+        return ColumnBlock(type_info, data, null_bitmap, _pool.get());
     }
 
     RowBlockRow row(size_t row_idx) const;
@@ -79,15 +81,16 @@ private:
     size_t _capacity;
     // keeps fixed-size (field_size x capacity) data vector for each column,
     // _column_datas[cid] == null if cid is not in `_schema`.
-    // memory are not allocated from `_arena` because we don't wan't to reallocate them in clear()
+    // memory are not allocated from `_pool` because we don't wan't to reallocate them in clear()
     std::vector<uint8_t*> _column_datas;
     // keeps null bitmap for each column,
     // _column_null_bitmaps[cid] == null if cid is not in `_schema` or the column is not null.
-    // memory are not allocated from `_arena` because we don't wan't to reallocate them in clear()
+    // memory are not allocated from `_pool` because we don't wan't to reallocate them in clear()
     std::vector<uint8_t*> _column_null_bitmaps;
     size_t _num_rows;
     // manages the memory for slice's data
-    std::unique_ptr<Arena> _arena;
+    MemTracker _tracker;
+    std::unique_ptr<MemPool> _pool;
 };
 
 // Stands for a row in RowBlockV2. It is consisted of a RowBlockV2 reference
diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
index c1cf607..adac8be 100644
--- a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
+++ b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
@@ -34,7 +34,8 @@ BinaryDictPageBuilder::BinaryDictPageBuilder(const PageBuilderOptions& options)
     _finished(false),
     _data_page_builder(nullptr),
     _dict_builder(nullptr),
-    _encoding_type(DICT_ENCODING) {
+    _encoding_type(DICT_ENCODING),
+    _pool(&_tracker) {
     // initially use DICT_ENCODING
     // TODO: the data page builder type can be created by Factory according to user config
     _data_page_builder.reset(new BitshufflePageBuilder<OLAP_FIELD_TYPE_INT>(options));
@@ -71,7 +72,7 @@ Status BinaryDictPageBuilder::add(const uint8_t* vals, size_t* count) {
                 }
                 Slice dict_item(src->data, src->size);
                 if (src->size > 0) {
-                    char* item_mem = _arena.Allocate(src->size);
+                    char* item_mem = (char*)_pool.allocate(src->size);
                     if (item_mem == nullptr) {
                         return Status::MemoryAllocFailed(Substitute("memory allocate failed, size:$0", src->size));
                     }
@@ -127,7 +128,7 @@ size_t BinaryDictPageBuilder::count() const {
 }
 
 uint64_t BinaryDictPageBuilder::size() const {
-    return _arena.MemoryUsage() + _data_page_builder->size();
+    return _pool.total_allocated_bytes() + _data_page_builder->size();
 }
 
 Status BinaryDictPageBuilder::get_dictionary_page(Slice* dictionary_page) {
@@ -206,7 +207,7 @@ Status BinaryDictPageDecoder::next_batch(size_t* n, ColumnBlockView* dst) {
     // And then copy the strings corresponding to the codewords to the destination buffer
     TypeInfo *type_info = get_type_info(OLAP_FIELD_TYPE_INT);
     // the data in page is not null
-    ColumnBlock column_block(type_info, _code_buf.data(), nullptr, dst->column_block()->arena());
+    ColumnBlock column_block(type_info, _code_buf.data(), nullptr, dst->column_block()->pool());
     ColumnBlockView tmp_block_view(&column_block);
     RETURN_IF_ERROR(_data_page_decoder->next_batch(n, &tmp_block_view));
     for (int i = 0; i < *n; ++i) {
@@ -214,7 +215,7 @@ Status BinaryDictPageDecoder::next_batch(size_t* n, ColumnBlockView* dst) {
         // get the string from the dict decoder
         Slice element = _dict_decoder->string_at_index(codeword);
         if (element.size > 0) {
-            char* destination = dst->column_block()->arena()->Allocate(element.size);
+            char* destination = (char*)dst->column_block()->pool()->allocate(element.size);
             if (destination == nullptr) {
                 return Status::MemoryAllocFailed(Substitute(
                     "memory allocate failed, size:$0", element.size));
diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.h b/be/src/olap/rowset/segment_v2/binary_dict_page.h
index 7951efd..6a2ef32 100644
--- a/be/src/olap/rowset/segment_v2/binary_dict_page.h
+++ b/be/src/olap/rowset/segment_v2/binary_dict_page.h
@@ -28,7 +28,8 @@
 #include "olap/rowset/segment_v2/options.h"
 #include "olap/rowset/segment_v2/common.h"
 #include "olap/column_block.h"
-#include "util/arena.h"
+#include "runtime/mem_pool.h"
+#include "runtime/mem_tracker.h"
 #include "gen_cpp/segment_v2.pb.h"
 #include "gutil/hash/string_hash.h"
 
@@ -94,7 +95,9 @@ private:
     std::unordered_map<Slice, uint32_t, HashOfSlice> _dictionary;
     // used to remember the insertion order of dict keys
     std::vector<Slice> _dict_items;
-    Arena _arena;
+    // TODO(zc): rethink about this mem pool
+    MemTracker _tracker;
+    MemPool _pool;
     faststring _buffer;
 };
 
diff --git a/be/src/olap/rowset/segment_v2/binary_plain_page.h b/be/src/olap/rowset/segment_v2/binary_plain_page.h
index d085164..b442eb8 100644
--- a/be/src/olap/rowset/segment_v2/binary_plain_page.h
+++ b/be/src/olap/rowset/segment_v2/binary_plain_page.h
@@ -28,7 +28,6 @@
 
 #pragma once
 
-#include "util/arena.h"
 #include "util/coding.h"
 #include "util/faststring.h"
 #include "olap/olap_common.h"
@@ -36,6 +35,7 @@
 #include "olap/rowset/segment_v2/page_decoder.h"
 #include "olap/rowset/segment_v2/options.h"
 #include "olap/types.h"
+#include "runtime/mem_pool.h"
 
 namespace doris {
 namespace segment_v2 {
@@ -183,7 +183,7 @@ public:
             Slice elem(string_at_index(_cur_idx));
             out->size = elem.size;
             if (elem.size != 0) {
-                out->data = reinterpret_cast<char*>(dst->arena()->Allocate(elem.size * sizeof(uint8_t)));
+                out->data = reinterpret_cast<char*>(dst->pool()->allocate(elem.size * sizeof(uint8_t)));
                 memcpy(out->data, elem.data, elem.size);
             }
         }
diff --git a/be/src/olap/rowset/segment_v2/column_reader.h b/be/src/olap/rowset/segment_v2/column_reader.h
index af34db2..af0de1f 100644
--- a/be/src/olap/rowset/segment_v2/column_reader.h
+++ b/be/src/olap/rowset/segment_v2/column_reader.h
@@ -34,7 +34,6 @@
 namespace doris {
 
 class ColumnBlock;
-class Arena;
 class RandomAccessFile;
 class TypeInfo;
 class BlockCompressionCodec;
@@ -134,7 +133,7 @@ public:
 
     // After one seek, we can call this function many times to read data 
     // into ColumnBlock. when read string type data, memory will allocated
-    // from Arena
+    // from MemPool
     virtual Status next_batch(size_t* n, ColumnBlock* dst) = 0;
 
     virtual rowid_t get_current_ordinal() const = 0;
@@ -151,8 +150,8 @@ public:
     //
     // In the case that the values are themselves references
     // to other memory (eg Slices), the referred-to memory is
-    // allocated in the dst column vector's arena.
-    Status scan(size_t* n, ColumnBlock* dst, Arena* arena);
+    // allocated in the dst column vector's MemPool.
+    Status scan(size_t* n, ColumnBlock* dst, MemPool* pool);
 
     // release next_batch related resource
     Status finish_batch();
diff --git a/be/src/olap/rowset/segment_v2/column_zone_map.cpp b/be/src/olap/rowset/segment_v2/column_zone_map.cpp
index 88609f1..659880c 100644
--- a/be/src/olap/rowset/segment_v2/column_zone_map.cpp
+++ b/be/src/olap/rowset/segment_v2/column_zone_map.cpp
@@ -23,15 +23,15 @@ namespace doris {
 
 namespace segment_v2 {
 
-ColumnZoneMapBuilder::ColumnZoneMapBuilder(Field* field) : _field(field) {
+ColumnZoneMapBuilder::ColumnZoneMapBuilder(Field* field) : _field(field), _pool(&_tracker) {
     PageBuilderOptions options;
     options.data_page_size = 0;
     _page_builder.reset(new BinaryPlainPageBuilder(options));
-    _zone_map.min_value = _field->allocate_value_from_arena(&_arena);
-    _zone_map.max_value = _field->allocate_value_from_arena(&_arena);
+    _zone_map.min_value = _field->allocate_value(&_pool);
+    _zone_map.max_value = _field->allocate_value(&_pool);
     _reset_page_zone_map();
-    _segment_zone_map.min_value = _field->allocate_value_from_arena(&_arena);
-    _segment_zone_map.max_value = _field->allocate_value_from_arena(&_arena);
+    _segment_zone_map.min_value = _field->allocate_value(&_pool);
+    _segment_zone_map.max_value = _field->allocate_value(&_pool);
     _reset_segment_zone_map();
 }
 
diff --git a/be/src/olap/rowset/segment_v2/column_zone_map.h b/be/src/olap/rowset/segment_v2/column_zone_map.h
index 3ed3f0c..f4aaa45 100644
--- a/be/src/olap/rowset/segment_v2/column_zone_map.h
+++ b/be/src/olap/rowset/segment_v2/column_zone_map.h
@@ -25,6 +25,8 @@
 #include "olap/field.h"
 #include "gen_cpp/segment_v2.pb.h"
 #include "olap/rowset/segment_v2/binary_plain_page.h"
+#include "runtime/mem_pool.h"
+#include "runtime/mem_tracker.h"
 
 namespace doris {
 
@@ -75,10 +77,13 @@ private:
 private:
     std::unique_ptr<BinaryPlainPageBuilder> _page_builder;
     Field* _field;
-    // memory will be managed by arena
+    // memory will be managed by MemPool
     ZoneMap _zone_map;
     ZoneMap _segment_zone_map;
-    Arena _arena;
+    // TODO(zc): we should replace this memory pool later, we only allocate min/max
+    // for field. But MemPool allocate 4KB least, it will a waste for most cases.
+    MemTracker _tracker;
+    MemPool _pool;
 };
 
 // ColumnZoneMap
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index d3ce736..a735c49 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -270,9 +270,9 @@ Status SegmentIterator::_seek_and_peek(rowid_t rowid) {
         _column_iterators[cid]->seek_to_ordinal(rowid);
     }
     size_t num_rows = 1;
-    // please note that usually RowBlockV2.clear() is called to free arena memory before reading the next block,
+    // please note that usually RowBlockV2.clear() is called to free MemPool memory before reading the next block,
     // but here since there won't be too many keys to seek, we don't call RowBlockV2.clear() so that we can use
-    // a single arena for all seeked keys.
+    // a single MemPool for all seeked keys.
     RETURN_IF_ERROR(_next_batch(_seek_block.get(), &num_rows));
     _seek_block->set_num_rows(num_rows);
     return Status::OK();
diff --git a/be/src/olap/skiplist.h b/be/src/olap/skiplist.h
index fb1da64..9f1a5d0 100644
--- a/be/src/olap/skiplist.h
+++ b/be/src/olap/skiplist.h
@@ -44,7 +44,7 @@ private:
 
 public:
     // Create a new SkipList object that will use "cmp" for comparing keys,
-    // and will allocate memory using "*arena".  Objects allocated in the arena
+    // and will allocate memory using "*mem_pool".  Objects allocated in the mem_pool
     // must remain allocated for the lifetime of the skiplist object.
     explicit SkipList(Comparator cmp, MemPool* mem_pool);
 
diff --git a/be/src/olap/types.cpp b/be/src/olap/types.cpp
index d7f9768..9ff8afc 100644
--- a/be/src/olap/types.cpp
+++ b/be/src/olap/types.cpp
@@ -27,7 +27,6 @@ TypeInfo::TypeInfo(TypeTraitsClass t)
         _cmp(TypeTraitsClass::cmp),
         _shallow_copy(TypeTraitsClass::shallow_copy),
         _deep_copy(TypeTraitsClass::deep_copy),
-        _deep_copy_with_arena(TypeTraitsClass::deep_copy_with_arena),
         _direct_copy(TypeTraitsClass::direct_copy),
         _from_string(TypeTraitsClass::from_string),
         _to_string(TypeTraitsClass::to_string),
diff --git a/be/src/olap/types.h b/be/src/olap/types.h
index 78bc093..3bb0aa2 100644
--- a/be/src/olap/types.h
+++ b/be/src/olap/types.h
@@ -34,7 +34,6 @@
 #include "util/mem_util.hpp"
 #include "util/slice.h"
 #include "util/types.h"
-#include "util/arena.h"
 
 namespace doris {
 
@@ -56,10 +55,6 @@ public:
         _deep_copy(dest, src, mem_pool);
     }
 
-    inline void deep_copy_with_arena(void* dest, const void* src, Arena* arena) const {
-        _deep_copy_with_arena(dest, src, arena);
-    }
-
     inline void direct_copy(void* dest, const void* src) const {
         _direct_copy(dest, src);
     }
@@ -83,7 +78,6 @@ private:
 
     void (*_shallow_copy)(void* dest, const void* src);
     void (*_deep_copy)(void* dest, const void* src, MemPool* mem_pool);
-    void (*_deep_copy_with_arena)(void* dest, const void* src, Arena* arena);
     void (*_direct_copy)(void* dest, const void* src);
 
     OLAPStatus (*_from_string)(void* buf, const std::string& scan_key);
@@ -193,10 +187,6 @@ struct BaseFieldtypeTraits : public CppTypeTraits<field_type> {
         *reinterpret_cast<CppType*>(dest) = *reinterpret_cast<const CppType*>(src);
     }
 
-    static inline void deep_copy_with_arena(void* dest, const void* src, Arena* arena) {
-        *reinterpret_cast<CppType*>(dest) = *reinterpret_cast<const CppType*>(src);
-    }
-
     static inline void direct_copy(void* dest, const void* src) {
         *reinterpret_cast<CppType*>(dest) = *reinterpret_cast<const CppType*>(src);
     }
@@ -343,9 +333,6 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_LARGEINT> : public BaseFieldtypeTraits<OL
     static void deep_copy(void* dest, const void* src, MemPool* mem_pool) {
         *reinterpret_cast<PackedInt128*>(dest) = *reinterpret_cast<const PackedInt128*>(src);
     }
-    static void deep_copy_with_arena(void* dest, const void* src, Arena* arena) {
-        *reinterpret_cast<PackedInt128*>(dest) = *reinterpret_cast<const PackedInt128*>(src);
-    }
     static void direct_copy(void* dest, const void* src) {
         *reinterpret_cast<PackedInt128*>(dest) = *reinterpret_cast<const PackedInt128*>(src);
     }
@@ -541,13 +528,6 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_CHAR> : public BaseFieldtypeTraits<OLAP_F
         memory_copy(l_slice->data, r_slice->data, r_slice->size);
         l_slice->size = r_slice->size;
     }
-    static void deep_copy_with_arena(void* dest, const void* src, Arena* arena) {
-        auto l_slice = reinterpret_cast<Slice*>(dest);
-        auto r_slice = reinterpret_cast<const Slice*>(src);
-        l_slice->data = reinterpret_cast<char*>(arena->Allocate(r_slice->size));
-        memory_copy(l_slice->data, r_slice->data, r_slice->size);
-        l_slice->size = r_slice->size;
-    }
     static void direct_copy(void* dest, const void* src) {
         auto l_slice = reinterpret_cast<Slice*>(dest);
         auto r_slice = reinterpret_cast<const Slice*>(src);
diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt
index 66e8a92..a69afcc 100644
--- a/be/src/util/CMakeLists.txt
+++ b/be/src/util/CMakeLists.txt
@@ -25,7 +25,6 @@ set(UTIL_FILES
   arrow/row_batch.cpp
   arrow/row_block.cpp
   arrow/utils.cpp
-  arena.cpp
   bfd_parser.cpp
   bitmap.cpp
   block_compression.cpp
diff --git a/be/src/util/arena.cpp b/be/src/util/arena.cpp
deleted file mode 100644
index cf72db8..0000000
--- a/be/src/util/arena.cpp
+++ /dev/null
@@ -1,68 +0,0 @@
-// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file. See the AUTHORS file for names of contributors.
-
-#include "util/arena.h"
-#include <assert.h>
-
-namespace doris {
-
-static const int kBlockSize = 4096;
-
-Arena::Arena() : memory_usage_(0) {
-    alloc_ptr_ = NULL;  // First allocation will allocate a block
-    alloc_bytes_remaining_ = 0;
-}
-
-Arena::~Arena() {
-    for (size_t i = 0; i < blocks_.size(); i++) {
-        delete[] blocks_[i];
-    }
-}
-
-char* Arena::AllocateFallback(size_t bytes) {
-    if (bytes > kBlockSize / 4) {
-        // Object is more than a quarter of our block size.  Allocate it separately
-        // to avoid wasting too much space in leftover bytes.
-        char* result = AllocateNewBlock(bytes);
-        return result;
-    }
-
-    // We waste the remaining space in the current block.
-    alloc_ptr_ = AllocateNewBlock(kBlockSize);
-    alloc_bytes_remaining_ = kBlockSize;
-
-    char* result = alloc_ptr_;
-    alloc_ptr_ += bytes;
-    alloc_bytes_remaining_ -= bytes;
-    return result;
-}
-
-char* Arena::AllocateAligned(size_t bytes) {
-    const int align = (sizeof(void*) > 8) ? sizeof(void*) : 8;
-    assert((align & (align-1)) == 0);   // Pointer size should be a power of 2
-    size_t current_mod = reinterpret_cast<uintptr_t>(alloc_ptr_) & (align-1);
-    size_t slop = (current_mod == 0 ? 0 : align - current_mod);
-    size_t needed = bytes + slop;
-    char* result;
-    if (needed <= alloc_bytes_remaining_) {
-        result = alloc_ptr_ + slop;
-        alloc_ptr_ += needed;
-        alloc_bytes_remaining_ -= needed;
-    } else {
-        // AllocateFallback always returned aligned memory
-        result = AllocateFallback(bytes);
-    }
-    assert((reinterpret_cast<uintptr_t>(result) & (align-1)) == 0);
-    return result;
-}
-
-char* Arena::AllocateNewBlock(size_t block_bytes) {
-    char* result = new char[block_bytes];
-    blocks_.push_back(result);
-    memory_usage_.store(MemoryUsage() + block_bytes + sizeof(char*),
-            std::memory_order_relaxed);
-    return result;
-}
-
-}  // namespace doris
diff --git a/be/src/util/arena.h b/be/src/util/arena.h
deleted file mode 100644
index ef042eb..0000000
--- a/be/src/util/arena.h
+++ /dev/null
@@ -1,74 +0,0 @@
-// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file. See the AUTHORS file for names of contributors.
-
-#ifndef DORIS_BE_SRC_COMMON_UTIL_ARENA_H
-#define DORIS_BE_SRC_COMMON_UTIL_ARENA_H
-
-#include <assert.h>
-#include <stddef.h>
-#include <stdint.h>
-
-#include <atomic>
-#include <vector>
-
-#include "common/compiler_util.h"
-
-namespace doris {
-
-class Arena {
-public:
-    Arena();
-    ~Arena();
-
-    // Return a pointer to a newly allocated memory block of "bytes" bytes.
-    char* Allocate(size_t bytes);
-
-    // Allocate memory with the normal alignment guarantees provided by malloc
-    char* AllocateAligned(size_t bytes);
-
-    // Returns an estimate of the total memory usage of data allocated
-    // by the arena.
-    size_t MemoryUsage() const {
-        //return reinterpret_cast<uintptr_t>(memory_usage_.NoBarrier_Load());
-        return memory_usage_.load(std::memory_order_relaxed);
-    }
-
-    // For the object wasn't allocated from Arena, but need to
-    // collect and control the object memory usage.
-    void track_memory(size_t bytes) {
-        memory_usage_.store(MemoryUsage() + bytes,std::memory_order_relaxed);
-    }
-private:
-    char* AllocateFallback(size_t bytes);
-    char* AllocateNewBlock(size_t block_bytes);
-
-    // Allocation state
-    char* alloc_ptr_;
-    size_t alloc_bytes_remaining_;
-
-    // Array of new[] allocated memory blocks
-    std::vector<char*> blocks_;
-
-    // Total memory usage of the arena.
-    std::atomic<size_t> memory_usage_;
-
-    // No copying allowed
-    Arena(const Arena&);
-    void operator=(const Arena&);
-};
-
-inline char* Arena::Allocate(size_t bytes) {
-    if (UNLIKELY(bytes == 0)) { return nullptr; }
-    if (bytes <= alloc_bytes_remaining_) {
-        char* result = alloc_ptr_;
-        alloc_ptr_ += bytes;
-        alloc_bytes_remaining_ -= bytes;
-        return result;
-    }
-    return AllocateFallback(bytes);
-}
-
-}  // namespace doris
-
-#endif  // DORIS_BE_SRC_COMMON_UTIL_ARENA_H
diff --git a/be/test/olap/delta_writer_test.cpp b/be/test/olap/delta_writer_test.cpp
index 495d011..134f09b 100644
--- a/be/test/olap/delta_writer_test.cpp
+++ b/be/test/olap/delta_writer_test.cpp
@@ -31,6 +31,8 @@
 #include "runtime/tuple.h"
 #include "runtime/descriptor_helper.h"
 #include "runtime/exec_env.h"
+#include "runtime/mem_pool.h"
+#include "runtime/mem_tracker.h"
 #include "util/logging.h"
 #include "olap/options.h"
 #include "olap/tablet_meta_manager.h"
@@ -345,10 +347,11 @@ TEST_F(TestDeltaWriter, write) {
     DeltaWriter::open(&write_req, k_mem_tracker, &delta_writer); 
     ASSERT_NE(delta_writer, nullptr);
 
-    Arena arena;
+    MemTracker tracker;
+    MemPool pool(&tracker);
     // Tuple 1
     {
-        Tuple* tuple = reinterpret_cast<Tuple*>(arena.Allocate(tuple_desc->byte_size()));
+        Tuple* tuple = reinterpret_cast<Tuple*>(pool.allocate(tuple_desc->byte_size()));
         memset(tuple, 0, tuple_desc->byte_size());
         *(int8_t*)(tuple->get_slot(slots[0]->tuple_offset())) = -127;
         *(int16_t*)(tuple->get_slot(slots[1]->tuple_offset())) = -32767;
@@ -362,12 +365,12 @@ TEST_F(TestDeltaWriter, write) {
         ((DateTimeValue*)(tuple->get_slot(slots[6]->tuple_offset())))->from_date_str("2636-08-16 19:39:43", 19); 
 
         StringValue* char_ptr = (StringValue*)(tuple->get_slot(slots[7]->tuple_offset()));
-        char_ptr->ptr = arena.Allocate(4);
+        char_ptr->ptr = (char*)pool.allocate(4);
         memcpy(char_ptr->ptr, "abcd", 4);
         char_ptr->len = 4; 
 
         StringValue* var_ptr = (StringValue*)(tuple->get_slot(slots[8]->tuple_offset()));
-        var_ptr->ptr = arena.Allocate(5);
+        var_ptr->ptr = (char*)pool.allocate(5);
         memcpy(var_ptr->ptr, "abcde", 5);
         var_ptr->len = 5; 
 
@@ -385,12 +388,12 @@ TEST_F(TestDeltaWriter, write) {
         ((DateTimeValue*)(tuple->get_slot(slots[16]->tuple_offset())))->from_date_str("2636-08-16 19:39:43", 19); 
 
         char_ptr = (StringValue*)(tuple->get_slot(slots[17]->tuple_offset()));
-        char_ptr->ptr = arena.Allocate(4);
+        char_ptr->ptr = (char*)pool.allocate(4);
         memcpy(char_ptr->ptr, "abcd", 4);
         char_ptr->len = 4; 
 
         var_ptr = (StringValue*)(tuple->get_slot(slots[18]->tuple_offset()));
-        var_ptr->ptr = arena.Allocate(5);
+        var_ptr->ptr = (char*)pool.allocate(5);
         memcpy(var_ptr->ptr, "abcde", 5);
         var_ptr->len = 5;
 
diff --git a/be/test/olap/key_coder_test.cpp b/be/test/olap/key_coder_test.cpp
index 9c6ddd6..8fb22c9 100644
--- a/be/test/olap/key_coder_test.cpp
+++ b/be/test/olap/key_coder_test.cpp
@@ -22,14 +22,19 @@
 #include <string.h>
 
 #include "util/debug_util.h"
+#include "runtime/mem_pool.h"
+#include "runtime/mem_tracker.h"
 
 namespace doris {
 
 class KeyCoderTest : public testing::Test {
 public:
-    KeyCoderTest() { }
+    KeyCoderTest() : _pool(&_tracker) { }
     virtual ~KeyCoderTest() {
     }
+private:
+    MemTracker _tracker;
+    MemPool _pool;
 };
 
 template<FieldType type>
@@ -97,7 +102,7 @@ void test_integer_encode() {
     }
 }
 
-TEST(KeyCoderTest, test_int) {
+TEST_F(KeyCoderTest, test_int) {
     test_integer_encode<OLAP_FIELD_TYPE_TINYINT>();
     test_integer_encode<OLAP_FIELD_TYPE_SMALLINT>();
     test_integer_encode<OLAP_FIELD_TYPE_INT>();
@@ -108,7 +113,7 @@ TEST(KeyCoderTest, test_int) {
     test_integer_encode<OLAP_FIELD_TYPE_DATETIME>();
 }
 
-TEST(KeyCoderTest, test_date) {
+TEST_F(KeyCoderTest, test_date) {
     using CppType = uint24_t;
     auto key_coder = get_key_coder(OLAP_FIELD_TYPE_DATE);
 
@@ -168,7 +173,7 @@ TEST(KeyCoderTest, test_date) {
     }
 }
 
-TEST(KeyCoderTest, test_decimal) {
+TEST_F(KeyCoderTest, test_decimal) {
     auto key_coder = get_key_coder(OLAP_FIELD_TYPE_DECIMAL);
 
     decimal12_t val1(1, 100000000);
@@ -212,7 +217,7 @@ TEST(KeyCoderTest, test_decimal) {
     }
 }
 
-TEST(KeyCoderTest, test_char) {
+TEST_F(KeyCoderTest, test_char) {
     auto key_coder = get_key_coder(OLAP_FIELD_TYPE_CHAR);
 
     char buf[] = "1234567890";
@@ -223,9 +228,8 @@ TEST(KeyCoderTest, test_char) {
         key_coder->encode_ascending(&slice, 10, &key);
         Slice encoded_key(key);
 
-        Arena arena;
         Slice check_slice;
-        auto st = key_coder->decode_ascending(&encoded_key, 10, (uint8_t*)&check_slice, &arena);
+        auto st = key_coder->decode_ascending(&encoded_key, 10, (uint8_t*)&check_slice, &_pool);
         ASSERT_TRUE(st.ok());
 
         ASSERT_STREQ("1234567890", check_slice.data);
@@ -236,16 +240,15 @@ TEST(KeyCoderTest, test_char) {
         key_coder->encode_ascending(&slice, 5, &key);
         Slice encoded_key(key);
 
-        Arena arena;
         Slice check_slice;
-        auto st = key_coder->decode_ascending(&encoded_key, 5, (uint8_t*)&check_slice, &arena);
+        auto st = key_coder->decode_ascending(&encoded_key, 5, (uint8_t*)&check_slice, &_pool);
         ASSERT_TRUE(st.ok());
 
         ASSERT_STREQ("12345", check_slice.data);
     }
 }
 
-TEST(KeyCoderTest, test_varchar) {
+TEST_F(KeyCoderTest, test_varchar) {
     auto key_coder = get_key_coder(OLAP_FIELD_TYPE_VARCHAR);
 
     char buf[] = "1234567890";
@@ -256,9 +259,8 @@ TEST(KeyCoderTest, test_varchar) {
         key_coder->encode_ascending(&slice, 15, &key);
         Slice encoded_key(key);
 
-        Arena arena;
         Slice check_slice;
-        auto st = key_coder->decode_ascending(&encoded_key, 15, (uint8_t*)&check_slice, &arena);
+        auto st = key_coder->decode_ascending(&encoded_key, 15, (uint8_t*)&check_slice, &_pool);
         ASSERT_TRUE(st.ok());
 
         ASSERT_STREQ("1234567890", check_slice.data);
@@ -269,9 +271,8 @@ TEST(KeyCoderTest, test_varchar) {
         key_coder->encode_ascending(&slice, 5, &key);
         Slice encoded_key(key);
 
-        Arena arena;
         Slice check_slice;
-        auto st = key_coder->decode_ascending(&encoded_key, 5, (uint8_t*)&check_slice, &arena);
+        auto st = key_coder->decode_ascending(&encoded_key, 5, (uint8_t*)&check_slice, &_pool);
         ASSERT_TRUE(st.ok());
 
         ASSERT_STREQ("12345", check_slice.data);
diff --git a/be/test/olap/rowset/segment_v2/binary_dict_page_test.cpp b/be/test/olap/rowset/segment_v2/binary_dict_page_test.cpp
index d082bca..a201a0c 100644
--- a/be/test/olap/rowset/segment_v2/binary_dict_page_test.cpp
+++ b/be/test/olap/rowset/segment_v2/binary_dict_page_test.cpp
@@ -27,6 +27,8 @@
 #include "olap/olap_common.h"
 #include "olap/types.h"
 #include "util/debug_util.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/mem_pool.h"
 
 namespace doris {
 namespace segment_v2 {
@@ -70,11 +72,12 @@ public:
         ASSERT_EQ(slices.size(), page_decoder.count());
 
         //check values
-        Arena arena;
+        MemTracker tracker;
+        MemPool pool(&tracker);
         TypeInfo* type_info = get_type_info(OLAP_FIELD_TYPE_VARCHAR);
         size_t size = slices.size();
-        Slice* values = reinterpret_cast<Slice*>(arena.Allocate(size * sizeof(Slice)));
-        ColumnBlock column_block(type_info, (uint8_t*)values, nullptr, &arena);
+        Slice* values = reinterpret_cast<Slice*>(pool.allocate(size * sizeof(Slice)));
+        ColumnBlock column_block(type_info, (uint8_t*)values, nullptr, &pool);
         ColumnBlockView block_view(&column_block);
 
         status = page_decoder.next_batch(&size, &block_view);
@@ -161,10 +164,11 @@ public:
             ASSERT_TRUE(status.ok());
 
             //check values
-            Arena arena;
+            MemTracker tracker;
+            MemPool pool(&tracker);
             TypeInfo* type_info = get_type_info(OLAP_FIELD_TYPE_VARCHAR);
-            Slice* values = reinterpret_cast<Slice*>(arena.Allocate(sizeof(Slice)));
-            ColumnBlock column_block(type_info, (uint8_t*)values, nullptr, &arena);
+            Slice* values = reinterpret_cast<Slice*>(pool.allocate(sizeof(Slice)));
+            ColumnBlock column_block(type_info, (uint8_t*)values, nullptr, &pool);
             ColumnBlockView block_view(&column_block);
 
             size_t num = 1;
diff --git a/be/test/olap/rowset/segment_v2/binary_plain_page_test.cpp b/be/test/olap/rowset/segment_v2/binary_plain_page_test.cpp
index 45f0242..133b227 100644
--- a/be/test/olap/rowset/segment_v2/binary_plain_page_test.cpp
+++ b/be/test/olap/rowset/segment_v2/binary_plain_page_test.cpp
@@ -25,6 +25,8 @@
 #include "olap/rowset/segment_v2/binary_plain_page.h"
 #include "olap/olap_common.h"
 #include "olap/types.h"
+#include "runtime/mem_pool.h"
+#include "runtime/mem_tracker.h"
 
 namespace doris {
 namespace segment_v2 {
@@ -59,13 +61,13 @@ public:
 
         //test1
         
-        size_t size = 3;
-        
-        Arena arena;
 
-        Slice* values = reinterpret_cast<Slice*>(arena.Allocate(size * sizeof(Slice)));
-        uint8_t* null_bitmap = reinterpret_cast<uint8_t*>(arena.Allocate(BitmapSize(size)));
-        ColumnBlock block(get_type_info(OLAP_FIELD_TYPE_VARCHAR), (uint8_t*)values, null_bitmap, &arena);
+        MemTracker tracker;
+        MemPool pool(&tracker);
+        size_t size = 3;
+        Slice* values = reinterpret_cast<Slice*>(pool.allocate(size * sizeof(Slice)));
+        uint8_t* null_bitmap = reinterpret_cast<uint8_t*>(pool.allocate(BitmapSize(size)));
+        ColumnBlock block(get_type_info(OLAP_FIELD_TYPE_VARCHAR), (uint8_t*)values, null_bitmap, &pool);
         ColumnBlockView column_block_view(&block);
 
         status = page_decoder.next_batch(&size, &column_block_view);
@@ -77,8 +79,8 @@ public:
         ASSERT_EQ (",", value[1].to_string());
         ASSERT_EQ ("Doris", value[2].to_string());
         
-        Slice* values2 = reinterpret_cast<Slice*>(arena.Allocate(size * sizeof(Slice)));
-        ColumnBlock block2(get_type_info(OLAP_FIELD_TYPE_VARCHAR), (uint8_t*)values2, null_bitmap, &arena);
+        Slice* values2 = reinterpret_cast<Slice*>(pool.allocate(size * sizeof(Slice)));
+        ColumnBlock block2(get_type_info(OLAP_FIELD_TYPE_VARCHAR), (uint8_t*)values2, null_bitmap, &pool);
         ColumnBlockView column_block_view2(&block2);
 
         size_t fetch_num = 1;
diff --git a/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp b/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp
index c993471..f721ecf 100644
--- a/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp
+++ b/be/test/olap/rowset/segment_v2/bitshuffle_page_test.cpp
@@ -22,7 +22,8 @@
 #include "olap/rowset/segment_v2/page_builder.h"
 #include "olap/rowset/segment_v2/page_decoder.h"
 #include "olap/rowset/segment_v2/bitshuffle_page.h"
-#include "util/arena.h"
+#include "runtime/mem_pool.h"
+#include "runtime/mem_tracker.h"
 #include "util/logging.h"
 
 using doris::segment_v2::PageBuilderOptions;
@@ -35,9 +36,10 @@ public:
 
     template<FieldType type, class PageDecoderType>
     void copy_one(PageDecoderType* decoder, typename TypeTraits<type>::CppType* ret) {
-        Arena arena;
+        MemTracker tracker;
+        MemPool pool(&tracker);
         uint8_t null_bitmap = 0;
-        ColumnBlock block(get_type_info(type), (uint8_t*)ret, &null_bitmap, &arena);
+        ColumnBlock block(get_type_info(type), (uint8_t*)ret, &null_bitmap, &pool);
         ColumnBlockView column_block_view(&block);
 
         size_t n = 1;
@@ -64,11 +66,12 @@ public:
         ASSERT_TRUE(status.ok());
         ASSERT_EQ(0, page_decoder.current_index());
 
-        Arena arena;
+        MemTracker tracker;
+        MemPool pool(&tracker);
 
-        CppType* values = reinterpret_cast<CppType*>(arena.Allocate(size * sizeof(CppType)));
-        uint8_t* null_bitmap = reinterpret_cast<uint8_t*>(arena.Allocate(BitmapSize(size)));
-        ColumnBlock block(get_type_info(Type), (uint8_t*)values, null_bitmap, &arena);
+        CppType* values = reinterpret_cast<CppType*>(pool.allocate(size * sizeof(CppType)));
+        uint8_t* null_bitmap = reinterpret_cast<uint8_t*>(pool.allocate(BitmapSize(size)));
+        ColumnBlock block(get_type_info(Type), (uint8_t*)values, null_bitmap, &pool);
         ColumnBlockView column_block_view(&block);
 
         status = page_decoder.next_batch(&size, &column_block_view);
diff --git a/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp b/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
index 91b8296..f4a0ba3 100644
--- a/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
+++ b/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp
@@ -29,16 +29,20 @@
 #include "olap/types.h"
 #include "olap/column_block.h"
 #include "util/file_utils.h"
-#include "util/arena.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/mem_pool.h"
 
 namespace doris {
 namespace segment_v2 {
 
 class ColumnReaderWriterTest : public testing::Test {
 public:
-    ColumnReaderWriterTest() { }
+    ColumnReaderWriterTest() : _pool(&_tracker) { }
     virtual ~ColumnReaderWriterTest() {
     }
+private:
+    MemTracker _tracker;
+    MemPool _pool;
 };
 
 template<FieldType type, EncodingTypePB encoding>
@@ -119,11 +123,12 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, s
             st = iter->seek_to_first();
             ASSERT_TRUE(st.ok());
 
-            Arena arena;
+            MemTracker tracker;
+            MemPool pool(&tracker);
             Type vals[1024];
             Type* vals_ = vals;
             uint8_t is_null[1024];
-            ColumnBlock col(type_info, (uint8_t*)vals, is_null, &arena);
+            ColumnBlock col(type_info, (uint8_t*)vals, is_null, &pool);
 
             int idx = 0;
             while (true) {
@@ -152,10 +157,11 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, s
         }
         // random read
         {
-            Arena arena;
+            MemTracker tracker;
+            MemPool pool(&tracker);
             Type vals[1024];
             uint8_t is_null[1024];
-            ColumnBlock col(type_info, (uint8_t*)vals, is_null, &arena);
+            ColumnBlock col(type_info, (uint8_t*)vals, is_null, &pool);
 
             for (int rowid = 0; rowid < num_rows; rowid += 4025) {
                 st = iter->seek_to_ordinal(rowid);
@@ -224,8 +230,6 @@ TEST_F(ColumnReaderWriterTest, test_nullable) {
 }
 
 TEST_F(ColumnReaderWriterTest, test_types) {
-    Arena arena;
-
     size_t num_uint8_rows = 1024 * 1024;
     uint8_t* is_null = new uint8_t[num_uint8_rows];
 
@@ -241,8 +245,8 @@ TEST_F(ColumnReaderWriterTest, test_types) {
         datetime_vals[i] = i + 33;
         decimal_vals[i] = decimal12_t(i, i); // 1.000000001
 
-        set_column_value_by_type(OLAP_FIELD_TYPE_VARCHAR, i, (char*)&varchar_vals[i], &arena);
-        set_column_value_by_type(OLAP_FIELD_TYPE_CHAR, i, (char*)&char_vals[i], &arena, 8);
+        set_column_value_by_type(OLAP_FIELD_TYPE_VARCHAR, i, (char*)&varchar_vals[i], &_pool);
+        set_column_value_by_type(OLAP_FIELD_TYPE_CHAR, i, (char*)&char_vals[i], &_pool, 8);
 
         BitmapChange(is_null, i, (i % 4) == 0);
     }
diff --git a/be/test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp b/be/test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp
index 756b096..2b52eb5 100644
--- a/be/test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp
+++ b/be/test/olap/rowset/segment_v2/frame_of_reference_page_test.cpp
@@ -22,7 +22,8 @@
 #include "olap/rowset/segment_v2/page_builder.h"
 #include "olap/rowset/segment_v2/page_decoder.h"
 #include "olap/rowset/segment_v2/frame_of_reference_page.h"
-#include "util/arena.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/mem_pool.h"
 #include "util/logging.h"
 
 using doris::segment_v2::PageBuilderOptions;
@@ -33,9 +34,10 @@ class FrameOfReferencePageTest : public testing::Test {
 public:
     template<FieldType type, class PageDecoderType>
     void copy_one(PageDecoderType* decoder, typename TypeTraits<type>::CppType* ret) {
-        Arena arena;
+        MemTracker tracker;
+        MemPool pool(&tracker);
         uint8_t null_bitmap = 0;
-        ColumnBlock block(get_type_info(type), (uint8_t*)ret, &null_bitmap, &arena);
+        ColumnBlock block(get_type_info(type), (uint8_t*)ret, &null_bitmap, &pool);
         ColumnBlockView column_block_view(&block);
 
         size_t n = 1;
@@ -63,10 +65,11 @@ public:
         ASSERT_EQ(0, for_page_decoder.current_index());
         ASSERT_EQ(size, for_page_decoder.count());
 
-        Arena arena;
-        CppType* values = reinterpret_cast<CppType*>(arena.Allocate(size * sizeof(CppType)));
-        uint8_t* null_bitmap = reinterpret_cast<uint8_t*>(arena.Allocate(BitmapSize(size)));
-        ColumnBlock block(get_type_info(Type), (uint8_t*)values, null_bitmap, &arena);
+        MemTracker tracker;
+        MemPool pool(&tracker);
+        CppType* values = reinterpret_cast<CppType*>(pool.allocate(size * sizeof(CppType)));
+        uint8_t* null_bitmap = reinterpret_cast<uint8_t*>(pool.allocate(BitmapSize(size)));
+        ColumnBlock block(get_type_info(Type), (uint8_t*)values, null_bitmap, &pool);
         ColumnBlockView column_block_view(&block);
         size_t size_to_fetch = size;
         status = for_page_decoder.next_batch(&size_to_fetch, &column_block_view);
diff --git a/be/test/olap/rowset/segment_v2/plain_page_test.cpp b/be/test/olap/rowset/segment_v2/plain_page_test.cpp
index 30b6bd4..48767b2 100644
--- a/be/test/olap/rowset/segment_v2/plain_page_test.cpp
+++ b/be/test/olap/rowset/segment_v2/plain_page_test.cpp
@@ -25,7 +25,8 @@
 #include "common/logging.h"
 #include "olap/olap_common.h"
 #include "olap/types.h"
-#include "util/arena.h"
+#include "runtime/mem_pool.h"
+#include "runtime/mem_tracker.h"
 
 namespace doris {
 namespace segment_v2 {
@@ -45,9 +46,10 @@ public:
 
     template<FieldType type, class PageDecoderType>
     void copy_one(PageDecoderType* decoder, typename TypeTraits<type>::CppType* ret) {
-        Arena arena;
+        MemTracker tracker;
+        MemPool pool(&tracker);
         uint8_t null_bitmap = 0;
-        ColumnBlock block(get_type_info(type), (uint8_t*)ret, &null_bitmap, &arena);
+        ColumnBlock block(get_type_info(type), (uint8_t*)ret, &null_bitmap, &pool);
         ColumnBlockView column_block_view(&block);
 
         size_t n = 1;
@@ -74,11 +76,12 @@ public:
         
         ASSERT_EQ(0, page_decoder.current_index());
 
-        Arena arena;
+        MemTracker tracker;
+        MemPool pool(&tracker);
 
-        CppType* values = reinterpret_cast<CppType*>(arena.Allocate(size * sizeof(CppType)));
-        uint8_t* null_bitmap = reinterpret_cast<uint8_t*>(arena.Allocate(BitmapSize(size)));
-        ColumnBlock block(get_type_info(Type), (uint8_t*)values, null_bitmap, &arena);
+        CppType* values = reinterpret_cast<CppType*>(pool.allocate(size * sizeof(CppType)));
+        uint8_t* null_bitmap = reinterpret_cast<uint8_t*>(pool.allocate(BitmapSize(size)));
+        ColumnBlock block(get_type_info(Type), (uint8_t*)values, null_bitmap, &pool);
         ColumnBlockView column_block_view(&block);
         status = page_decoder.next_batch(&size, &column_block_view);
         ASSERT_TRUE(status.ok());
diff --git a/be/test/olap/rowset/segment_v2/rle_page_test.cpp b/be/test/olap/rowset/segment_v2/rle_page_test.cpp
index 9701595..84a6628 100644
--- a/be/test/olap/rowset/segment_v2/rle_page_test.cpp
+++ b/be/test/olap/rowset/segment_v2/rle_page_test.cpp
@@ -22,8 +22,9 @@
 #include "olap/rowset/segment_v2/page_builder.h"
 #include "olap/rowset/segment_v2/page_decoder.h"
 #include "olap/rowset/segment_v2/rle_page.h"
-#include "util/arena.h"
 #include "util/logging.h"
+#include "runtime/mem_tracker.h"
+#include "runtime/mem_pool.h"
 
 using doris::segment_v2::PageBuilderOptions;
 using doris::segment_v2::PageDecoderOptions;
@@ -36,9 +37,10 @@ public:
 
     template<FieldType type, class PageDecoderType>
     void copy_one(PageDecoderType* decoder, typename TypeTraits<type>::CppType* ret) {
-        Arena arena;
+        MemTracker tracker;
+        MemPool pool(&tracker);
         uint8_t null_bitmap = 0;
-        ColumnBlock block(get_type_info(type), (uint8_t*)ret, &null_bitmap, &arena);
+        ColumnBlock block(get_type_info(type), (uint8_t*)ret, &null_bitmap, &pool);
         ColumnBlockView column_block_view(&block);
 
         size_t n = 1;
@@ -66,11 +68,11 @@ public:
         ASSERT_EQ(0, rle_page_decoder.current_index());
         ASSERT_EQ(size, rle_page_decoder.count());
 
-        Arena arena;
-
-        CppType* values = reinterpret_cast<CppType*>(arena.Allocate(size * sizeof(CppType)));
-        uint8_t* null_bitmap = reinterpret_cast<uint8_t*>(arena.Allocate(BitmapSize(size)));
-        ColumnBlock block(get_type_info(Type), (uint8_t*)values, null_bitmap, &arena);
+        MemTracker tracker;
+        MemPool pool(&tracker);
+        CppType* values = reinterpret_cast<CppType*>(pool.allocate(size * sizeof(CppType)));
+        uint8_t* null_bitmap = reinterpret_cast<uint8_t*>(pool.allocate(BitmapSize(size)));
+        ColumnBlock block(get_type_info(Type), (uint8_t*)values, null_bitmap, &pool);
         ColumnBlockView column_block_view(&block);
         size_t size_to_fetch = size;
         status = rle_page_decoder.next_batch(&size_to_fetch, &column_block_view);
diff --git a/be/test/olap/rowset/segment_v2/segment_test.cpp b/be/test/olap/rowset/segment_v2/segment_test.cpp
index 698e3be..d798137 100644
--- a/be/test/olap/rowset/segment_v2/segment_test.cpp
+++ b/be/test/olap/rowset/segment_v2/segment_test.cpp
@@ -32,6 +32,8 @@
 #include "olap/types.h"
 #include "olap/tablet_schema_helper.h"
 #include "util/file_utils.h"
+#include "runtime/mem_pool.h"
+#include "runtime/mem_tracker.h"
 
 namespace doris {
 namespace segment_v2 {
@@ -640,7 +642,8 @@ TEST_F(SegmentReaderWriterTest, TestDefaultValueColumn) {
 
 TEST_F(SegmentReaderWriterTest, TestStringDict) {
     size_t num_rows_per_block = 10;
-    Arena _arena;
+    MemTracker tracker;
+    MemPool pool(&tracker);
 
     std::shared_ptr<TabletSchema> tablet_schema(new TabletSchema());
     tablet_schema->_num_columns = 4;
@@ -677,7 +680,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) {
         for (int j = 0; j < 4; ++j) {
             auto cell = row.cell(j);
             cell.set_not_null();
-            set_column_value_by_type(tablet_schema->_cols[j]._type, i * 10 + j, (char*)cell.mutable_cell_ptr(), &_arena, tablet_schema->_cols[j]._length);
+            set_column_value_by_type(tablet_schema->_cols[j]._type, i * 10 + j, (char*)cell.mutable_cell_ptr(), &pool, tablet_schema->_cols[j]._length);
         }
         Status status = writer.append_row(row);
         ASSERT_TRUE(status.ok());
@@ -722,7 +725,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) {
                         const Slice* actual = reinterpret_cast<const Slice*>(column_block.cell_ptr(i));
 
                         Slice expect;
-                        set_column_value_by_type(tablet_schema->_cols[j]._type, rid * 10 + cid, reinterpret_cast<char*>(&expect), &_arena, tablet_schema->_cols[j]._length);
+                        set_column_value_by_type(tablet_schema->_cols[j]._type, rid * 10 + cid, reinterpret_cast<char*>(&expect), &pool, tablet_schema->_cols[j]._length);
                         ASSERT_EQ(expect.to_string(), actual->to_string());
                     }
                 }
@@ -738,7 +741,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) {
             {
                 auto cell = lower_bound->cell(0);
                 cell.set_not_null();
-                set_column_value_by_type(OLAP_FIELD_TYPE_CHAR, 40970, (char*)cell.mutable_cell_ptr(), &_arena, tablet_schema->_cols[0]._length);
+                set_column_value_by_type(OLAP_FIELD_TYPE_CHAR, 40970, (char*)cell.mutable_cell_ptr(), &pool, tablet_schema->_cols[0]._length);
             }
 
             StorageReadOptions read_opts;
@@ -760,7 +763,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) {
             {
                 auto cell = lower_bound->cell(0);
                 cell.set_not_null();
-                set_column_value_by_type(OLAP_FIELD_TYPE_CHAR, -2, (char*)cell.mutable_cell_ptr(), &_arena, tablet_schema->_cols[0]._length);
+                set_column_value_by_type(OLAP_FIELD_TYPE_CHAR, -2, (char*)cell.mutable_cell_ptr(), &pool, tablet_schema->_cols[0]._length);
             }
 
             std::unique_ptr<RowCursor> upper_bound(new RowCursor());
@@ -768,7 +771,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) {
             {
                 auto cell = upper_bound->cell(0);
                 cell.set_not_null();
-                set_column_value_by_type(OLAP_FIELD_TYPE_CHAR, -1, (char*)cell.mutable_cell_ptr(), &_arena, tablet_schema->_cols[0]._length);
+                set_column_value_by_type(OLAP_FIELD_TYPE_CHAR, -1, (char*)cell.mutable_cell_ptr(), &pool, tablet_schema->_cols[0]._length);
             }
 
             StorageReadOptions read_opts;
@@ -820,7 +823,7 @@ TEST_F(SegmentReaderWriterTest, TestStringDict) {
 
                         const Slice* actual = reinterpret_cast<const Slice*>(column_block.cell_ptr(i));
                         Slice expect;
-                        set_column_value_by_type(tablet_schema->_cols[j]._type, rid * 10 + cid, reinterpret_cast<char*>(&expect), &_arena, tablet_schema->_cols[j]._length);
+                        set_column_value_by_type(tablet_schema->_cols[j]._type, rid * 10 + cid, reinterpret_cast<char*>(&expect), &pool, tablet_schema->_cols[j]._length);
                         ASSERT_EQ(expect.to_string(), actual->to_string()) << "rid:" << rid << ", i:" << i;;
                     }
                 }
diff --git a/be/test/olap/tablet_schema_helper.h b/be/test/olap/tablet_schema_helper.h
index 7e7082f..d0a03b7 100644
--- a/be/test/olap/tablet_schema_helper.h
+++ b/be/test/olap/tablet_schema_helper.h
@@ -20,6 +20,8 @@
 #include <string>
 #include "olap/tablet_schema.h"
 
+#include "runtime/mem_pool.h"
+
 namespace doris {
 
 TabletColumn create_int_key(int32_t id, bool is_nullable = true) {
@@ -79,14 +81,14 @@ TabletColumn create_varchar_key(int32_t id, bool is_nullable = true) {
     return column;
 }
 
-void set_column_value_by_type(FieldType fieldType, int src, char* target, Arena* _arena, size_t _length = 0) {
+void set_column_value_by_type(FieldType fieldType, int src, char* target, MemPool* pool, size_t _length = 0) {
     if (fieldType == OLAP_FIELD_TYPE_CHAR) {
         char* src_value = &std::to_string(src)[0];
         int src_len = strlen(src_value);
 
         auto* dest_slice = (Slice*)target;
         dest_slice->size = _length;
-        dest_slice->data = _arena->Allocate(dest_slice->size);
+        dest_slice->data = (char*)pool->allocate(dest_slice->size);
         memcpy(dest_slice->data, src_value, src_len);
         memset(dest_slice->data + src_len, 0, dest_slice->size - src_len);
     } else if (fieldType == OLAP_FIELD_TYPE_VARCHAR) {
@@ -95,7 +97,7 @@ void set_column_value_by_type(FieldType fieldType, int src, char* target, Arena*
 
         auto* dest_slice = (Slice*)target;
         dest_slice->size = src_len;
-        dest_slice->data = _arena->Allocate(src_len);
+        dest_slice->data = (char*)pool->allocate(src_len);
         std::memcpy(dest_slice->data, src_value, src_len);
     } else {
         *(int*)target = src;
diff --git a/be/test/util/CMakeLists.txt b/be/test/util/CMakeLists.txt
index c9cfb5e..26cd532 100644
--- a/be/test/util/CMakeLists.txt
+++ b/be/test/util/CMakeLists.txt
@@ -37,7 +37,6 @@ ADD_BE_TEST(types_test)
 ADD_BE_TEST(json_util_test)
 ADD_BE_TEST(byte_buffer_test2)
 ADD_BE_TEST(uid_util_test)
-ADD_BE_TEST(arena_test)
 ADD_BE_TEST(aes_util_test)
 ADD_BE_TEST(md5_test)
 ADD_BE_TEST(bitmap_test)
diff --git a/be/test/util/arena_test.cpp b/be/test/util/arena_test.cpp
deleted file mode 100644
index 7194fb2..0000000
--- a/be/test/util/arena_test.cpp
+++ /dev/null
@@ -1,81 +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.
-
-#include <gtest/gtest.h>
-#include "util/arena.h"
-#include "util/random.h"
-
-namespace doris {
-
-class ArenaTest : public testing::Test { };
-
-TEST_F(ArenaTest, Empty) {
-    Arena arena;
-}
-
-TEST_F(ArenaTest, Simple) {
-    std::vector<std::pair<size_t, char*> > allocated;
-    Arena arena;
-    const int N = 100000;
-    size_t bytes = 0;
-    Random rnd(301);
-    for (int i = 0; i < N; i++) {
-        size_t s;
-        if (i % (N / 10) == 0) {
-            s = i;
-        } else {
-            s = rnd.OneIn(4000) ? rnd.Uniform(6000) :
-                (rnd.OneIn(10) ? rnd.Uniform(100) : rnd.Uniform(20));
-        }
-        if (s == 0) {
-            // Our arena disallows size 0 allocations.
-            s = 1;
-        }
-        char* r;
-        if (rnd.OneIn(10)) {
-            r = arena.AllocateAligned(s);
-        } else {
-            r = arena.Allocate(s);
-        }
-
-        for (size_t b = 0; b < s; b++) {
-            // Fill the "i"th allocation with a known bit pattern
-            r[b] = i % 256;
-        }
-        bytes += s;
-        allocated.push_back(std::make_pair(s, r));
-        ASSERT_GE(arena.MemoryUsage(), bytes);
-        if (i > N / 10) {
-            ASSERT_LE(arena.MemoryUsage(), bytes * 1.10);
-        }
-    }
-    for (size_t i = 0; i < allocated.size(); i++) {
-        size_t num_bytes = allocated[i].first;
-        const char* p = allocated[i].second;
-        for (size_t b = 0; b < num_bytes; b++) {
-            // Check the "i"th allocation for the known bit pattern
-            ASSERT_EQ(int(p[b]) & 0xff, i % 256);
-        }
-    }
-}
-
-}  // namespace doris
-
-int main(int argc, char** argv) {
-    ::testing::InitGoogleTest(&argc, argv);
-    return RUN_ALL_TESTS();
-}
diff --git a/run-ut.sh b/run-ut.sh
index 6edfa50..6818f67 100755
--- a/run-ut.sh
+++ b/run-ut.sh
@@ -145,7 +145,6 @@ ${DORIS_TEST_BINARY_DIR}/util/new_metrics_test
 ${DORIS_TEST_BINARY_DIR}/util/doris_metrics_test
 ${DORIS_TEST_BINARY_DIR}/util/system_metrics_test
 ${DORIS_TEST_BINARY_DIR}/util/core_local_test
-${DORIS_TEST_BINARY_DIR}/util/arena_test
 ${DORIS_TEST_BINARY_DIR}/util/types_test
 ${DORIS_TEST_BINARY_DIR}/util/json_util_test
 ${DORIS_TEST_BINARY_DIR}/util/byte_buffer_test2


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